From f683c37c78758926225333c451adf943bc6a3e70 Mon Sep 17 00:00:00 2001 From: dapeng Date: Wed, 18 Mar 2020 15:56:31 +0800 Subject: [PATCH 001/109] =?UTF-8?q?=E7=BB=B4=E8=A1=A8=E5=BC=82=E6=AD=A5cac?= =?UTF-8?q?he=E7=9A=84=E6=8A=BD=E8=B1=A1=E5=92=8C=E5=BC=82=E5=B8=B8?= =?UTF-8?q?=E5=A4=84=E7=90=86?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../side/cassandra/CassandraAsyncReqRow.java | 87 ++++--------- .../flink/sql/side/AbstractSideTableInfo.java | 13 ++ .../flink/sql/side/BaseAsyncReqRow.java | 114 +++++++++++++++++- .../sql/table/AbstractSideTableParser.java | 7 ++ .../Elasticsearch6AsyncReqRow.java | 63 +++------- .../sql/side/hbase/HbaseAsyncReqRow.java | 50 +------- .../flink/sql/side/kudu/KuduAsyncReqRow.java | 77 +++--------- .../sql/side/mongo/MongoAsyncReqRow.java | 56 +++------ 8 files changed, 210 insertions(+), 257 deletions(-) diff --git a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java index e1ed6d8e1..319e82ff3 100644 --- a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java +++ b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java @@ -162,61 +162,13 @@ private void connCassandraDB(CassandraSideTableInfo tableInfo) { } @Override - public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exception { - CRow inputCopy = new CRow(input.row(), input.change()); - JsonArray inputParams = new JsonArray(); - StringBuffer stringBuffer = new StringBuffer(); - String sqlWhere = " where "; - - for (int i = 0; i < sideInfo.getEqualFieldList().size(); i++) { - Integer conValIndex = sideInfo.getEqualValIndex().get(i); - Object equalObj = inputCopy.row().getField(conValIndex); - if (equalObj == null) { - dealMissKey(inputCopy, resultFuture); - return; - } - inputParams.add(equalObj); - StringBuffer sqlTemp = stringBuffer.append(sideInfo.getEqualFieldList().get(i)) - .append(" = "); - if (equalObj instanceof String) { - sqlTemp.append("'" + equalObj + "'") - .append(" and "); - } else { - sqlTemp.append(equalObj) - .append(" and "); - } - - } + public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { String key = buildCacheKey(inputParams); - sqlWhere = sqlWhere + stringBuffer.toString().substring(0, stringBuffer.lastIndexOf(" and ")); - - if (openCache()) { - CacheObj val = getFromCache(key); - if (val != null) { - - if (ECacheContentType.MissVal == val.getType()) { - dealMissKey(inputCopy, resultFuture); - return; - } else if (ECacheContentType.MultiLine == val.getType()) { - List rowList = Lists.newArrayList(); - for (Object jsonArray : (List) val.getContent()) { - Row row = fillData(inputCopy.row(), jsonArray); - rowList.add(new CRow(row, inputCopy.change())); - } - resultFuture.complete(rowList); - } else { - throw new RuntimeException("not support cache obj type " + val.getType()); - } - return; - } - } - //connect Cassandra connCassandraDB(cassandraSideTableInfo); - String sqlCondition = sideInfo.getSqlCondition() + " " + sqlWhere + " ALLOW FILTERING "; - System.out.println("sqlCondition:" + sqlCondition); + String sqlCondition = sideInfo.getSqlCondition() + " " + buildWhereCondition(inputParams) + " ALLOW FILTERING "; ListenableFuture resultSet = Futures.transformAsync(session, new AsyncFunction() { @@ -242,18 +194,18 @@ public void onSuccess(List rows) { List cacheContent = Lists.newArrayList(); List rowList = Lists.newArrayList(); for (com.datastax.driver.core.Row line : rows) { - Row row = fillData(inputCopy.row(), line); + Row row = fillData(input.row(), line); if (openCache()) { cacheContent.add(line); } - rowList.add(new CRow(row,inputCopy.change())); + rowList.add(new CRow(row, input.change())); } resultFuture.complete(rowList); if (openCache()) { putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); } } else { - dealMissKey(inputCopy, resultFuture); + dealMissKey(input, resultFuture); if (openCache()) { putCache(key, CacheMissVal.getMissKeyObj()); } @@ -265,13 +217,30 @@ public void onSuccess(List rows) { public void onFailure(Throwable t) { LOG.error("Failed to retrieve the data: %s%n", t.getMessage()); - System.out.println("Failed to retrieve the data: " + t.getMessage()); cluster.closeAsync(); resultFuture.completeExceptionally(t); } }); } + @Override + public String buildCacheKey(Map inputParams) { + StringBuilder sb = new StringBuilder(); + for (Object ele : inputParams.values()) { + sb.append(ele.toString()).append("_"); + } + return sb.toString(); + } + + private String buildWhereCondition(Map inputParams){ + StringBuilder sb = new StringBuilder(" where "); + for(Map.Entry entry : inputParams.entrySet()){ + Object value = entry.getValue() instanceof String ? "'" + entry.getValue() + "'" : entry.getValue(); + sb.append(String.format("%s = %s", entry.getKey(), value)); + } + return sb.toString(); + } + @Override public Row fillData(Row input, Object line) { com.datastax.driver.core.Row rowArray = (com.datastax.driver.core.Row) line; @@ -306,14 +275,4 @@ public void close() throws Exception { cluster = null; } } - - public String buildCacheKey(JsonArray jsonArray) { - StringBuilder sb = new StringBuilder(); - for (Object ele : jsonArray.getList()) { - sb.append(ele.toString()) - .append("_"); - } - - return sb.toString(); - } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java index cacba74fd..0a6babc8a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java @@ -27,6 +27,7 @@ import java.io.Serializable; import java.util.List; +import java.util.Objects; /** * Reason: @@ -55,6 +56,8 @@ public abstract class AbstractSideTableInfo extends AbstractTableInfo implements public static final String ASYNC_TIMEOUT_NUM_KEY = "asyncTimeoutNum"; + public static final String ASYNC_FAIL_MAX_NUM_KEY = "asyncFailMaxNum"; + private String cacheType = "none";//None or LRU or ALL private int cacheSize = 10000; @@ -71,6 +74,8 @@ public abstract class AbstractSideTableInfo extends AbstractTableInfo implements private String cacheMode="ordered"; + private Integer asyncFailMaxNum; + private List predicateInfoes = Lists.newArrayList(); public RowTypeInfo getRowTypeInfo(){ @@ -156,4 +161,12 @@ public void setAsyncTimeoutNumLimit(int asyncTimeoutNumLimit) { this.asyncTimeoutNumLimit = asyncTimeoutNumLimit; } + public Integer getAsyncFailMaxNum(Integer defaultValue) { + return Objects.isNull(asyncFailMaxNum) ? defaultValue : asyncFailMaxNum; + } + + public void setAsyncFailMaxNum(Integer asyncFailMaxNum) { + this.asyncFailMaxNum = asyncFailMaxNum; + } + } 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 a0deb6b74..ae987e142 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 @@ -20,24 +20,36 @@ package com.dtstack.flink.sql.side; +import com.dtstack.flink.sql.enums.ECacheContentType; import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.metric.MetricConstant; import com.dtstack.flink.sql.side.cache.AbstractSideCache; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.cache.LRUSideCache; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.calcite.sql.JoinType; +import org.apache.commons.collections.MapUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.api.operators.async.queue.StreamRecordQueueEntry; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Collections; -import java.util.concurrent.TimeoutException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; /** * All interfaces inherit naming rules: type + "AsyncReqRow" such as == "MysqlAsyncReqRow @@ -126,7 +138,6 @@ public void timeout(CRow input, ResultFuture resultFuture) throws Exceptio if(timeOutNum % TIMEOUT_LOG_FLUSH_NUM == 0){ LOG.info("Async function call has timed out. input:{}, timeOutNum:{}",input.toString(), timeOutNum); } - timeOutNum ++; if(timeOutNum > sideInfo.getSideTableInfo().getAsyncTimeoutNumLimit()){ resultFuture.completeExceptionally(new Exception("Async function call timedoutNum beyond limit.")); @@ -135,6 +146,105 @@ public void timeout(CRow input, ResultFuture resultFuture) throws Exceptio } } + protected void preInvoke(CRow input, ResultFuture resultFuture){ + ScheduledFuture timeFuture = registerTimer(input, resultFuture); + cancelTimerWhenComplete(resultFuture, timeFuture); + } + + @Override + public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exception { + preInvoke(input, resultFuture); + Map inputParams = parseInputParam(input); + if(MapUtils.isEmpty(inputParams)){ + dealMissKey(input, resultFuture); + return; + } + if(isUseCache(inputParams)){ + invokeWithCache(inputParams, input, resultFuture); + return; + } + handleAsyncInvoke(inputParams, input, resultFuture); + } + + private Map parseInputParam(CRow input){ + Map inputParams = Maps.newHashMap(); + for (int i = 0; i < sideInfo.getEqualValIndex().size(); i++) { + Integer conValIndex = sideInfo.getEqualValIndex().get(i); + Object equalObj = input.row().getField(conValIndex); + if(equalObj == null){ + return inputParams; + } + String columnName = sideInfo.getEqualFieldList().get(conValIndex); + inputParams.put(columnName, equalObj.toString()); + } + return inputParams; + } + protected boolean isUseCache(Map inputParams){ + return openCache() && getFromCache(buildCacheKey(inputParams)) != null; + } + + private void invokeWithCache(Map inputParams, CRow input, ResultFuture resultFuture){ + if (openCache()) { + CacheObj val = getFromCache(buildCacheKey(inputParams)); + if (val != null) { + if (ECacheContentType.MissVal == val.getType()) { + dealMissKey(input, resultFuture); + return; + }else if(ECacheContentType.SingleLine == val.getType()){ + try { + Row row = fillData(input.row(), val); + resultFuture.complete(Collections.singleton(new CRow(row, input.change()))); + } catch (Exception e) { + dealFillDataError(resultFuture, e, input); + } + } else if (ECacheContentType.MultiLine == val.getType()) { + try { + List rowList = Lists.newArrayList(); + for (Object one : (List) val.getContent()) { + Row row = fillData(input.row(), one); + rowList.add(new CRow(row, input.change())); + } + resultFuture.complete(rowList); + } catch (Exception e) { + dealFillDataError(resultFuture, e, input); + } + } else { + resultFuture.completeExceptionally(new RuntimeException("not support cache obj type " + val.getType())); + } + return; + } + } + } + + public abstract void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception; + + public abstract String buildCacheKey(Map inputParams); + + private ProcessingTimeService getProcessingTimeService(){ + return ((StreamingRuntimeContext)this.getRuntimeContext()).getProcessingTimeService(); + } + + protected ScheduledFuture registerTimer(CRow input, ResultFuture resultFuture){ + long timeoutTimestamp = sideInfo.getSideTableInfo().getAsyncTimeout() + getProcessingTimeService().getCurrentProcessingTime(); + return getProcessingTimeService().registerTimer( + timeoutTimestamp, + new ProcessingTimeCallback() { + @Override + public void onProcessingTime(long timestamp) throws Exception { + timeout(input, resultFuture); + } + }); + } + + protected void cancelTimerWhenComplete(ResultFuture resultFuture, ScheduledFuture timerFuture){ + ThreadPoolExecutor executors = new ThreadPoolExecutor(1, 1,0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue()); + if(resultFuture instanceof StreamRecordQueueEntry){ + StreamRecordQueueEntry streamRecordBufferEntry = (StreamRecordQueueEntry) resultFuture; + streamRecordBufferEntry.onComplete((Object value) -> { + timerFuture.cancel(true); + },executors); + } + } protected void dealFillDataError(ResultFuture resultFuture, Exception e, Object sourceData) { LOG.debug("source data {} join side table error ", sourceData); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java index 39b72c22f..83a06d256 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java @@ -114,6 +114,13 @@ protected void parseCacheProp(AbstractSideTableInfo sideTableInfo, Map 0){ + sideTableInfo.setAsyncFailMaxNum(asyncFailNum); + } + } } } } diff --git a/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java b/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java index e8164edb2..f8eb66860 100644 --- a/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java +++ b/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java @@ -81,39 +81,8 @@ public void open(Configuration parameters) throws Exception { @Override - public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exception { - CRow copyCrow = new CRow(input.row(), input.change()); - List inputParams = Lists.newArrayList(); - for (Integer conValIndex : sideInfo.getEqualValIndex()) { - Object equalObj = copyCrow.row().getField(conValIndex); - if (equalObj == null) { - dealMissKey(copyCrow, resultFuture); - return; - } - inputParams.add(equalObj); - } - + public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { String key = buildCacheKey(inputParams); - if (openCache()) { - CacheObj val = getFromCache(key); - if (val != null) { - if (ECacheContentType.MissVal == val.getType()) { - dealMissKey(copyCrow, resultFuture); - return; - } else if (ECacheContentType.MultiLine == val.getType()) { - try { - List rowList = getRows(copyCrow, null, (List) val.getContent()); - resultFuture.complete(rowList); - } catch (Exception e) { - dealFillDataError(resultFuture, e, copyCrow); - } - } else { - resultFuture.completeExceptionally(new RuntimeException("not support cache obj type " + val.getType())); - } - return; - } - } - BoolQueryBuilder boolQueryBuilder = Es6Util.setPredicateclause(sideInfo); boolQueryBuilder = setInputParams(inputParams, boolQueryBuilder); SearchSourceBuilder searchSourceBuilder = initConfiguration(); @@ -135,7 +104,7 @@ public void onResponse(SearchResponse searchResponse) { RestHighLevelClient tmpRhlClient = null; try { while (true) { - loadDataToCache(searchHits, rowList, cacheContent, copyCrow); + loadDataToCache(searchHits, rowList, cacheContent, input); // determine if all results haven been ferched if (searchHits.length < getFetchSize()) { break; @@ -154,7 +123,7 @@ public void onResponse(SearchResponse searchResponse) { dealCacheData(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); resultFuture.complete(rowList); } catch (Exception e) { - dealFillDataError(resultFuture, e, copyCrow); + dealFillDataError(resultFuture, e, input); } finally { if (tmpRhlClient != null) { try { @@ -165,7 +134,7 @@ public void onResponse(SearchResponse searchResponse) { } } } else { - dealMissKey(copyCrow, resultFuture); + dealMissKey(input, resultFuture); dealCacheData(key, CacheMissVal.getMissKeyObj()); } } @@ -177,7 +146,17 @@ public void onFailure(Exception e) { resultFuture.completeExceptionally(new RuntimeException("Response failed!")); } }); + } + + @Override + public String buildCacheKey(Map inputParams) { + StringBuilder sb = new StringBuilder(); + for (Object ele : inputParams.values()) { + sb.append(ele.toString()) + .append("_"); + } + return sb.toString(); } private void loadDataToCache(SearchHit[] searchHits, List rowList, List cacheContent, CRow copyCrow) { @@ -238,16 +217,6 @@ public void close() throws Exception { } - public String buildCacheKey(List equalValList) { - StringBuilder sb = new StringBuilder(); - for (Object ele : equalValList) { - sb.append(ele.toString()) - .append("_"); - } - - return sb.toString(); - } - private SearchSourceBuilder initConfiguration() { SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); searchSourceBuilder.size(getFetchSize()); @@ -258,7 +227,7 @@ private SearchSourceBuilder initConfiguration() { return searchSourceBuilder; } - private BoolQueryBuilder setInputParams(List inputParams, BoolQueryBuilder boolQueryBuilder) { + private BoolQueryBuilder setInputParams(Map inputParams, BoolQueryBuilder boolQueryBuilder) { if (boolQueryBuilder == null) { boolQueryBuilder = new BoolQueryBuilder(); } @@ -266,7 +235,7 @@ private BoolQueryBuilder setInputParams(List inputParams, BoolQueryBuild for (int i = 0; i < sqlJoinCompareOperate.size(); i++) { String fieldName = sideInfo.getEqualFieldList().get(i); String operatorKind = sqlJoinCompareOperate.get(sideInfo.getEqualFieldList().indexOf(fieldName)); - String condition = String.valueOf(inputParams.get(i)); + String condition = String.valueOf(inputParams.get(fieldName)); boolQueryBuilder = Es6Util.buildFilterCondition(boolQueryBuilder, new PredicateInfo(null, operatorKind, null, fieldName, condition), sideInfo); } 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 24ee01786..007de8e15 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 @@ -123,55 +123,17 @@ public void open(Configuration parameters) throws Exception { } @Override - public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exception { - CRow inputCopy = new CRow(input.row(), input.change()); - Map refData = Maps.newHashMap(); - for (int i = 0; i < sideInfo.getEqualValIndex().size(); i++) { - Integer conValIndex = sideInfo.getEqualValIndex().get(i); - Object equalObj = inputCopy.row().getField(conValIndex); - if(equalObj == null){ - dealMissKey(inputCopy, resultFuture); - return; - } - refData.put(sideInfo.getEqualFieldList().get(i), equalObj); - } - - String rowKeyStr = ((HbaseAsyncSideInfo)sideInfo).getRowKeyBuilder().getRowKey(refData); - - //get from cache - if (openCache()) { - CacheObj val = getFromCache(rowKeyStr); - if (val != null) { - if (ECacheContentType.MissVal == val.getType()) { - dealMissKey(inputCopy, resultFuture); - return; - } else if (ECacheContentType.SingleLine == val.getType()) { - try { - Row row = fillData(inputCopy.row(), val); - resultFuture.complete(Collections.singleton(new CRow(row, inputCopy.change()))); - } catch (Exception e) { - dealFillDataError(resultFuture, e, inputCopy); - } - } else if (ECacheContentType.MultiLine == val.getType()) { - try { - for (Object one : (List) val.getContent()) { - Row row = fillData(inputCopy.row(), one); - resultFuture.complete(Collections.singleton(new CRow(row, inputCopy.change()))); - } - } catch (Exception e) { - dealFillDataError(resultFuture, e, inputCopy); - } - } - return; - } - } + public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { + rowKeyMode.asyncGetData(tableName, buildCacheKey(inputParams), input, resultFuture, sideInfo.getSideCache()); + } - rowKeyMode.asyncGetData(tableName, rowKeyStr, inputCopy, resultFuture, sideInfo.getSideCache()); + @Override + public String buildCacheKey(Map inputParams) { + return ((HbaseAsyncSideInfo)sideInfo).getRowKeyBuilder().getRowKey(inputParams); } @Override public Row fillData(Row input, Object sideInput){ - List sideInputList = (List) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ 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 e534a85f9..e3200c293 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 @@ -128,26 +128,18 @@ private void connKuDu() throws KuduException { scannerBuilder.setProjectedColumnNames(projectColumns); } - @Override - public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exception { + public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { CRow inputCopy = new CRow(input.row(), input.change()); - //scannerBuilder 设置为null重新加载过滤条件 + //scannerBuilder 设置为null重新加载过滤条件,然后connkudu重新赋值 + //todo:代码需要优化 scannerBuilder = null; connKuDu(); - JsonArray inputParams = new JsonArray(); Schema schema = table.getSchema(); // @wenbaoup fix bug - for (int i = 0; i < sideInfo.getEqualValIndex().size(); i++) { - Object equalObj = inputCopy.row().getField(sideInfo.getEqualValIndex().get(i)); - if (equalObj == null) { - dealMissKey(inputCopy, resultFuture); - return; - } - //增加过滤条件 - scannerBuilder.addPredicate(KuduPredicate.newInListPredicate(schema.getColumn(sideInfo.getEqualFieldList().get(i)), Collections.singletonList(equalObj))); - inputParams.add(equalObj); - } + inputParams.entrySet().forEach(e ->{ + scannerBuilder.addPredicate(KuduPredicate.newInListPredicate(schema.getColumn(e.getKey()), Collections.singletonList(e.getValue()))); + }); // 填充谓词信息 List predicateInfoes = sideInfo.getSideTableInfo().getPredicateInfoes(); @@ -161,46 +153,23 @@ public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exce }).count(); } - - String key = buildCacheKey(inputParams); - - if (openCache()) { - //判断数据是否已经加载到缓存中 - CacheObj val = getFromCache(key); - if (val != null) { - if (ECacheContentType.MissVal == val.getType()) { - dealMissKey(inputCopy, resultFuture); - return; - } else if (ECacheContentType.SingleLine == val.getType()) { - try { - Row row = fillData(inputCopy.row(), val); - resultFuture.complete(Collections.singleton(new CRow(row, inputCopy.change()))); - } catch (Exception e) { - dealFillDataError(resultFuture, e, inputCopy); - } - } else if (ECacheContentType.MultiLine == val.getType()) { - try { - List rowList = Lists.newArrayList(); - for (Object jsonArray : (List) val.getContent()) { - Row row = fillData(inputCopy.row(), jsonArray); - rowList.add(new CRow(row, inputCopy.change())); - } - resultFuture.complete(rowList); - } catch (Exception e) { - dealFillDataError(resultFuture, e, inputCopy); - } - } else { - resultFuture.completeExceptionally(new RuntimeException("not support cache obj type " + val.getType())); - } - return; - } - } List> cacheContent = Lists.newArrayList(); AsyncKuduScanner asyncKuduScanner = scannerBuilder.build(); List rowList = Lists.newArrayList(); Deferred data = asyncKuduScanner.nextRows(); //从之前的同步修改为调用异步的Callback - data.addCallbackDeferring(new GetListRowCB(inputCopy, cacheContent, rowList, asyncKuduScanner, resultFuture, key)); + data.addCallbackDeferring(new GetListRowCB(inputCopy, cacheContent, rowList, asyncKuduScanner, resultFuture, buildCacheKey(inputParams))); + } + + @Override + public String buildCacheKey(Map inputParams) { + StringBuilder sb = new StringBuilder(); + for (Object ele : inputParams.values()) { + sb.append(ele.toString()) + .append("_"); + } + + return sb.toString(); } @@ -230,16 +199,6 @@ public Row fillData(Row input, Object sideInput) { return row; } - public String buildCacheKey(JsonArray jsonArray) { - StringBuilder sb = new StringBuilder(); - for (Object ele : jsonArray.getList()) { - sb.append(ele.toString()) - .append("_"); - } - - return sb.toString(); - } - @Override public void close() throws Exception { super.close(); diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index d85a9990d..99cc3b2db 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -102,19 +102,11 @@ public void connMongoDb() throws Exception { } @Override - public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exception { + public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { CRow inputCopy = new CRow(input.row(), input.change()); BasicDBObject basicDbObject = new BasicDBObject(); - for (int i = 0; i < sideInfo.getEqualFieldList().size(); i++) { - Integer conValIndex = sideInfo.getEqualValIndex().get(i); - Object equalObj = inputCopy.row().getField(conValIndex); - if (equalObj == null) { - dealMissKey(inputCopy, resultFuture); - return; - } - basicDbObject.put(sideInfo.getEqualFieldList().get(i), equalObj); - } try { + basicDbObject.putAll(inputParams); // 填充谓词 sideInfo.getSideTableInfo().getPredicateInfoes().stream().map(info -> { BasicDBObject filterCondition = MongoUtil.buildFilterObject(info); @@ -127,27 +119,8 @@ public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exce LOG.info("add predicate infoes error ", e); } - String key = buildCacheKey(basicDbObject.values()); - if (openCache()) { - CacheObj val = getFromCache(key); - if (val != null) { - - if (ECacheContentType.MissVal == val.getType()) { - dealMissKey(inputCopy, resultFuture); - return; - } else if (ECacheContentType.MultiLine == val.getType()) { - List rowList = Lists.newArrayList(); - for (Object jsonArray : (List) val.getContent()) { - Row row = fillData(inputCopy.row(), jsonArray); - rowList.add(new CRow(row, inputCopy.change())); - } - resultFuture.complete(rowList); - } else { - throw new RuntimeException("not support cache obj type " + val.getType()); - } - return; - } - } + String key = buildCacheKey(inputParams); + AtomicInteger atomicInteger = new AtomicInteger(0); MongoCollection dbCollection = db.getCollection(mongoSideTableInfo.getTableName(), Document.class); List cacheContent = Lists.newArrayList(); @@ -178,6 +151,17 @@ public void onResult(final Void result, final Throwable t) { dbCollection.find(basicDbObject).forEach(printDocumentBlock, callbackWhenFinished); } + @Override + public String buildCacheKey(Map inputParams) { + StringBuilder sb = new StringBuilder(); + for (Object ele : inputParams.values()) { + sb.append(ele.toString()) + .append("_"); + } + + return sb.toString(); + } + @Override public Row fillData(Row input, Object line) { Document doc = (Document) line; @@ -216,14 +200,4 @@ public void close() throws Exception { } } - public String buildCacheKey(Collection collection) { - StringBuilder sb = new StringBuilder(); - for (Object ele : collection) { - sb.append(ele.toString()) - .append("_"); - } - - return sb.toString(); - } - } From 118d0fd5feef1f97a26be1e271f55f9b6ee9c59a Mon Sep 17 00:00:00 2001 From: dapeng Date: Wed, 18 Mar 2020 15:57:04 +0800 Subject: [PATCH 002/109] =?UTF-8?q?=E8=A1=A5=E5=85=85=E6=8F=90=E4=BA=A4?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../sql/side/operator/SideAsyncOperator.java | 4 +- .../sql/side/rdb/async/RdbAsyncReqRow.java | 164 +++++++++--------- .../sql/side/redis/RedisAsyncReqRow.java | 55 ++---- 3 files changed, 90 insertions(+), 133 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java index 3c2010907..b23de8102 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java @@ -66,10 +66,10 @@ public static DataStream getSideJoinDataStream(DataStream inputStream, String si //TODO How much should be set for the degree of parallelism? Timeout? capacity settings? if (ORDERED.equals(sideTableInfo.getCacheMode())){ - return AsyncDataStream.orderedWait(inputStream, asyncDbReq, sideTableInfo.getAsyncTimeout(), TimeUnit.MILLISECONDS, sideTableInfo.getAsyncCapacity()) + return AsyncDataStream.orderedWait(inputStream, asyncDbReq, -1, TimeUnit.MILLISECONDS, sideTableInfo.getAsyncCapacity()) .setParallelism(sideTableInfo.getParallelism()); }else { - return AsyncDataStream.unorderedWait(inputStream, asyncDbReq, sideTableInfo.getAsyncTimeout(), TimeUnit.MILLISECONDS, sideTableInfo.getAsyncCapacity()) + return AsyncDataStream.unorderedWait(inputStream, asyncDbReq, -1, TimeUnit.MILLISECONDS, sideTableInfo.getAsyncCapacity()) .setParallelism(sideTableInfo.getParallelism()); } 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 f78c30c30..5e5de7e38 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 @@ -25,10 +25,12 @@ import com.dtstack.flink.sql.side.BaseSideInfo; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.rdb.util.SwitchUtil; +import io.vertx.core.json.Json; import io.vertx.core.json.JsonArray; import io.vertx.ext.sql.SQLClient; import io.vertx.ext.sql.SQLConnection; import com.google.common.collect.Lists; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; @@ -37,8 +39,12 @@ import org.slf4j.LoggerFactory; import java.sql.Timestamp; +import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; /** * Date: 2018/11/26 @@ -69,94 +75,45 @@ public class RdbAsyncReqRow extends BaseAsyncReqRow { private transient SQLClient rdbSqlClient; + private Logger logger = LoggerFactory.getLogger(getClass()); + public RdbAsyncReqRow(BaseSideInfo sideInfo) { super(sideInfo); } - @Override - public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exception { - CRow copyCrow = new CRow(input.row(), input.change()); - JsonArray inputParams = new JsonArray(); - for (Integer conValIndex : sideInfo.getEqualValIndex()) { - Object equalObj = copyCrow.row().getField(conValIndex); - if (equalObj == null) { - dealMissKey(copyCrow, resultFuture); - return; - } - inputParams.add(equalObj); - } - String key = buildCacheKey(inputParams); - if (openCache()) { - CacheObj val = getFromCache(key); - if (val != null) { - if (ECacheContentType.MissVal == val.getType()) { - dealMissKey(copyCrow, resultFuture); - return; - } else if (ECacheContentType.MultiLine == val.getType()) { - try { - List rowList = getRows(copyCrow, null, (List) val.getContent()); - resultFuture.complete(rowList); - } catch (Exception e) { - dealFillDataError(resultFuture, e, copyCrow); - } - } else { - resultFuture.completeExceptionally(new RuntimeException("not support cache obj type " + val.getType())); - } - return; - } - } + @Override + protected void preInvoke(CRow input, ResultFuture resultFuture){ - rdbSqlClient.getConnection(conn -> { - if (conn.failed()) { - //Treatment failures - resultFuture.completeExceptionally(conn.cause()); - return; - } + } - final SQLConnection connection = conn.result(); - String sqlCondition = sideInfo.getSqlCondition(); - connection.queryWithParams(sqlCondition, inputParams, rs -> { - if (rs.failed()) { - LOG.error("Cannot retrieve the data from the database", rs.cause()); - resultFuture.completeExceptionally(rs.cause()); - return; - } - List cacheContent = Lists.newArrayList(); - List results = rs.result().getResults(); - if (results.size() > 0) { - try { - List rowList = getRows(copyCrow, cacheContent, results); - dealCacheData(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); - resultFuture.complete(rowList); - } catch (Exception e){ - dealFillDataError(resultFuture, e, copyCrow); + @Override + public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { + AtomicBoolean connectFinish = new AtomicBoolean(false); + AtomicInteger counter = new AtomicInteger(0); + while(!connectFinish.get()){ + rdbSqlClient.getConnection(conn -> { + if(conn.failed()){ + if(counter.get() % 1000 == 0){ + logger.error("getConnection error", conn.cause()); } - } else { - dealMissKey(copyCrow, resultFuture); - dealCacheData(key, CacheMissVal.getMissKeyObj()); - } - - // and close the connection - connection.close(done -> { - if (done.failed()) { - throw new RuntimeException(done.cause()); + if(counter.incrementAndGet() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3)){ + resultFuture.completeExceptionally(conn.cause()); + connectFinish.set(true); + return; } - }); + } + connectFinish.set(true); + ScheduledFuture timerFuture = registerTimer(input, resultFuture); + cancelTimerWhenComplete(resultFuture, timerFuture); + handleQuery(conn.result(), inputParams, input, resultFuture); }); - }); + } } - protected List getRows(CRow inputRow, List cacheContent, List results) { - List rowList = Lists.newArrayList(); - for (JsonArray line : results) { - Row row = fillData(inputRow.row(), line); - if (null != cacheContent && openCache()) { - cacheContent.add(line); - } - rowList.add(new CRow(row, inputRow.change())); - } - return rowList; + @Override + public String buildCacheKey(Map inputParam) { + return StringUtils.join(inputParam.values(),"_"); } @Override @@ -195,18 +152,53 @@ public void close() throws Exception { } - public String buildCacheKey(JsonArray jsonArray) { - StringBuilder sb = new StringBuilder(); - for (Object ele : jsonArray.getList()) { - sb.append(ele.toString()) - .append("_"); - } - - return sb.toString(); - } - public void setRdbSqlClient(SQLClient rdbSqlClient) { this.rdbSqlClient = rdbSqlClient; } + 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 -> { + if (rs.failed()) { + LOG.error("Cannot retrieve the data from the database", rs.cause()); + resultFuture.completeExceptionally(rs.cause()); + return; + } + + List cacheContent = Lists.newArrayList(); + + int resultSize = rs.result().getResults().size(); + if (resultSize > 0) { + List rowList = Lists.newArrayList(); + + for (JsonArray line : rs.result().getResults()) { + Row row = fillData(input.row(), line); + if (openCache()) { + cacheContent.add(line); + } + rowList.add(new CRow(row, input.change())); + } + + if (openCache()) { + putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); + } + + resultFuture.complete(rowList); + } else { + dealMissKey(input, resultFuture); + if (openCache()) { + putCache(key, CacheMissVal.getMissKeyObj()); + } + } + + // and close the connection + connection.close(done -> { + if (done.failed()) { + throw new RuntimeException(done.cause()); + } + }); + }); + } + } 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 d2a2aed56..b6b6c5a9d 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 @@ -126,49 +126,13 @@ public Row fillData(Row input, Object sideInput) { } @Override - public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exception { - CRow inputCopy = new CRow(input.row(),input.change()); - List keyData = Lists.newLinkedList(); - for (int i = 0; i < sideInfo.getEqualValIndex().size(); i++) { - Integer conValIndex = sideInfo.getEqualValIndex().get(i); - Object equalObj = inputCopy.row().getField(conValIndex); - if(equalObj == null){ - dealMissKey(inputCopy, resultFuture); - return; - } - String value = equalObj.toString(); - keyData.add(sideInfo.getEqualFieldList().get(i)); - keyData.add(value); - } - - String key = buildCacheKey(keyData); - - if(openCache()){ - CacheObj val = getFromCache(key); - if(val != null){ - if(ECacheContentType.MissVal == val.getType()){ - dealMissKey(inputCopy, resultFuture); - return; - }else if(ECacheContentType.MultiLine == val.getType()){ - try { - Row row = fillData(inputCopy.row(), val.getContent()); - resultFuture.complete(Collections.singleton(new CRow(row, input.change()))); - } catch (Exception e) { - dealFillDataError(resultFuture, e, inputCopy); - } - }else{ - RuntimeException exception = new RuntimeException("not support cache obj type " + val.getType()); - resultFuture.completeExceptionally(exception); - } - return; - } - } - + public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { + String key = buildCacheKey(inputParams); Map keyValue = Maps.newHashMap(); List value = async.keys(key + ":*").get(); String[] values = value.toArray(new String[value.size()]); if (values.length == 0) { - dealMissKey(inputCopy, resultFuture); + dealMissKey(input, resultFuture); } else { RedisFuture>> future = ((RedisStringAsyncCommands) async).mget(values); future.thenAccept(new Consumer>>() { @@ -181,14 +145,14 @@ public void accept(List> keyValues) { keyValue.put(splitKeys[3], keyValues.get(i).getValue()); } try { - Row row = fillData(inputCopy.row(), keyValue); + Row row = fillData(input.row(), keyValue); dealCacheData(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, keyValue)); - resultFuture.complete(Collections.singleton(new CRow(row, inputCopy.change()))); + resultFuture.complete(Collections.singleton(new CRow(row, input.change()))); } catch (Exception e) { - dealFillDataError(resultFuture, e, inputCopy); + dealFillDataError(resultFuture, e, input); } } else { - dealMissKey(inputCopy, resultFuture); + dealMissKey(input, resultFuture); dealCacheData(key, CacheMissVal.getMissKeyObj()); } } @@ -196,8 +160,9 @@ public void accept(List> keyValues) { } } - private String buildCacheKey(List keyData) { - String kv = String.join(":", keyData); + @Override + public String buildCacheKey(Map inputParams) { + String kv = StringUtils.join(inputParams.values(), ":"); String tableName = redisSideTableInfo.getTableName(); StringBuilder preKey = new StringBuilder(); preKey.append(tableName).append(":").append(kv); From 615ea8363bf60c1ee57c586a0fc64720a6fc8535 Mon Sep 17 00:00:00 2001 From: dapeng Date: Fri, 20 Mar 2020 15:55:18 +0800 Subject: [PATCH 003/109] =?UTF-8?q?=E5=BC=82=E5=B8=B8=E9=87=8D=E8=AF=95?= =?UTF-8?q?=E9=80=BB=E8=BE=91=E4=BF=AE=E6=94=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/side/BaseAsyncReqRow.java | 10 ++++++-- .../sql/side/rdb/async/RdbAsyncReqRow.java | 23 +++++++++++-------- 2 files changed, 21 insertions(+), 12 deletions(-) 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 ae987e142..221bf2c85 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 @@ -30,6 +30,7 @@ import com.google.common.collect.Maps; import org.apache.calcite.sql.JoinType; import org.apache.commons.collections.MapUtils; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; import org.apache.flink.streaming.api.functions.async.ResultFuture; @@ -62,6 +63,7 @@ public abstract class BaseAsyncReqRow extends RichAsyncFunction implements ISideReqRow { private static final Logger LOG = LoggerFactory.getLogger(BaseAsyncReqRow.class); private static final long serialVersionUID = 2098635244857937717L; + private RuntimeContext runtimeContext; private static int TIMEOUT_LOG_FLUSH_NUM = 10; private int timeOutNum = 0; @@ -72,7 +74,11 @@ public abstract class BaseAsyncReqRow extends RichAsyncFunction impl public BaseAsyncReqRow(BaseSideInfo sideInfo){ this.sideInfo = sideInfo; } - + @Override + public void setRuntimeContext(RuntimeContext runtimeContext) { + super.setRuntimeContext(runtimeContext); + this.runtimeContext = runtimeContext; + } @Override public void open(Configuration parameters) throws Exception { super.open(parameters); @@ -221,7 +227,7 @@ private void invokeWithCache(Map inputParams, CRow input, Result public abstract String buildCacheKey(Map inputParams); private ProcessingTimeService getProcessingTimeService(){ - return ((StreamingRuntimeContext)this.getRuntimeContext()).getProcessingTimeService(); + return ((StreamingRuntimeContext)this.runtimeContext).getProcessingTimeService(); } protected ScheduledFuture registerTimer(CRow input, ResultFuture resultFuture){ 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 5e5de7e38..6b4298f24 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 @@ -89,25 +89,28 @@ protected void preInvoke(CRow input, ResultFuture resultFuture){ @Override public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { - AtomicBoolean connectFinish = new AtomicBoolean(false); - AtomicInteger counter = new AtomicInteger(0); - while(!connectFinish.get()){ + AtomicInteger failCounter = new AtomicInteger(0); + while(true){ + AtomicBoolean connectFinish = new AtomicBoolean(false); rdbSqlClient.getConnection(conn -> { + connectFinish.set(true); if(conn.failed()){ - if(counter.get() % 1000 == 0){ + if(failCounter.getAndIncrement() % 1000 == 0){ logger.error("getConnection error", conn.cause()); } - if(counter.incrementAndGet() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3)){ - resultFuture.completeExceptionally(conn.cause()); - connectFinish.set(true); - return; - } + conn.result().close(); } - connectFinish.set(true); ScheduledFuture timerFuture = registerTimer(input, resultFuture); cancelTimerWhenComplete(resultFuture, timerFuture); handleQuery(conn.result(), inputParams, input, resultFuture); }); + while(!connectFinish.get()){ + Thread.sleep(50); + } + if(failCounter.get() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3)){ + resultFuture.completeExceptionally(new RuntimeException("connection fail")); + return; + } } } From a78738963037df956cacf51bdca24f58b5f3165e Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 24 Mar 2020 15:51:38 +0800 Subject: [PATCH 004/109] =?UTF-8?q?=E8=A1=A5=E5=85=85=E7=BB=B4=E8=A1=A8?= =?UTF-8?q?=E8=B6=85=E6=97=B6=E8=BE=93=E5=87=BAleft=20join?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flink/sql/side/BaseAsyncReqRow.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) 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 221bf2c85..8713b541a 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 @@ -145,11 +145,15 @@ public void timeout(CRow input, ResultFuture resultFuture) throws Exceptio LOG.info("Async function call has timed out. input:{}, timeOutNum:{}",input.toString(), timeOutNum); } timeOutNum ++; - if(timeOutNum > sideInfo.getSideTableInfo().getAsyncTimeoutNumLimit()){ - resultFuture.completeExceptionally(new Exception("Async function call timedoutNum beyond limit.")); - } else { + if(sideInfo.getJoinType() == JoinType.LEFT){ resultFuture.complete(null); + return; + } + if(timeOutNum > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Integer.MAX_VALUE)){ + resultFuture.completeExceptionally(new Exception("Async function call timedoutNum beyond limit.")); + return; } + resultFuture.complete(null); } protected void preInvoke(CRow input, ResultFuture resultFuture){ From 23b410c04682342f37be1b785e10f14640e97b1e Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 24 Mar 2020 16:03:52 +0800 Subject: [PATCH 005/109] =?UTF-8?q?=E8=BE=93=E5=87=BA=E5=A0=86=E6=A0=88?= =?UTF-8?q?=E4=BF=A1=E6=81=AF?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) 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 6b4298f24..10b9d93db 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 @@ -30,6 +30,7 @@ import io.vertx.ext.sql.SQLClient; import io.vertx.ext.sql.SQLConnection; import com.google.common.collect.Lists; +import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.table.runtime.types.CRow; @@ -45,6 +46,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; /** * Date: 2018/11/26 @@ -90,6 +92,7 @@ protected void preInvoke(CRow input, ResultFuture resultFuture){ @Override public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { AtomicInteger failCounter = new AtomicInteger(0); + AtomicReference connErrMsg = new AtomicReference<>(); while(true){ AtomicBoolean connectFinish = new AtomicBoolean(false); rdbSqlClient.getConnection(conn -> { @@ -98,17 +101,20 @@ public void handleAsyncInvoke(Map inputParams, CRow input, Resul if(failCounter.getAndIncrement() % 1000 == 0){ logger.error("getConnection error", conn.cause()); } + connErrMsg.set(conn.cause()); conn.result().close(); } ScheduledFuture timerFuture = registerTimer(input, resultFuture); cancelTimerWhenComplete(resultFuture, timerFuture); handleQuery(conn.result(), inputParams, input, resultFuture); }); + while(!connectFinish.get()){ Thread.sleep(50); } + if(failCounter.get() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3)){ - resultFuture.completeExceptionally(new RuntimeException("connection fail")); + resultFuture.completeExceptionally(connErrMsg.get()); return; } } From bfce8c88b479658f9ab02d7b303fb178b6089815 Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 24 Mar 2020 16:26:47 +0800 Subject: [PATCH 006/109] =?UTF-8?q?=E4=BF=AE=E5=A4=8D=E9=80=80=E5=87=BA?= =?UTF-8?q?=E9=80=BB=E8=BE=91?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/side/rdb/async/RdbAsyncReqRow.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) 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 10b9d93db..c48635bf3 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 @@ -93,30 +93,32 @@ protected void preInvoke(CRow input, ResultFuture resultFuture){ public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { AtomicInteger failCounter = new AtomicInteger(0); AtomicReference connErrMsg = new AtomicReference<>(); - while(true){ + AtomicBoolean finishFlag = new AtomicBoolean(false); + while(!finishFlag.get()){ AtomicBoolean connectFinish = new AtomicBoolean(false); rdbSqlClient.getConnection(conn -> { - connectFinish.set(true); if(conn.failed()){ + connectFinish.set(true); if(failCounter.getAndIncrement() % 1000 == 0){ logger.error("getConnection error", conn.cause()); } + if(failCounter.get() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3)){ + resultFuture.completeExceptionally(connErrMsg.get()); + finishFlag.set(true); + } connErrMsg.set(conn.cause()); conn.result().close(); + return; } ScheduledFuture timerFuture = registerTimer(input, resultFuture); cancelTimerWhenComplete(resultFuture, timerFuture); handleQuery(conn.result(), inputParams, input, resultFuture); + finishFlag.set(true); }); while(!connectFinish.get()){ Thread.sleep(50); } - - if(failCounter.get() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3)){ - resultFuture.completeExceptionally(connErrMsg.get()); - return; - } } } From 267511a2659e2fbc7ba162293af75f6333f7f964 Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 24 Mar 2020 16:27:36 +0800 Subject: [PATCH 007/109] =?UTF-8?q?=E4=BF=AE=E5=A4=8D=E9=80=80=E5=87=BA?= =?UTF-8?q?=E9=80=BB=E8=BE=91?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) 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 c48635bf3..827d94dad 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 @@ -92,7 +92,6 @@ protected void preInvoke(CRow input, ResultFuture resultFuture){ @Override public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { AtomicInteger failCounter = new AtomicInteger(0); - AtomicReference connErrMsg = new AtomicReference<>(); AtomicBoolean finishFlag = new AtomicBoolean(false); while(!finishFlag.get()){ AtomicBoolean connectFinish = new AtomicBoolean(false); @@ -103,10 +102,9 @@ public void handleAsyncInvoke(Map inputParams, CRow input, Resul logger.error("getConnection error", conn.cause()); } if(failCounter.get() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3)){ - resultFuture.completeExceptionally(connErrMsg.get()); + resultFuture.completeExceptionally(conn.cause()); finishFlag.set(true); } - connErrMsg.set(conn.cause()); conn.result().close(); return; } From 4e8a00e4ccf12b865f163e2d87a923be53cb9fce Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 24 Mar 2020 17:39:43 +0800 Subject: [PATCH 008/109] =?UTF-8?q?=E9=98=BB=E5=A1=9E=E9=80=BB=E8=BE=91?= =?UTF-8?q?=E9=87=8D=E6=96=B0=E8=B0=83=E6=95=B4?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../sql/side/rdb/async/RdbAsyncReqRow.java | 36 ++++++++++++++++--- 1 file changed, 32 insertions(+), 4 deletions(-) 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 827d94dad..011920afe 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 @@ -43,6 +43,7 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.concurrent.Future; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -77,6 +78,8 @@ public class RdbAsyncReqRow extends BaseAsyncReqRow { private transient SQLClient rdbSqlClient; + private final static AtomicBoolean CONN_STATUS = new AtomicBoolean(true); + private Logger logger = LoggerFactory.getLogger(getClass()); public RdbAsyncReqRow(BaseSideInfo sideInfo) { @@ -91,13 +94,32 @@ protected void preInvoke(CRow input, ResultFuture resultFuture){ @Override public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { + + while (!CONN_STATUS.get()){//network is unhealth + Thread.sleep(100); + } + rdbSqlClient.getConnection(conn -> { + if(conn.failed()){ + CONN_STATUS.set(false); + connectWithRetry(inputParams, input, resultFuture, rdbSqlClient); + return; + } + CONN_STATUS.set(true); + ScheduledFuture timerFuture = registerTimer(input, resultFuture); + cancelTimerWhenComplete(resultFuture, timerFuture); + handleQuery(conn.result(), inputParams, input, resultFuture); + }); + + } + + private void connectWithRetry(Map inputParams, CRow input, ResultFuture resultFuture, SQLClient rdbSqlClient) { AtomicInteger failCounter = new AtomicInteger(0); AtomicBoolean finishFlag = new AtomicBoolean(false); while(!finishFlag.get()){ - AtomicBoolean connectFinish = new AtomicBoolean(false); + AtomicBoolean connFinish = new AtomicBoolean(false); rdbSqlClient.getConnection(conn -> { + connFinish.set(true); if(conn.failed()){ - connectFinish.set(true); if(failCounter.getAndIncrement() % 1000 == 0){ logger.error("getConnection error", conn.cause()); } @@ -108,16 +130,22 @@ public void handleAsyncInvoke(Map inputParams, CRow input, Resul conn.result().close(); return; } + CONN_STATUS.set(true); ScheduledFuture timerFuture = registerTimer(input, resultFuture); cancelTimerWhenComplete(resultFuture, timerFuture); handleQuery(conn.result(), inputParams, input, resultFuture); finishFlag.set(true); }); + while(!connFinish.get()){ + try { + Thread.sleep(50); + } catch (InterruptedException e){ + logger.error("", e); + } - while(!connectFinish.get()){ - Thread.sleep(50); } } + } @Override From c895ba2d0961d5ec07e632bc8f2de6fe06c3fcab Mon Sep 17 00:00:00 2001 From: dapeng Date: Mon, 30 Mar 2020 09:56:58 +0800 Subject: [PATCH 009/109] =?UTF-8?q?=E4=BF=AE=E5=A4=8Dkafka-source=20?= =?UTF-8?q?=E4=BD=9C=E4=B8=BA=E6=95=B0=E6=8D=AE=E6=BA=90=E9=94=99=E8=AF=AF?= =?UTF-8?q?=EF=BC=8Ctype=3D'kakfa'?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../source/kafka/KafkaDeserializationMetricWrapper.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaDeserializationMetricWrapper.java b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaDeserializationMetricWrapper.java index afa950c5b..d0d76747d 100644 --- a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaDeserializationMetricWrapper.java +++ b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaDeserializationMetricWrapper.java @@ -76,8 +76,13 @@ protected void beforeDeserialize() throws IOException { } protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + Field consumerThreadField = null; + if(fetcher.getClass().getDeclaredField("consumerThread") != null){ + consumerThreadField = fetcher.getClass().getDeclaredField("consumerThread"); + } else { + consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + } - Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); consumerThreadField.setAccessible(true); KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); From a8947ec9f4740494432542ec0ac8c947de006ef4 Mon Sep 17 00:00:00 2001 From: dapeng Date: Mon, 30 Mar 2020 11:46:44 +0800 Subject: [PATCH 010/109] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E9=98=BB=E5=A1=9E?= =?UTF-8?q?=E9=80=BB=E8=BE=91?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../sql/side/rdb/async/RdbAsyncReqRow.java | 29 +++++++++++-------- 1 file changed, 17 insertions(+), 12 deletions(-) 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 011920afe..da6cb3876 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 @@ -43,6 +43,7 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.atomic.AtomicBoolean; @@ -116,9 +117,8 @@ private void connectWithRetry(Map inputParams, CRow input, Resul AtomicInteger failCounter = new AtomicInteger(0); AtomicBoolean finishFlag = new AtomicBoolean(false); while(!finishFlag.get()){ - AtomicBoolean connFinish = new AtomicBoolean(false); + CountDownLatch latch = new CountDownLatch(1); rdbSqlClient.getConnection(conn -> { - connFinish.set(true); if(conn.failed()){ if(failCounter.getAndIncrement() % 1000 == 0){ logger.error("getConnection error", conn.cause()); @@ -127,22 +127,27 @@ private void connectWithRetry(Map inputParams, CRow input, Resul resultFuture.completeExceptionally(conn.cause()); finishFlag.set(true); } + latch.countDown(); conn.result().close(); return; } - CONN_STATUS.set(true); - ScheduledFuture timerFuture = registerTimer(input, resultFuture); - cancelTimerWhenComplete(resultFuture, timerFuture); - handleQuery(conn.result(), inputParams, input, resultFuture); - finishFlag.set(true); - }); - while(!connFinish.get()){ try { - Thread.sleep(50); - } catch (InterruptedException e){ + CONN_STATUS.set(true); + ScheduledFuture timerFuture = registerTimer(input, resultFuture); + cancelTimerWhenComplete(resultFuture, timerFuture); + handleQuery(conn.result(), inputParams, input, resultFuture); + finishFlag.set(true); + } catch (Exception e) { logger.error("", e); + } finally { + latch.countDown(); } - + }); + //主线程阻塞 + try { + latch.wait(); + } catch (InterruptedException e) { + logger.error("", e); } } From e04f99b102b962fb45c2903bccaefd6adc0844ba Mon Sep 17 00:00:00 2001 From: dapeng Date: Mon, 30 Mar 2020 14:44:55 +0800 Subject: [PATCH 011/109] =?UTF-8?q?hbase=20=E7=BB=93=E6=9E=9C=E8=A1=A8?= =?UTF-8?q?=E6=94=AF=E6=8C=81=20rowKey?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../sink/hbase/AbstractReplaceOperator.java | 58 +++++++ .../sql/sink/hbase/HbaseOutputFormat.java | 37 ++--- .../sql/sink/hbase/Md5ReplaceOperator.java | 43 +++++ .../flink/sql/sink/hbase/ReplaceInfo.java | 74 +++++++++ .../flink/sql/sink/hbase/RowKeyBuilder.java | 148 ++++++++++++++++++ .../sql/sink/hbase/enums/EReplaceOpType.java | 39 +++++ .../sql/sink/hbase/enums/EReplaceType.java | 43 +++++ 7 files changed, 419 insertions(+), 23 deletions(-) create mode 100644 hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/AbstractReplaceOperator.java create mode 100644 hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/Md5ReplaceOperator.java create mode 100644 hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/ReplaceInfo.java create mode 100644 hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/RowKeyBuilder.java create mode 100644 hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/enums/EReplaceOpType.java create mode 100644 hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/enums/EReplaceType.java diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/AbstractReplaceOperator.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/AbstractReplaceOperator.java new file mode 100644 index 000000000..c3e5701aa --- /dev/null +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/AbstractReplaceOperator.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + + +package com.dtstack.flink.sql.sink.hbase; + + +import com.dtstack.flink.sql.sink.hbase.enums.EReplaceOpType; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/8/23 + * Company: www.dtstack.com + * @author xuchao + */ + +public abstract class AbstractReplaceOperator { + + private EReplaceOpType opType; + + public AbstractReplaceOperator(EReplaceOpType opType){ + this.opType = opType; + } + + public String doOperator(Map refData){ + String replaceStr = replaceStr(refData); + return doFunc(replaceStr); + } + + public String replaceStr(Map refData){ + return ""; + } + + /** + * The processing function to provide custom + * @param replaceStr + * @return + */ + abstract String doFunc(String replaceStr); +} 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 382e8f975..1769fbda7 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 @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.enums.EUpdateMode; import com.dtstack.flink.sql.outputformat.AbstractDtRichOutputFormat; 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 org.apache.flink.configuration.Configuration; @@ -40,7 +41,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.text.SimpleDateFormat; import java.util.List; import java.util.Map; import java.util.Set; @@ -69,8 +69,6 @@ public class HbaseOutputFormat extends AbstractDtRichOutputFormat { private transient Connection conn; private transient Table table; - public final SimpleDateFormat ROWKEY_DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss"); - @Override public void configure(Configuration parameters) { LOG.warn("---configure---"); @@ -178,30 +176,23 @@ private String buildRowKey(Row record) { private List getRowKeyValues(Row record) { List rowKeyValues = Lists.newArrayList(); - for (int i = 0; i < rowkey.length; ++i) { - String colName = rowkey[i]; - int rowKeyIndex = 0; - for (; rowKeyIndex < columnNames.length; ++rowKeyIndex) { - if (columnNames[rowKeyIndex].equals(colName)) { - break; - } - } - - if (rowKeyIndex != columnNames.length && record.getField(rowKeyIndex) != null) { - Object field = record.getField(rowKeyIndex); - if (field == null) { - continue; - } else if (field instanceof java.util.Date) { - java.util.Date d = (java.util.Date) field; - rowKeyValues.add(ROWKEY_DATE_FORMAT.format(d)); - } else { - rowKeyValues.add(field.toString()); - } - } + Map row = rowConvertMap(record); + for (String key : rowkey) { + RowKeyBuilder rowKeyBuilder = new RowKeyBuilder(); + rowKeyBuilder.init(key); + rowKeyValues.add(rowKeyBuilder.getRowKey(row)); } return rowKeyValues; } + private Map rowConvertMap(Row record){ + Map rowValue = Maps.newHashMap(); + for(int i = 0; i < columnNames.length; i++){ + rowValue.put(columnNames[i], record.getField(i)); + } + return rowValue; + } + @Override public void close() throws IOException { if (conn != null) { diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/Md5ReplaceOperator.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/Md5ReplaceOperator.java new file mode 100644 index 000000000..e9474220a --- /dev/null +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/Md5ReplaceOperator.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + + +package com.dtstack.flink.sql.sink.hbase; + +import com.dtstack.flink.sql.sink.hbase.enums.EReplaceOpType; +import com.dtstack.flink.sql.util.MD5Utils; + +/** + * Reason: + * Date: 2018/8/23 + * Company: www.dtstack.com + * @author xuchao + */ + +public class Md5ReplaceOperator extends AbstractReplaceOperator { + + public Md5ReplaceOperator(EReplaceOpType opType) { + super(opType); + } + + @Override + String doFunc(String replaceStr) { + return MD5Utils.getMD5String(replaceStr); + } +} diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/ReplaceInfo.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/ReplaceInfo.java new file mode 100644 index 000000000..c233bfe6b --- /dev/null +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/ReplaceInfo.java @@ -0,0 +1,74 @@ +/* + * 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.hbase; + + +import com.dtstack.flink.sql.sink.hbase.enums.EReplaceType; + +import java.io.Serializable; +import java.util.List; + +/** + * Reason: + * Date: 2018/8/23 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class ReplaceInfo implements Serializable { + + private static final long serialVersionUID = 2058635242957737717L; + + private EReplaceType type; + + private String param; + + private List subReplaceInfos; + + public ReplaceInfo(EReplaceType type){ + this.type = type; + } + + public EReplaceType getType() { + return type; + } + + public void setType(EReplaceType type) { + this.type = type; + } + + public String getParam() { + return param; + } + + public void setParam(String param) { + this.param = param; + } + + public List getSubReplaceInfos() { + return subReplaceInfos; + } + + public void setSubReplaceInfos(List subReplaceInfos) { + this.subReplaceInfos = subReplaceInfos; + } +} 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 new file mode 100644 index 000000000..89667ee28 --- /dev/null +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/RowKeyBuilder.java @@ -0,0 +1,148 @@ +/* + * 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.hbase; + +import com.dtstack.flink.sql.sink.hbase.enums.EReplaceType; +import com.dtstack.flink.sql.util.MD5Utils; +import com.google.common.collect.Lists; +import org.apache.commons.collections.CollectionUtils; + +import javax.xml.crypto.Data; +import java.io.Serializable; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * rowkey rule + * Date: 2018/8/23 + * Company: www.dtstack.com + * @author xuchao + */ + +public class RowKeyBuilder implements Serializable{ + + private static final long serialVersionUID = 2058635242857937717L; + + private static Pattern Md5Operator = Pattern.compile("(?i)^md5\\(\\s*(.*)\\s*\\)$"); + + private List operatorChain = Lists.newArrayList(); + + public void init(String rowKeyTempl){ + operatorChain.addAll(makeFormula(rowKeyTempl)); + } + + /** + * + * @param refData + * @return + */ + public String getRowKey(Map refData){ + return buildStr(operatorChain, refData); + } + + + + private String buildStr(List fieldList, Map refData){ + if(CollectionUtils.isEmpty(fieldList)){ + return ""; + } + StringBuffer sb = new StringBuffer(""); + for(ReplaceInfo replaceInfo : fieldList){ + + if(replaceInfo.getType() == EReplaceType.CONSTANT){ + sb.append(replaceInfo.getParam()); + continue; + } + + if(replaceInfo.getType() == EReplaceType.FUNC){ + sb.append(MD5Utils.getMD5String(buildStr(replaceInfo.getSubReplaceInfos(), refData))); + continue; + } + String replaceName = replaceInfo.getParam(); + if(!refData.containsKey(replaceName)){ + throw new RuntimeException(String.format("build rowKey with field %s which value not found.", replaceName)); + } + Object value = refData.get(replaceName); + if(value instanceof Date){ + SimpleDateFormat sdf = new SimpleDateFormat("yyyyMMddHHmmss"); + java.util.Date d = (java.util.Date) value; + sb.append(sdf.format(d)); + } else { + sb.append(value); + } + } + + return sb.toString(); + } + + public static String[] splitIgnoreQuotaBrackets(String str, String delimiter){ + String splitPatternStr = delimiter + "(?![^()]*+\\))(?![^{}]*+})(?![^\\[\\]]*+\\])"; + return str.split(splitPatternStr); + } + + /** + * + * @param field + * @return + */ + 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("'")){ + ReplaceInfo replaceInfo = new ReplaceInfo(EReplaceType.CONSTANT); + field = field.substring(1, field.length() - 1); + replaceInfo.setParam(field); + return replaceInfo; + } + + ReplaceInfo replaceInfo = new ReplaceInfo(EReplaceType.PARAM); + replaceInfo.setParam(field); + return replaceInfo; + } + + private List makeFormula(String formula){ + if(formula == null || formula.length() <= 0){ + Lists.newArrayList(); + } + List result = Lists.newArrayList(); + for(String meta: splitIgnoreQuotaBrackets(formula, "\\+")){ + Matcher matcher = Md5Operator.matcher(meta.trim()); + if(matcher.find()){ + ReplaceInfo replaceInfo = new ReplaceInfo(EReplaceType.FUNC); + replaceInfo.setSubReplaceInfos(makeFormula(matcher.group(1))); + result.add(replaceInfo); + } else { + result.add(getReplaceInfo(meta)); + } + } + return result; + } +} diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/enums/EReplaceOpType.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/enums/EReplaceOpType.java new file mode 100644 index 000000000..8621f352d --- /dev/null +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/enums/EReplaceOpType.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.sink.hbase.enums; + +/** + * Reason: + * Date: 2018/8/23 + * Company: www.dtstack.com + * @author xuchao + */ + +public enum EReplaceOpType { + /** + * 没有func + */ + NO_FUNC, + /** + * md5 func + */ + MD5_FUNC; +} diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/enums/EReplaceType.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/enums/EReplaceType.java new file mode 100644 index 000000000..ad6bbcc4e --- /dev/null +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/enums/EReplaceType.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + + +package com.dtstack.flink.sql.sink.hbase.enums; + +/** + * Reason: + * Date: 2018/8/23 + * Company: www.dtstack.com + * + * @author xuchao + */ +public enum EReplaceType { + /** + * 参数 + */ + PARAM, + /** + * 函数 + */ + FUNC, + /** + * 常量 + */ + CONSTANT; +} From 6754700c3708b46af6cdf5dfbd8c15277665c497 Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 31 Mar 2020 09:43:46 +0800 Subject: [PATCH 012/109] =?UTF-8?q?fix=20=E7=B1=BB=E5=9E=8B=E5=88=A4?= =?UTF-8?q?=E6=96=ADbug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../kafka/KafkaDeserializationMetricWrapper.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaDeserializationMetricWrapper.java b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaDeserializationMetricWrapper.java index d0d76747d..bf0cee7dd 100644 --- a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaDeserializationMetricWrapper.java +++ b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaDeserializationMetricWrapper.java @@ -76,13 +76,7 @@ protected void beforeDeserialize() throws IOException { } protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { - Field consumerThreadField = null; - if(fetcher.getClass().getDeclaredField("consumerThread") != null){ - consumerThreadField = fetcher.getClass().getDeclaredField("consumerThread"); - } else { - consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); - } - + Field consumerThreadField = getConsumerThreadField(fetcher); consumerThreadField.setAccessible(true); KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); @@ -123,4 +117,12 @@ public Long getValue() { public void setFetcher(AbstractFetcher fetcher) { this.fetcher = fetcher; } + + private Field getConsumerThreadField(AbstractFetcher fetcher) throws NoSuchFieldException { + try { + return fetcher.getClass().getDeclaredField("consumerThread"); + } catch (Exception e) { + return fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + } + } } From 7676cc7f89355bb7918a2e5fecc08361bc334c08 Mon Sep 17 00:00:00 2001 From: maqi Date: Wed, 1 Apr 2020 15:49:52 +0800 Subject: [PATCH 013/109] remove unuse --- .../flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java deleted file mode 100644 index e69de29bb..000000000 From a7cff4d5010430db436cbc1a42a4558fd41db300 Mon Sep 17 00:00:00 2001 From: maqi Date: Wed, 1 Apr 2020 17:58:34 +0800 Subject: [PATCH 014/109] rdb sink check connection --- .../rdb/format/RetractJDBCOutputFormat.java | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 28479f4f6..0f3f1f827 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -51,7 +51,8 @@ public class RetractJDBCOutputFormat extends DtRichOutputFormat { private static final long serialVersionUID = 1L; private static final Logger LOG = LoggerFactory.getLogger(RetractJDBCOutputFormat.class); - + private static final int CONNECTION_CHECK_FREQUENCY = 100; + private int checkTimes; private String username; private String password; private String drivername; @@ -181,7 +182,6 @@ public void writeRecord(Tuple2 tuple2) { private void insertWrite(Row row) { - checkConnectionOpen(dbConn); try { if (batchNum == 1) { writeSingleRecord(row); @@ -315,6 +315,7 @@ private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLExce private synchronized void submitExecuteBatch() { try { + checkConnectionOpen(); this.upload.executeBatch(); dbConn.commit(); } catch (SQLException e) { @@ -330,11 +331,17 @@ private synchronized void submitExecuteBatch() { } } - private void checkConnectionOpen(Connection dbConn) { + private void checkConnectionOpen() { + LOG.info("test db connection Valid check !"); + checkTimes++; + if (checkTimes % CONNECTION_CHECK_FREQUENCY != 0) { + return; + } + LOG.warn("db connection Valid check !"); try { - if (dbConn.isClosed()) { + if (dbConn.isClosed() || !dbConn.isValid(100)) { LOG.info("db connection reconnect.."); - dbConn= establishConnection(); + dbConn = establishConnection(); upload = dbConn.prepareStatement(insertQuery); this.dbConn = dbConn; } @@ -345,6 +352,7 @@ private void checkConnectionOpen(Connection dbConn) { } catch (IOException e) { LOG.error("kerberos authentication failed..", e); } + checkTimes = 0; } /** From 64c2050759a21e436c7e68acd19b4f0147ffe1f4 Mon Sep 17 00:00:00 2001 From: maqi Date: Thu, 2 Apr 2020 16:03:33 +0800 Subject: [PATCH 015/109] deal rdb sink error --- .../flink/sql/enums/EConnectionErrorCode.java | 62 +++++++++++++++++++ .../sql/side/EConnectionErrorCodeTest.java | 39 ++++++++++++ .../rdb/format/RetractJDBCOutputFormat.java | 56 ++++++++++------- 3 files changed, 135 insertions(+), 22 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/enums/EConnectionErrorCode.java create mode 100644 core/src/test/java/com/dtstack/flink/sql/side/EConnectionErrorCodeTest.java diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/EConnectionErrorCode.java b/core/src/main/java/com/dtstack/flink/sql/enums/EConnectionErrorCode.java new file mode 100644 index 000000000..feed69785 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/enums/EConnectionErrorCode.java @@ -0,0 +1,62 @@ +/* + * 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.enums; + +import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang.exception.ExceptionUtils; + +import java.util.Arrays; + +/** + * Date: 2020/4/2 + * Company: www.dtstack.com + * @author maqi + */ +public enum EConnectionErrorCode { + ERROR_NOT_MATCH(0, "错误信息未匹配", new String[]{}), + CONN_DB_INVALID(1, "数据库连接失效,请重新打开", new String[]{"the last packet successfully received from the server was"}), + CONN_DB_FAILED(2, "数据库连接失败,请检查用户名或密码是否正确", new String[]{"Access denied for user"}), + DB_TABLE_NOT_EXIST(3, "操作的表名不存在", new String[]{"doesn't exist"}); + + private int code; + private String description; + private String[] baseErrorInfo; + + EConnectionErrorCode(int code, String description, String[] baseErrorInfo) { + this.code = code; + this.description = description; + this.baseErrorInfo = baseErrorInfo; + } + + + public static EConnectionErrorCode resolveErrorCodeFromException(Throwable e) { + final String stackErrorMsg = ExceptionUtils.getFullStackTrace(e); + return Arrays.stream(values()) + .filter(errorCode -> matchKnowError(errorCode, stackErrorMsg)) + .findAny() + .orElse(ERROR_NOT_MATCH); + } + + public static boolean matchKnowError(EConnectionErrorCode errorCode, String errorMsg) { + return Arrays.stream(errorCode.baseErrorInfo) + .filter(baseInfo -> StringUtils.containsIgnoreCase(errorMsg, baseInfo)) + .findAny() + .isPresent(); + } +} diff --git a/core/src/test/java/com/dtstack/flink/sql/side/EConnectionErrorCodeTest.java b/core/src/test/java/com/dtstack/flink/sql/side/EConnectionErrorCodeTest.java new file mode 100644 index 000000000..a43b253df --- /dev/null +++ b/core/src/test/java/com/dtstack/flink/sql/side/EConnectionErrorCodeTest.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.side; + +import com.dtstack.flink.sql.enums.EConnectionErrorCode; +import org.junit.Assert; +import org.junit.Test; + +/** + * Date: 2020/4/2 + * Company: www.dtstack.com + * @author maqi + */ +public class EConnectionErrorCodeTest { + + @Test + public void testResolveErrorCodeFromException(){ + EConnectionErrorCode errorCode = + EConnectionErrorCode.resolveErrorCodeFromException(new Exception("The last packet successfully received from the server was 179 milliseconds")); + + Assert.assertEquals(errorCode, EConnectionErrorCode.CONN_DB_INVALID); + } +} diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 0f3f1f827..22d7814fc 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.sink.rdb.format; +import com.dtstack.flink.sql.enums.EConnectionErrorCode; import com.dtstack.flink.sql.sink.rdb.RdbSink; import com.dtstack.flink.sql.util.JDBCUtils; import org.apache.commons.lang3.StringUtils; @@ -181,7 +182,7 @@ public void writeRecord(Tuple2 tuple2) { } - private void insertWrite(Row row) { + private void insertWrite(Row row) { try { if (batchNum == 1) { writeSingleRecord(row); @@ -201,8 +202,6 @@ private void insertWrite(Row row) { } else { outDirtyRecords.inc(batchNum == 1 ? batchNum : rows.size()); } - - } } @@ -213,7 +212,7 @@ private void writeSingleRecord(Row row) { upload.executeUpdate(); dbConn.commit(); } catch (SQLException e) { - + dealSQLException(e); if (outDirtyRecords.getCount() % DIRTY_PRINT_FREQUENCY == 0 || LOG.isDebugEnabled()) { LOG.error("record insert failed,dirty record num:{}, current row:{}", outDirtyRecords.getCount(), row.toString()); LOG.error("", e); @@ -315,46 +314,59 @@ private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLExce private synchronized void submitExecuteBatch() { try { - checkConnectionOpen(); + regularlyCheckConnection(); this.upload.executeBatch(); dbConn.commit(); } catch (SQLException e) { + LOG.warn("submitExecuteBatch error {}", e); try { dbConn.rollback(); } catch (SQLException e1) { - LOG.error("rollback data error !", e); + dealSQLException(e1); } - rows.forEach(this::writeSingleRecord); } finally { rows.clear(); } } - private void checkConnectionOpen() { - LOG.info("test db connection Valid check !"); + private void dealSQLException(Exception e) { + EConnectionErrorCode errorCode = EConnectionErrorCode.resolveErrorCodeFromException(e); + switch (errorCode) { + case CONN_DB_INVALID: + reconnection(); + break; + case CONN_DB_FAILED: + case DB_TABLE_NOT_EXIST: + throw new RuntimeException(e); + default: + } + } + + + private void regularlyCheckConnection() throws SQLException { checkTimes++; if (checkTimes % CONNECTION_CHECK_FREQUENCY != 0) { return; } LOG.warn("db connection Valid check !"); - try { - if (dbConn.isClosed() || !dbConn.isValid(100)) { - LOG.info("db connection reconnect.."); - dbConn = establishConnection(); - upload = dbConn.prepareStatement(insertQuery); - this.dbConn = dbConn; - } - } catch (SQLException e) { - LOG.error("check connection open failed..", e); - } catch (ClassNotFoundException e) { - LOG.error("load jdbc class error when reconnect db..", e); - } catch (IOException e) { - LOG.error("kerberos authentication failed..", e); + if (dbConn.isClosed() || !dbConn.isValid(100)) { + reconnection(); } checkTimes = 0; } + public void reconnection() throws RuntimeException { + try { + LOG.info("db connection reconnect.."); + dbConn = establishConnection(); + upload = dbConn.prepareStatement(insertQuery); + this.dbConn = dbConn; + } catch (Exception e) { + throw new RuntimeException("connection open failed..", e); + } + } + /** * Executes prepared statement and closes all resources of this instance. * From f544d1a011155af15fcab52c8b9f24f96e87a773 Mon Sep 17 00:00:00 2001 From: maqi Date: Fri, 3 Apr 2020 11:17:40 +0800 Subject: [PATCH 016/109] jedis npe check --- .../flink/sql/side/redis/RedisAllReqRow.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) 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 5d09c8f3d..6c82d9902 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 @@ -103,6 +103,7 @@ protected void reloadCache() { loadData(newCache); } catch (SQLException e) { LOG.error("", e); + throw new RuntimeException(e); } cacheRef.set(newCache); @@ -160,21 +161,22 @@ private void loadData(Map> tmpCache) throws SQLExcep JedisCommands jedis = null; try { StringBuilder keyPattern = new StringBuilder(tableInfo.getTableName()); - for(String key : tableInfo.getPrimaryKeys()){ + for (String key : tableInfo.getPrimaryKeys()) { keyPattern.append("_").append("*"); - }; + } jedis = getJedisWithRetry(CONN_RETRY_NUM); + if (null == jedis) { + throw new RuntimeException("redis all load data error,get jedis commands error!"); + } Set keys = getRedisKeys(RedisType.parse(tableInfo.getRedisType()), jedis, keyPattern.toString()); - if(CollectionUtils.isEmpty(keys)){ + if (CollectionUtils.isEmpty(keys)) { return; } - for(String key : keys){ + for (String key : keys) { tmpCache.put(key, jedis.hgetAll(key)); } - } catch (Exception e){ - LOG.error("", e); } finally { - if (jedis != null){ + if (jedis != null) { try { ((Closeable) jedis).close(); } catch (IOException e) { From 304f18061944f240b30e7b1c456ce85a9e28cc72 Mon Sep 17 00:00:00 2001 From: maqi Date: Fri, 3 Apr 2020 14:49:18 +0800 Subject: [PATCH 017/109] code opt --- .../side/cassandra/CassandraAllReqRow.java | 2 +- .../environment/MyLocalStreamEnvironment.java | 10 ++-- .../flink/sql/side/AbstractSideTableInfo.java | 4 +- .../dtstack/flink/sql/side/BaseAllReqRow.java | 2 + .../com/dtstack/flink/sql/util/DateUtil.java | 53 +++++++++---------- .../com/dtstack/flink/sql/util/JDBCUtils.java | 10 +--- .../Elasticsearch6AllReqRow.java | 2 +- .../sql/side/elasticsearch6/util/Es6Util.java | 7 --- .../side/elasticsearch6/util/MathUtil.java | 2 +- .../flink/sql/side/hbase/RowKeyBuilder.java | 5 +- .../sink/kafka/CustomerFlinkPartition.java | 1 + .../sql/launcher/ClusterClientFactory.java | 2 +- .../flink/sql/launcher/LauncherMain.java | 34 +++--------- .../flink/sql/side/mongo/MongoAllReqRow.java | 10 ++-- .../flink/sql/side/redis/RedisAllReqRow.java | 6 +-- 15 files changed, 60 insertions(+), 90 deletions(-) diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java index 3ad8b934e..5f642d176 100644 --- a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java @@ -267,7 +267,7 @@ private void loadData(Map>> tmpCache) throws SQ String connInfo = "address:" + tableInfo.getAddress() + ";userName:" + tableInfo.getUserName() + ",pwd:" + tableInfo.getPassword(); LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + connInfo); - Thread.sleep(5 * 1000); + Thread.sleep(LOAD_DATA_ERROR_SLEEP_TIME); } catch (InterruptedException e1) { LOG.error("", e1); } diff --git a/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java b/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java index 8ac1edd41..a8728c595 100644 --- a/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java +++ b/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java @@ -113,14 +113,16 @@ public JobExecutionResult execute(String jobName) throws Exception { LOG.info("Running job on local embedded Flink mini cluster"); } - MiniCluster exec = new MiniCluster(configBuilder.build()); + MiniCluster exec = null; try { + exec = new MiniCluster(configBuilder.build()); exec.start(); return exec.executeJobBlocking(jobGraph); - } - finally { + } finally { transformations.clear(); - exec.closeAsync(); + if (null != exec) { + exec.closeAsync(); + } } } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java index 7a832d0a1..9e114fe87 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java @@ -57,11 +57,11 @@ public abstract class AbstractSideTableInfo extends AbstractTableInfo implements public static final String ASYNC_REQ_POOL_KEY = "asyncPoolSize"; - private String cacheType = "none";//None or LRU or ALL + private String cacheType = "none"; private int cacheSize = 10000; - private long cacheTimeout = 60 * 1000;// + private long cacheTimeout = 60_000L; private int asyncCapacity=100; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/BaseAllReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/BaseAllReqRow.java index 3e3d0a8fb..ad9dcf1bd 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/BaseAllReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/BaseAllReqRow.java @@ -47,6 +47,8 @@ public abstract class BaseAllReqRow extends RichFlatMapFunction impl private static final Logger LOG = LoggerFactory.getLogger(BaseAllReqRow.class); + public static final long LOAD_DATA_ERROR_SLEEP_TIME = 5_000L; + protected BaseSideInfo sideInfo; private ScheduledExecutorService es; 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 409602baa..a3bb949a8 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 @@ -75,7 +75,6 @@ public static Date stringToDate(String strDate) { return null; } try { - ; return localDateTimetoDate(LocalDateTime.parse(strDate, DATE_TIME_FORMATTER)); } catch (DateTimeParseException ignored) { } @@ -113,13 +112,13 @@ public static long getTodayStart(long day) { if (("" + day).length() > 10) { cal.setTime(new Date(day)); } else { - cal.setTime(new Date(day * 1000)); + cal.setTime(new Date(day * 1000L)); } cal.set(Calendar.HOUR_OF_DAY, 0); cal.set(Calendar.MINUTE, 0); cal.set(Calendar.SECOND, 0); cal.set(Calendar.MILLISECOND, 0); - firstDay = cal.getTimeInMillis() / 1000; + firstDay = cal.getTimeInMillis() / 1000L; return firstDay; } @@ -131,7 +130,7 @@ public static long getTodayStart(long day) { */ public static long getTodayStart(long day,String scope) { if("MS".equals(scope)){ - return getTodayStart(day)*1000; + return getTodayStart(day)*1000L; }else if("S".equals(scope)){ return getTodayStart(day); }else{ @@ -151,13 +150,13 @@ public static long getNextDayStart(long day) { if (("" + day).length() > 10) { cal.setTime(new Date(day)); } else { - cal.setTime(new Date(day * 1000)); + cal.setTime(new Date(day * 1000L)); } cal.set(Calendar.HOUR_OF_DAY, 0); cal.set(Calendar.MINUTE, 0); cal.set(Calendar.SECOND, 0); cal.set(Calendar.MILLISECOND, 0); - nextDay = (cal.getTimeInMillis() + daySpanMill) / 1000; + nextDay = (cal.getTimeInMillis() + daySpanMill) / 1000L; return nextDay; } @@ -169,7 +168,7 @@ public static long getNextDayStart(long day) { */ public static long getNextDayStart(long day,String scope) { if("MS".equals(scope)){ - return getNextDayStart(day)*1000; + return getNextDayStart(day)*1000L; }else if("S".equals(scope)){ return getNextDayStart(day); }else{ @@ -186,13 +185,13 @@ public static long getNextDayStart(long day,String scope) { public static long getMonthFirst(long day) { long firstDay = 0L; Calendar cal = Calendar.getInstance(); - cal.setTime(new Date(day * 1000)); + cal.setTime(new Date(day * 1000L)); cal.set(Calendar.DAY_OF_MONTH, 1); cal.set(Calendar.HOUR_OF_DAY, 0); cal.set(Calendar.MINUTE, 0); cal.set(Calendar.SECOND, 0); cal.set(Calendar.MILLISECOND, 0); - firstDay = cal.getTimeInMillis() / 1000; + firstDay = cal.getTimeInMillis() / 1000L; return firstDay; } @@ -224,7 +223,7 @@ public static int getYear(long day) { public static long getWeekFirst(long day) { long firstDay = 0L; Calendar cal = Calendar.getInstance(); - cal.setTime(new Date(day * 1000)); + cal.setTime(new Date(day * 1000L)); cal.setFirstDayOfWeek(Calendar.MONDAY); cal.set(Calendar.DAY_OF_WEEK, Calendar.MONDAY); cal.set(Calendar.HOUR_OF_DAY, 0); @@ -243,7 +242,7 @@ public static long getWeekFirst(long day) { */ public static int getWeekOfYear(long day) { Calendar cal = Calendar.getInstance(); - cal.setTime(new Date(day * 1000)); + cal.setTime(new Date(day * 1000L)); return cal.get(Calendar.WEEK_OF_YEAR); } @@ -363,7 +362,7 @@ public static long getDateMillToFormat(String day, String inFormat) throws Parse Date date = sdf.parse(day); Calendar calendar = Calendar.getInstance(); calendar.setTime(date); - return calendar.getTimeInMillis()/1000; + return calendar.getTimeInMillis()/1000L; } /** @@ -383,7 +382,7 @@ public static long getFirstDay4Month(int year, int month) { cal.set(Calendar.MINUTE, 0); cal.set(Calendar.SECOND, 0); cal.set(Calendar.MILLISECOND, 0); - firstDay = cal.getTimeInMillis() / 1000; + firstDay = cal.getTimeInMillis() / 1000L; return firstDay; } @@ -405,7 +404,7 @@ public static long getLastDay4Month(int year, int month) { cal.set(Calendar.MINUTE, 0); cal.set(Calendar.SECOND, 0); cal.set(Calendar.MILLISECOND, 0); - lastDay = cal.getTimeInMillis() / 1000; + lastDay = cal.getTimeInMillis() / 1000L; return lastDay; } @@ -444,7 +443,7 @@ public static long getMillByOneDay() { cal.set(Calendar.MINUTE, 0); cal.set(Calendar.SECOND, 0); cal.set(Calendar.MILLISECOND, 0); - return cal.getTimeInMillis() / 1000; + return cal.getTimeInMillis() / 1000L; } /** @@ -458,7 +457,7 @@ public static long getMillByYesDay() { cal.set(Calendar.MINUTE, 0); cal.set(Calendar.SECOND, 0); cal.set(Calendar.MILLISECOND, 0); - return cal.getTimeInMillis() / 1000; + return cal.getTimeInMillis() / 1000L; } /** @@ -472,7 +471,7 @@ public static long getMillByLastWeekDay() { cal.set(Calendar.MINUTE, 0); cal.set(Calendar.SECOND, 0); cal.set(Calendar.MILLISECOND, 0); - return cal.getTimeInMillis() / 1000; + return cal.getTimeInMillis() / 1000L; } /** @@ -534,7 +533,7 @@ public static long getMillToDay(Calendar cal,int dateT){ cal.set(Calendar.MINUTE, 0); cal.set(Calendar.SECOND, 0); cal.set(Calendar.MILLISECOND, 0); - return cal.getTimeInMillis()/1000; + return cal.getTimeInMillis()/1000L; } /** @@ -563,7 +562,7 @@ public static String getDate(long day, String format) { if (("" + day).length() > 10) { cal.setTime(new Date(day)); } else { - cal.setTime(new Date(day * 1000)); + cal.setTime(new Date(day * 1000L)); } SimpleDateFormat sf = new SimpleDateFormat(format); return sf.format(cal.getTime()); @@ -619,7 +618,7 @@ public static Date stringToDate(String day, String format) { */ public static String longToString(long day, String format) throws ParseException { if (("" + day).length() <= 10){ - day=day*1000; + day=day*1000L; } SimpleDateFormat dateFormat = new SimpleDateFormat(format); String date = dateFormat.format(day); @@ -634,7 +633,7 @@ public static String longToString(long day, String format) throws ParseException */ public static int getMinusDate(int day, int minusDay) { Calendar cal = Calendar.getInstance(); - cal.setTime(new Date(day * 1000)); + cal.setTime(new Date(day * 1000L)); cal.set(Calendar.DATE, cal.get(Calendar.DATE) - minusDay); cal.set(Calendar.HOUR_OF_DAY, 0); cal.set(Calendar.MINUTE, 0); @@ -663,7 +662,7 @@ public static int getWeeksBetweenTwoDates(long startDay, long endDay) { public static int getMaxWeekOfYear(long startDay) { Calendar cal = Calendar.getInstance(); - cal.setTime(new Date(startDay * 1000)); + cal.setTime(new Date(startDay * 1000L)); return cal.getMaximum(Calendar.WEEK_OF_YEAR); } @@ -696,7 +695,7 @@ public static long getMinuteStart(long time) { if (("" + time).length() > 10) { cal.setTime(new Date(time)); } else { - cal.setTime(new Date(time * 1000)); + cal.setTime(new Date(time * 1000L)); } cal.set(Calendar.SECOND, 0); cal.set(Calendar.MILLISECOND, 0); @@ -719,7 +718,7 @@ public static long getHourStart(long time) { cal.set(Calendar.SECOND, 0); cal.set(Calendar.MILLISECOND, 0); cal.set(Calendar.MINUTE, 0); - firstDay = cal.getTimeInMillis() / 1000; + firstDay = cal.getTimeInMillis() / 1000L; return firstDay; } @@ -757,7 +756,7 @@ public static java.sql.Timestamp columnToTimestamp(Object column) { if (column == null) { return null; } else if(column instanceof String) { - return new java.sql.Timestamp(stringToDate((String)column).getTime()); + return null == stringToDate((String) column) ? null : new java.sql.Timestamp(stringToDate((String) column).getTime()); } else if (column instanceof Integer) { Integer rawData = (Integer) column; return new java.sql.Timestamp(rawData.longValue()); @@ -791,7 +790,7 @@ public static Timestamp getTimestampFromStr(String timeStr) { Instant instant = Instant.from(ISO_INSTANT.parse(timeStr)); return new Timestamp(instant.getEpochSecond() * MILLIS_PER_SECOND); } - return new Timestamp(stringToDate(timeStr).getTime()); + return null == stringToDate(timeStr) ? null : new Timestamp(stringToDate(timeStr).getTime()); } public static java.sql.Date getDateFromStr(String dateStr) { @@ -803,7 +802,7 @@ public static java.sql.Date getDateFromStr(String dateStr) { Instant instant = Instant.from(ISO_INSTANT.parse(dateStr)); return new java.sql.Date(instant.toEpochMilli()); } - return new java.sql.Date(stringToDate(dateStr).getTime()); + return null == stringToDate(dateStr) ? null : new java.sql.Date(stringToDate(dateStr).getTime()); } } 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 3d66d8a6d..faab22172 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,19 +19,13 @@ package com.dtstack.flink.sql.util; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.sql.DriverManager; public class JDBCUtils { - - private static final Logger LOG = LoggerFactory.getLogger(ClassUtil.class); - - public final static String LOCK_STR = "jdbc_lock_str"; + private static final Object LOCK = new Object(); public static void forName(String clazz, ClassLoader classLoader) { - synchronized (LOCK_STR){ + synchronized (LOCK){ try { Class.forName(clazz, true, classLoader); DriverManager.setLoginTimeout(10); diff --git a/elasticsearch6/elasticsearch6-side/elasticsearch6-all-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AllReqRow.java b/elasticsearch6/elasticsearch6-side/elasticsearch6-all-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AllReqRow.java index b8cf2c46a..e50ed5edc 100644 --- a/elasticsearch6/elasticsearch6-side/elasticsearch6-all-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AllReqRow.java +++ b/elasticsearch6/elasticsearch6-side/elasticsearch6-all-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AllReqRow.java @@ -187,7 +187,7 @@ private void loadData(Map>> tmpCache) throws IO try { String connInfo = "url: " + tableInfo.getAddress() + "; userName: " + tableInfo.getUserName() + ", pwd:" + tableInfo.getPassword(); LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + connInfo); - Thread.sleep(5 * 1000); + Thread.sleep(LOAD_DATA_ERROR_SLEEP_TIME); } catch (InterruptedException e1) { LOG.error("", e1); } 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 3d9e4d62a..bc5637515 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 @@ -176,13 +176,6 @@ public static BoolQueryBuilder buildFilterCondition(BoolQueryBuilder boolQueryBu case "NOT_EQUALS": return boolQueryBuilder.mustNot(QueryBuilders.termQuery(textConvertToKeyword(info.getFieldName(), sideInfo), removeSpaceAndApostrophe(info.getCondition())[0])); default: - try { - throw new Exception("elasticsearch6 does not support this operation: " + info.getOperatorKind()); - } catch (Exception e) { - - e.printStackTrace(); - LOG.error(e.getMessage()); - } return boolQueryBuilder; } 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 d6af042f4..d72530a56 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 @@ -216,7 +216,7 @@ public static BigDecimal getBigDecimal(Object obj) { } else if (obj instanceof BigInteger) { return new BigDecimal((BigInteger) obj); } else if (obj instanceof Number) { - return new BigDecimal(((Number) obj).doubleValue()); + return BigDecimal.valueOf(((Number) obj).doubleValue()); } throw new RuntimeException("not support type of " + obj.getClass() + " convert to BigDecimal."); } 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 b8530f224..4b3f35b68 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 @@ -120,9 +120,10 @@ public ReplaceInfo getReplaceInfo(String field){ } private List makeFormula(String formula){ - if(formula == null || formula.length() <= 0){ - Lists.newArrayList(); + if (formula == null || formula.length() <= 0) { + return Lists.newArrayList(); } + List result = Lists.newArrayList(); for(String meta: splitIgnoreQuotaBrackets(formula, "\\+")){ Matcher matcher = Md5Operator.matcher(meta.trim()); diff --git a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkPartition.java b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkPartition.java index 90dfe996b..d8028fcac 100644 --- a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkPartition.java +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkPartition.java @@ -20,6 +20,7 @@ public void open(int parallelInstanceId, int parallelInstances) { @Override public int partition(T record, byte[] key, byte[] value, String targetTopic, int[] partitions) { Preconditions.checkArgument(partitions != null && partitions.length > 0, "Partitions of the target topic is empty."); + if(key == null){ return partitions[this.parallelInstanceId % partitions.length]; } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 8f0090db5..11b7dec34 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -158,7 +158,7 @@ private static ApplicationId getYarnClusterApplicationId(YarnClient yarnClient) } - if (StringUtils.isEmpty(applicationId.toString())) { + if (null == applicationId) { throw new RuntimeException("No flink session found on yarn cluster."); } return applicationId; diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 0b036c87c..c39d8b8a9 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -22,6 +22,7 @@ import com.aiweiergou.tool.logger.api.ChangeLogLevelProcess; import com.dtstack.flink.sql.constrant.ConfigConstrant; +import com.dtstack.flink.sql.launcher.perjob.PerJobClusterClientBuilder; import com.google.common.collect.Lists; import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.TypeReference; @@ -41,6 +42,9 @@ import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.BufferedReader; import java.io.File; @@ -64,7 +68,6 @@ public class LauncherMain { private static String SP = File.separator; - private static String getLocalCoreJarPath(String localSqlRootJar) throws Exception { String jarPath = PluginUtil.getCoreJarFileName(localSqlRootJar, CORE_JAR); String corePath = localSqlRootJar + SP + jarPath; @@ -114,32 +117,11 @@ public static void main(String[] args) throws Exception { } - private static String[] parseJson(String[] args) { - BufferedReader reader = null; - String lastStr = ""; - try{ - FileInputStream fileInputStream = new FileInputStream(args[0]); - InputStreamReader inputStreamReader = new InputStreamReader(fileInputStream, "UTF-8"); - reader = new BufferedReader(inputStreamReader); - String tempString = null; - while((tempString = reader.readLine()) != null){ - lastStr += tempString; - } - reader.close(); - }catch(IOException e){ - e.printStackTrace(); - }finally{ - if(reader != null){ - try { - reader.close(); - } catch (IOException e) { - e.printStackTrace(); - } - } - } - Map map = JSON.parseObject(lastStr, new TypeReference>(){} ); + private static String[] parseJson(String[] args) throws IOException { + String lastStr = FileUtils.readFileUtf8(new File(args[0])); + Map map = JSON.parseObject(lastStr, new TypeReference>() { + }); List list = new LinkedList<>(); - for (Map.Entry entry : map.entrySet()) { list.add("-" + entry.getKey()); list.add(entry.getValue().toString()); diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java index 3373f3667..c7fa2398e 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java @@ -205,7 +205,7 @@ private void loadData(Map>> tmpCache) throws SQ try { String connInfo = "url:" + tableInfo.getAddress() + ";userName:" + tableInfo.getUserName() + ",pwd:" + tableInfo.getPassword(); LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + connInfo); - Thread.sleep(5 * 1000); + Thread.sleep(LOAD_DATA_ERROR_SLEEP_TIME); } catch (InterruptedException e1) { LOG.error("", e1); } @@ -248,12 +248,8 @@ private void loadData(Map>> tmpCache) throws SQ } catch (Exception e) { LOG.error("", e); } finally { - try { - if (mongoClient != null) { - mongoClient.close(); - } - } catch (Exception e) { - throw new RuntimeException("[closeMongoDB]:" + e.getMessage()); + if (mongoClient != null) { + mongoClient.close(); } } } 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 6c82d9902..bdee79f07 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 @@ -239,17 +239,17 @@ private JedisCommands getJedis(RedisSideTableInfo tableInfo) { } private JedisCommands getJedisWithRetry(int retryNum) { - while (retryNum-- > 0){ + while (retryNum-- > 0) { try { return getJedis(tableInfo); } catch (Exception e) { - if(retryNum <= 0){ + if (retryNum <= 0) { throw new RuntimeException("getJedisWithRetry error", e); } try { String jedisInfo = "url:" + tableInfo.getUrl() + ",pwd:" + tableInfo.getPassword() + ",database:" + tableInfo.getDatabase(); LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + jedisInfo); - Thread.sleep(5 * 1000); + Thread.sleep(LOAD_DATA_ERROR_SLEEP_TIME); } catch (InterruptedException e1) { LOG.error("", e1); } From c27194e6829d5a77c360db353319deefc171ebc0 Mon Sep 17 00:00:00 2001 From: maqi Date: Fri, 3 Apr 2020 15:37:57 +0800 Subject: [PATCH 018/109] code opt --- .../format/SerializationMetricWrapper.java | 1 - .../flink/sql/option/OptionParser.java | 2 -- .../sql/parser/CreateTmpTableParser.java | 6 ++++- .../flink/sql/parser/InsertSqlParser.java | 9 ++++++- .../com/dtstack/flink/sql/side/JoinInfo.java | 1 - .../com/dtstack/flink/sql/util/DateUtil.java | 24 +++++++++++++++---- .../com/dtstack/flink/sql/util/MathUtil.java | 12 ---------- .../dtstack/flink/sql/util/ParseUtils.java | 13 +++++++--- .../sql/watermarker/WaterMarkerAssigner.java | 2 -- .../sink/elasticsearch/CustomerSinkFunc.java | 1 - .../flink/sql/side/hbase/HbaseAllReqRow.java | 17 ++++++++++--- .../CustomerKeyedSerializationSchema.java | 2 -- .../AvroCRowSerializationSchema.java | 2 +- .../CsvCRowSerializationSchema.java | 7 ++---- .../kafka/table/KafkaSourceTableInfo.java | 1 - .../flink/sql/sink/kafka/KafkaProducer09.java | 1 - .../sink/kafka/KafkaProducer09Factory.java | 1 - .../flink/sql/source/kafka/KafkaSource.java | 3 --- .../sink/kafka/KafkaProducer010Factory.java | 1 - .../sink/kafka/KafkaProducer011Factory.java | 1 - .../flink/sql/launcher/LauncherMain.java | 7 ------ .../perjob/PerJobClusterClientBuilder.java | 2 -- .../flink/sql/side/mongo/MongoAllReqRow.java | 2 -- .../sql/side/mongo/MongoAsyncReqRow.java | 7 ------ .../sql/sink/mongo/MongoOutputFormat.java | 2 -- .../sink/postgresql/PostgresqlDialect.java | 1 - .../sql/sink/rdb/JDBCTypeConvertUtils.java | 1 - .../flink/sql/side/redis/RedisAllReqRow.java | 1 - .../side/sqlserver/SqlserverAsyncReqRow.java | 4 +++- 29 files changed, 62 insertions(+), 72 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/format/SerializationMetricWrapper.java b/core/src/main/java/com/dtstack/flink/sql/format/SerializationMetricWrapper.java index 8802198a0..fda394af7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/format/SerializationMetricWrapper.java +++ b/core/src/main/java/com/dtstack/flink/sql/format/SerializationMetricWrapper.java @@ -25,7 +25,6 @@ import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.MeterView; import org.apache.flink.table.runtime.types.CRow; -import org.apache.flink.types.Row; /** diff --git a/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java b/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java index 0fd057c5a..47416cd1b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java @@ -29,9 +29,7 @@ import java.util.List; import java.util.Map; import java.io.File; -import java.io.FileInputStream; import java.net.URLEncoder; -import java.util.stream.Stream; import org.apache.commons.codec.Charsets; import org.apache.flink.util.FileUtils; diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index de7141eb5..443e83816 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -22,7 +22,11 @@ import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.calcite.config.Lex; -import org.apache.calcite.sql.*; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlJoin; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import com.google.common.collect.Lists; diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index a7c6db9eb..479aecc5c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -21,7 +21,14 @@ package com.dtstack.flink.sql.parser; import org.apache.calcite.config.Lex; -import org.apache.calcite.sql.*; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlInsert; +import org.apache.calcite.sql.SqlJoin; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlMatchRecognize; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOrderBy; +import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.commons.lang3.StringUtils; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java index 8a8fe21f6..40b21ef03 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java @@ -25,7 +25,6 @@ import com.google.common.base.Strings; import java.io.Serializable; -import java.util.Map; /** * Join信息 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 a3bb949a8..a420c9d82 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 @@ -23,10 +23,21 @@ import java.sql.Timestamp; import java.text.ParseException; import java.text.SimpleDateFormat; -import java.time.*; + +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeParseException; -import java.util.*; + +import java.util.Calendar; +import java.util.Date; +import java.util.Locale; +import java.util.SimpleTimeZone; +import java.util.TimeZone; import java.util.regex.Pattern; import static java.time.format.DateTimeFormatter.ISO_INSTANT; @@ -756,7 +767,8 @@ public static java.sql.Timestamp columnToTimestamp(Object column) { if (column == null) { return null; } else if(column instanceof String) { - return null == stringToDate((String) column) ? null : new java.sql.Timestamp(stringToDate((String) column).getTime()); + Date date = stringToDate((String) column); + return null == date ? null : new java.sql.Timestamp(date.getTime()); } else if (column instanceof Integer) { Integer rawData = (Integer) column; return new java.sql.Timestamp(rawData.longValue()); @@ -790,7 +802,8 @@ public static Timestamp getTimestampFromStr(String timeStr) { Instant instant = Instant.from(ISO_INSTANT.parse(timeStr)); return new Timestamp(instant.getEpochSecond() * MILLIS_PER_SECOND); } - return null == stringToDate(timeStr) ? null : new Timestamp(stringToDate(timeStr).getTime()); + Date date = stringToDate(timeStr); + return null == date ? null : new Timestamp(date.getTime()); } public static java.sql.Date getDateFromStr(String dateStr) { @@ -802,7 +815,8 @@ public static java.sql.Date getDateFromStr(String dateStr) { Instant instant = Instant.from(ISO_INSTANT.parse(dateStr)); return new java.sql.Date(instant.toEpochMilli()); } - return null == stringToDate(dateStr) ? null : new java.sql.Date(stringToDate(dateStr).getTime()); + Date date = stringToDate(dateStr); + return null == date ? null : new java.sql.Date(date.getTime()); } } 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 5aaa40a15..c5584b5d8 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 @@ -23,18 +23,6 @@ import java.math.BigInteger; import java.sql.Date; import java.sql.Timestamp; -import java.text.ParseException; -import java.text.SimpleDateFormat; - - -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalTime; -import java.time.ZoneOffset; -import java.util.TimeZone; -import java.util.regex.Pattern; - -import static java.time.format.DateTimeFormatter.ISO_INSTANT; /** * Convert val to specified numeric type diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java index d399b533c..dfbab87a7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java @@ -37,16 +37,23 @@ package com.dtstack.flink.sql.util; import com.google.common.collect.HashBasedTable; -import org.apache.calcite.sql.*; + +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlJoin; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; 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.Set; -import static org.apache.calcite.sql.SqlKind.*; +import static org.apache.calcite.sql.SqlKind.AS; +import static org.apache.calcite.sql.SqlKind.IDENTIFIER; +import static org.apache.calcite.sql.SqlKind.JOIN; + /** * @Auther: jiangjunjie diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java index b0ca0335f..ffce0c646 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java @@ -29,8 +29,6 @@ import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import java.sql.Timestamp; -import java.lang.Long; - /** * define watermarker * Date: 2018/6/29 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 0d97f5995..23213e721 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 @@ -32,7 +32,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.stream.Collectors; diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java index ed2931fff..63d26d5dd 100644 --- a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java @@ -20,7 +20,10 @@ package com.dtstack.flink.sql.side.hbase; -import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.BaseAllReqRow; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; import org.apache.calcite.sql.JoinType; import org.apache.commons.collections.map.HashedMap; @@ -34,7 +37,12 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,7 +50,10 @@ import java.io.IOException; import java.sql.SQLException; import java.sql.Timestamp; -import java.util.*; +import java.util.Calendar; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicReference; public class HbaseAllReqRow extends BaseAllReqRow { diff --git a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKeyedSerializationSchema.java b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKeyedSerializationSchema.java index cde8d1b1d..78d316c99 100644 --- a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKeyedSerializationSchema.java +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKeyedSerializationSchema.java @@ -4,12 +4,10 @@ import com.dtstack.flink.sql.format.SerializationMetricWrapper; import com.dtstack.flink.sql.sink.kafka.serialization.JsonCRowSerializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; import org.apache.flink.table.runtime.types.CRow; -import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/AvroCRowSerializationSchema.java b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/AvroCRowSerializationSchema.java index 692e208b5..5f852832b 100644 --- a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/AvroCRowSerializationSchema.java +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/AvroCRowSerializationSchema.java @@ -53,7 +53,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.TimeZone; import java.util.stream.Collectors; @@ -287,6 +286,7 @@ private Object convertFlinkType(Schema schema, Object object) { case DOUBLE: case BOOLEAN: return object; + default: } throw new RuntimeException("Unsupported Avro type:" + schema); } diff --git a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/CsvCRowSerializationSchema.java b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/CsvCRowSerializationSchema.java index 4e57b6f2a..4e7c727d7 100644 --- a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/CsvCRowSerializationSchema.java +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/CsvCRowSerializationSchema.java @@ -47,10 +47,7 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.util.Arrays; -import java.util.Iterator; import java.util.Objects; -import java.util.stream.IntStream; -import java.util.stream.Stream; /** * Serialization schema that serializes an object of Flink types into a CSV bytes. @@ -132,9 +129,9 @@ public Builder setFieldDelimiter(char c) { public Builder setLineDelimiter(String delimiter) { Preconditions.checkNotNull(delimiter, "Delimiter must not be null."); - if (!delimiter.equals("\n") && !delimiter.equals("\r") && !delimiter.equals("\r\n")) { + if (!("\n".equals(delimiter)) && !("\r".equals(delimiter)) && !("\r\n".equals(delimiter))) { throw new IllegalArgumentException( - "Unsupported new line delimiter. Only \\n, \\r, or \\r\\n are supported."); + "Unsupported new line delimiter. Only \\n, \\r, or \\r\\n are supported."); } this.csvSchema = this.csvSchema.rebuild().setLineSeparator(delimiter).build(); return this; diff --git a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index c27eee376..1210aba40 100644 --- a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -19,7 +19,6 @@ package com.dtstack.flink.sql.source.kafka.table; -import com.dtstack.flink.sql.format.FormatType; import com.dtstack.flink.sql.table.AbstractSourceTableInfo; import com.google.common.base.Preconditions; diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09.java index bee1865dd..19e0feddb 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09.java @@ -24,7 +24,6 @@ import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer09; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.runtime.types.CRow; -import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09Factory.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09Factory.java index ee3423b07..b64727ee3 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09Factory.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09Factory.java @@ -23,7 +23,6 @@ import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.runtime.types.CRow; -import org.apache.flink.types.Row; import java.util.Optional; import java.util.Properties; diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 9f8917761..01c67cd56 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -21,18 +21,15 @@ import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.AbstractSourceTableInfo; -import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.types.Row; -import java.util.Map; import java.util.Properties; /** diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer010Factory.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer010Factory.java index e0e023586..eec398de0 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer010Factory.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer010Factory.java @@ -23,7 +23,6 @@ import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.runtime.types.CRow; -import org.apache.flink.types.Row; import java.util.Optional; import java.util.Properties; diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer011Factory.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer011Factory.java index 0cb11da82..b1cc53fb7 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer011Factory.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer011Factory.java @@ -23,7 +23,6 @@ import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.runtime.types.CRow; -import org.apache.flink.types.Row; import java.util.Optional; import java.util.Properties; diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index c39d8b8a9..551a00533 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -20,9 +20,7 @@ package com.dtstack.flink.sql.launcher; -import com.aiweiergou.tool.logger.api.ChangeLogLevelProcess; import com.dtstack.flink.sql.constrant.ConfigConstrant; -import com.dtstack.flink.sql.launcher.perjob.PerJobClusterClientBuilder; import com.google.common.collect.Lists; import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.TypeReference; @@ -43,14 +41,9 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.util.FileUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.io.BufferedReader; import java.io.File; -import java.io.FileInputStream; import java.io.IOException; -import java.io.InputStreamReader; import java.net.URLDecoder; import java.util.LinkedList; import java.util.List; diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java index 0e5089bc9..5dc74900e 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -32,14 +32,12 @@ import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; -import java.io.IOException; import java.net.MalformedURLException; import java.net.URL; import java.util.ArrayList; diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java index c7fa2398e..c1a32cb64 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java @@ -37,8 +37,6 @@ import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index d85a9990d..a1cf94000 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -44,13 +44,6 @@ import com.mongodb.async.client.MongoClients; import com.mongodb.async.client.MongoCollection; import com.mongodb.async.client.MongoDatabase; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import com.google.common.collect.Lists; -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.table.typeutils.TimeIndicatorTypeInfo; -import org.apache.flink.types.Row; import org.bson.Document; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java index 3f1f9f7e2..37f7f5952 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java @@ -35,8 +35,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; /** * Reason: diff --git a/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlDialect.java b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlDialect.java index 58e92c143..1d775adac 100644 --- a/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlDialect.java +++ b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlDialect.java @@ -20,7 +20,6 @@ import com.dtstack.flink.sql.sink.rdb.dialect.JDBCDialect; -import com.dtstack.flink.sql.util.DtStringUtil; import java.util.Arrays; import java.util.Optional; 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 28811d2fd..a41cad5ef 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 @@ -25,7 +25,6 @@ import java.math.BigDecimal; import java.sql.Date; import java.sql.PreparedStatement; -import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Timestamp; import java.sql.Types; 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 bdee79f07..8e6a08791 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 @@ -22,7 +22,6 @@ import com.dtstack.flink.sql.side.BaseAllReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.*; import com.dtstack.flink.sql.side.redis.enums.RedisType; import com.dtstack.flink.sql.side.redis.table.RedisSideReqRow; import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java index a63d28607..1f4fbf13a 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java @@ -19,7 +19,9 @@ package com.dtstack.flink.sql.side.sqlserver; -import com.dtstack.flink.sql.side.*; +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.rdb.async.RdbAsyncReqRow; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import io.vertx.core.Vertx; From 91e8880ff70bafc2864abdf6f84598ed85f911fb Mon Sep 17 00:00:00 2001 From: maqi Date: Fri, 3 Apr 2020 21:03:31 +0800 Subject: [PATCH 019/109] rm --- .gitlab-ci.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitlab-ci.yml b/.gitlab-ci.yml index 2952551ff..40ff81133 100644 --- a/.gitlab-ci.yml +++ b/.gitlab-ci.yml @@ -2,7 +2,7 @@ build: stage: test script: - mvn clean org.jacoco:jacoco-maven-plugin:0.7.8:prepare-agent package -Dmaven.test.failure.ignore=true -q - - mvn sonar:sonar -Dsonar.projectKey="dt-insight-engine/flinkStreamSQL" -Dsonar.branch.name="v1.8.0_dev" -Dsonar.login=11974c5e9a29625efa09fdc3c3fdc031efb1aab1 -Dsonar.host.url=http://172.16.100.198:9000 -Dsonar.jdbc.url=jdbc:postgresql://172.16.100.198:5432/sonar -Dsonar.java.binaries=target/sonar + - mvn sonar:sonar -Dsonar.projectKey="dt-insight-engine/flinkStreamSQL" -Dsonar.login=11974c5e9a29625efa09fdc3c3fdc031efb1aab1 -Dsonar.host.url=http://172.16.100.198:9000 -Dsonar.jdbc.url=jdbc:postgresql://172.16.100.198:5432/sonar -Dsonar.java.binaries=target/sonar - sh ci/sonar_notify.sh only: - v1.8.0_dev From 388e7f8b8397594c96822a69d5f4c565ff1a13da Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Sat, 4 Apr 2020 14:01:05 +0800 Subject: [PATCH 020/109] deal minicluster --- .../sql/environment/MyLocalStreamEnvironment.java | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java b/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java index a8728c595..029a07f99 100644 --- a/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java +++ b/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java @@ -113,16 +113,13 @@ public JobExecutionResult execute(String jobName) throws Exception { LOG.info("Running job on local embedded Flink mini cluster"); } - MiniCluster exec = null; - try { - exec = new MiniCluster(configBuilder.build()); + try (MiniCluster exec = new MiniCluster(configBuilder.build());) { exec.start(); - return exec.executeJobBlocking(jobGraph); - } finally { + JobExecutionResult jobExecutionResult = exec.executeJobBlocking(jobGraph); transformations.clear(); - if (null != exec) { - exec.closeAsync(); - } + return jobExecutionResult; + } catch (Exception e) { + throw new RuntimeException(e); } } } From 2079ce409b665d21ca4ecf04efc0869f1f507ecf Mon Sep 17 00:00:00 2001 From: dapeng Date: Wed, 8 Apr 2020 10:35:27 +0800 Subject: [PATCH 021/109] =?UTF-8?q?=E6=B7=BB=E5=8A=A0=E5=B0=8Dcountdown?= =?UTF-8?q?=E7=9A=84=E6=8D=95=E7=8D=B2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../sql/side/rdb/async/RdbAsyncReqRow.java | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) 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 da6cb3876..19d78eccd 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 @@ -119,19 +119,18 @@ private void connectWithRetry(Map inputParams, CRow input, Resul while(!finishFlag.get()){ CountDownLatch latch = new CountDownLatch(1); rdbSqlClient.getConnection(conn -> { - if(conn.failed()){ - if(failCounter.getAndIncrement() % 1000 == 0){ - logger.error("getConnection error", conn.cause()); - } - if(failCounter.get() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3)){ - resultFuture.completeExceptionally(conn.cause()); - finishFlag.set(true); - } - latch.countDown(); - conn.result().close(); - return; - } try { + if(conn.failed()){ + if(failCounter.getAndIncrement() % 1000 == 0){ + logger.error("getConnection error", conn.cause()); + } + if(failCounter.get() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3)){ + resultFuture.completeExceptionally(conn.cause()); + finishFlag.set(true); + } + conn.result().close(); + return; + } CONN_STATUS.set(true); ScheduledFuture timerFuture = registerTimer(input, resultFuture); cancelTimerWhenComplete(resultFuture, timerFuture); From aa9845a1e93a37243dc9fa2abfb7ac1679a5c1fe Mon Sep 17 00:00:00 2001 From: dapeng Date: Wed, 8 Apr 2020 11:27:18 +0800 Subject: [PATCH 022/109] =?UTF-8?q?=E6=9F=A5=E8=AF=A2=E6=A0=B9=E6=8D=AE?= =?UTF-8?q?=E5=A4=B1=E8=B4=A5=E6=AC=A1=E6=95=B0=E5=92=8CjoinType=E8=BE=93?= =?UTF-8?q?=E5=87=BA?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/side/AbstractSideTableInfo.java | 16 +++--------- .../flink/sql/side/BaseAsyncReqRow.java | 2 +- .../sql/table/AbstractSideTableParser.java | 2 +- .../sql/side/rdb/async/RdbAsyncReqRow.java | 26 ++++++++++++++++--- 4 files changed, 27 insertions(+), 19 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java index 0a6babc8a..71a309411 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java @@ -68,13 +68,11 @@ public abstract class AbstractSideTableInfo extends AbstractTableInfo implements private int asyncTimeout=10000; - private int asyncTimeoutNumLimit = Integer.MAX_VALUE; - private boolean partitionedJoin = false; private String cacheMode="ordered"; - private Integer asyncFailMaxNum; + private Long asyncFailMaxNum; private List predicateInfoes = Lists.newArrayList(); @@ -153,19 +151,11 @@ public List getPredicateInfoes() { return predicateInfoes; } - public int getAsyncTimeoutNumLimit() { - return asyncTimeoutNumLimit; - } - - public void setAsyncTimeoutNumLimit(int asyncTimeoutNumLimit) { - this.asyncTimeoutNumLimit = asyncTimeoutNumLimit; - } - - public Integer getAsyncFailMaxNum(Integer defaultValue) { + public Long getAsyncFailMaxNum(Long defaultValue) { return Objects.isNull(asyncFailMaxNum) ? defaultValue : asyncFailMaxNum; } - public void setAsyncFailMaxNum(Integer asyncFailMaxNum) { + public void setAsyncFailMaxNum(Long asyncFailMaxNum) { this.asyncFailMaxNum = asyncFailMaxNum; } 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 8713b541a..bfe14c3ea 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 @@ -149,7 +149,7 @@ public void timeout(CRow input, ResultFuture resultFuture) throws Exceptio resultFuture.complete(null); return; } - if(timeOutNum > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Integer.MAX_VALUE)){ + if(timeOutNum > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Long.MAX_VALUE)){ resultFuture.completeExceptionally(new Exception("Async function call timedoutNum beyond limit.")); return; } diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java index 83a06d256..7df86b3cb 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java @@ -116,7 +116,7 @@ protected void parseCacheProp(AbstractSideTableInfo sideTableInfo, Map 0){ sideTableInfo.setAsyncFailMaxNum(asyncFailNum); } 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 19d78eccd..e115ed21c 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 @@ -30,6 +30,7 @@ import io.vertx.ext.sql.SQLClient; import io.vertx.ext.sql.SQLConnection; import com.google.common.collect.Lists; +import org.apache.calcite.sql.JoinType; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.api.functions.async.ResultFuture; @@ -38,16 +39,19 @@ import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.Int; import java.sql.Timestamp; import java.util.Collection; import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; /** @@ -81,6 +85,8 @@ public class RdbAsyncReqRow extends BaseAsyncReqRow { private final static AtomicBoolean CONN_STATUS = new AtomicBoolean(true); + private final static AtomicLong TIMOUT_NUM = new AtomicLong(0); + private Logger logger = LoggerFactory.getLogger(getClass()); public RdbAsyncReqRow(BaseSideInfo sideInfo) { @@ -114,7 +120,7 @@ public void handleAsyncInvoke(Map inputParams, CRow input, Resul } private void connectWithRetry(Map inputParams, CRow input, ResultFuture resultFuture, SQLClient rdbSqlClient) { - AtomicInteger failCounter = new AtomicInteger(0); + AtomicLong failCounter = new AtomicLong(0); AtomicBoolean finishFlag = new AtomicBoolean(false); while(!finishFlag.get()){ CountDownLatch latch = new CountDownLatch(1); @@ -124,8 +130,8 @@ private void connectWithRetry(Map inputParams, CRow input, Resul if(failCounter.getAndIncrement() % 1000 == 0){ logger.error("getConnection error", conn.cause()); } - if(failCounter.get() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3)){ - resultFuture.completeExceptionally(conn.cause()); + if(failCounter.get() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3L)){ + outByJoinType(resultFuture, conn.cause()); finishFlag.set(true); } conn.result().close(); @@ -202,8 +208,12 @@ private void handleQuery(SQLConnection connection,Map inputParam JsonArray params = new JsonArray(Lists.newArrayList(inputParams.values())); connection.queryWithParams(sideInfo.getSqlCondition(), params, rs -> { if (rs.failed()) { + if(TIMOUT_NUM.incrementAndGet() > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Long.MAX_VALUE)){ + outByJoinType(resultFuture, rs.cause()); + return; + } LOG.error("Cannot retrieve the data from the database", rs.cause()); - resultFuture.completeExceptionally(rs.cause()); + resultFuture.complete(null); return; } @@ -242,4 +252,12 @@ private void handleQuery(SQLConnection connection,Map inputParam }); } + private void outByJoinType(ResultFuture resultFuture, Throwable e){ + if(sideInfo.getJoinType() == JoinType.LEFT){ + resultFuture.complete(null); + return; + } + resultFuture.completeExceptionally(e); + } + } From caa8824a6bcd8e14f13dbd2c3b6ffa284dea5c04 Mon Sep 17 00:00:00 2001 From: dapeng Date: Wed, 8 Apr 2020 11:29:11 +0800 Subject: [PATCH 023/109] =?UTF-8?q?=E5=8E=BB=E9=99=A4=E5=A4=9A=E4=BD=99?= =?UTF-8?q?=E7=9A=84import?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/side/rdb/async/RdbAsyncReqRow.java | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) 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 e115ed21c..c86434528 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 @@ -21,17 +21,15 @@ import com.dtstack.flink.sql.enums.ECacheContentType; import com.dtstack.flink.sql.side.BaseAsyncReqRow; -import com.dtstack.flink.sql.side.CacheMissVal; import com.dtstack.flink.sql.side.BaseSideInfo; +import com.dtstack.flink.sql.side.CacheMissVal; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.rdb.util.SwitchUtil; -import io.vertx.core.json.Json; +import com.google.common.collect.Lists; import io.vertx.core.json.JsonArray; import io.vertx.ext.sql.SQLClient; import io.vertx.ext.sql.SQLConnection; -import com.google.common.collect.Lists; import org.apache.calcite.sql.JoinType; -import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.table.runtime.types.CRow; @@ -39,20 +37,14 @@ import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Int; import java.sql.Timestamp; -import java.util.Collection; import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; /** * Date: 2018/11/26 From ac65b81d92c5ca181418e082a929a905acadac60 Mon Sep 17 00:00:00 2001 From: dapeng Date: Wed, 8 Apr 2020 11:40:37 +0800 Subject: [PATCH 024/109] =?UTF-8?q?rowkey=E7=9A=84=E5=AD=97=E6=AE=B5?= =?UTF-8?q?=E6=8B=BC=E6=8E=A5=E5=BD=A2=E5=BC=8F=E4=BF=AE=E6=94=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../sql/sink/hbase/HbaseOutputFormat.java | 22 ++++++++----------- .../flink/sql/sink/hbase/HbaseSink.java | 2 +- .../sql/sink/hbase/table/HbaseSinkParser.java | 2 +- .../sql/sink/hbase/table/HbaseTableInfo.java | 6 ++--- 4 files changed, 14 insertions(+), 18 deletions(-) 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 1769fbda7..a3189b0a6 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 @@ -55,7 +55,7 @@ public class HbaseOutputFormat extends AbstractDtRichOutputFormat { private String host; private String zkParent; - private String[] rowkey; + private String rowkey; private String tableName; private String[] columnNames; private String updateMode; @@ -164,25 +164,21 @@ private Put getPutByRow(Row record) { } private String buildRowKey(Row record) { - List rowKeyValues = getRowKeyValues(record); + String rowKeyValues = getRowKeyValues(record); // all rowkey not null - if (rowKeyValues.size() != rowkey.length) { + if (StringUtils.isBlank(rowKeyValues)) { LOG.error("row key value must not null,record is ..{}", record); outDirtyRecords.inc(); return ""; } - return StringUtils.join(rowKeyValues, "-"); + return rowKeyValues; } - private List getRowKeyValues(Row record) { - List rowKeyValues = Lists.newArrayList(); + private String getRowKeyValues(Row record) { Map row = rowConvertMap(record); - for (String key : rowkey) { - RowKeyBuilder rowKeyBuilder = new RowKeyBuilder(); - rowKeyBuilder.init(key); - rowKeyValues.add(rowKeyBuilder.getRowKey(row)); - } - return rowKeyValues; + RowKeyBuilder rowKeyBuilder = new RowKeyBuilder(); + rowKeyBuilder.init(rowkey); + return rowKeyBuilder.getRowKey(row); } private Map rowConvertMap(Row record){ @@ -232,7 +228,7 @@ public HbaseOutputFormatBuilder setTable(String tableName) { return this; } - public HbaseOutputFormatBuilder setRowkey(String[] rowkey) { + public HbaseOutputFormatBuilder setRowkey(String rowkey) { format.rowkey = rowkey; return this; } 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 9308725a2..13bd98b70 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 @@ -51,7 +51,7 @@ public class HbaseSink implements RetractStreamTableSink, IStreamSinkGener< protected String parent; protected String tableName; protected String updateMode; - protected String[] rowkey; + protected String rowkey; public HbaseSink() { // TO DO NOTHING diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java index 8d8996a23..bd1b7cac3 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java @@ -69,7 +69,7 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map columnNameFamily; @@ -90,11 +90,11 @@ public void setParent(String parent) { this.parent = parent; } - public String[] getRowkey() { + public String getRowkey() { return rowkey; } - public void setRowkey(String[] rowkey) { + public void setRowkey(String rowkey) { this.rowkey = rowkey; } From c4ce9357300195d20e2a2a72fe59a99c3a4998c4 Mon Sep 17 00:00:00 2001 From: maqi Date: Fri, 10 Apr 2020 10:43:35 +0800 Subject: [PATCH 025/109] add zk session expired error --- .../com/dtstack/flink/sql/enums/EConnectionErrorCode.java | 2 +- .../com/dtstack/flink/sql/side/EConnectionErrorCodeTest.java | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/EConnectionErrorCode.java b/core/src/main/java/com/dtstack/flink/sql/enums/EConnectionErrorCode.java index feed69785..a0a281c4a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/enums/EConnectionErrorCode.java +++ b/core/src/main/java/com/dtstack/flink/sql/enums/EConnectionErrorCode.java @@ -30,7 +30,7 @@ */ public enum EConnectionErrorCode { ERROR_NOT_MATCH(0, "错误信息未匹配", new String[]{}), - CONN_DB_INVALID(1, "数据库连接失效,请重新打开", new String[]{"the last packet successfully received from the server was"}), + CONN_DB_INVALID(1, "数据库连接失效,请重新打开", new String[]{"the last packet successfully received from the server was", "Zookeeper session has been expired"}), CONN_DB_FAILED(2, "数据库连接失败,请检查用户名或密码是否正确", new String[]{"Access denied for user"}), DB_TABLE_NOT_EXIST(3, "操作的表名不存在", new String[]{"doesn't exist"}); diff --git a/core/src/test/java/com/dtstack/flink/sql/side/EConnectionErrorCodeTest.java b/core/src/test/java/com/dtstack/flink/sql/side/EConnectionErrorCodeTest.java index a43b253df..6ae55c425 100644 --- a/core/src/test/java/com/dtstack/flink/sql/side/EConnectionErrorCodeTest.java +++ b/core/src/test/java/com/dtstack/flink/sql/side/EConnectionErrorCodeTest.java @@ -34,6 +34,10 @@ public void testResolveErrorCodeFromException(){ EConnectionErrorCode errorCode = EConnectionErrorCode.resolveErrorCodeFromException(new Exception("The last packet successfully received from the server was 179 milliseconds")); + EConnectionErrorCode ckSessionExpired = + EConnectionErrorCode.resolveErrorCodeFromException(new Exception("Excepetion: Zookeeper session has been expired")); + Assert.assertEquals(errorCode, EConnectionErrorCode.CONN_DB_INVALID); + Assert.assertEquals(ckSessionExpired, EConnectionErrorCode.CONN_DB_INVALID); } } From 4a4f21c0d31fe6614ddd94c741abfd88daa5ff30 Mon Sep 17 00:00:00 2001 From: maqi Date: Fri, 10 Apr 2020 12:00:19 +0800 Subject: [PATCH 026/109] rdb async connection pool size --- .../clickhouse/ClickhouseAsyncReqRow.java | 4 +-- .../dtstack/flink/sql/side/SideTableInfo.java | 30 +++++++++++++++++++ .../flink/sql/table/AbsSideTableParser.java | 8 +++++ .../flink/sql/side/db2/Db2AsyncReqRow.java | 4 +-- .../sql/side/impala/ImpalaAsyncReqRow.java | 29 ++++++++---------- .../sql/side/mysql/MysqlAsyncReqRow.java | 4 +-- .../sql/side/oracle/OracleAsyncReqRow.java | 4 +-- .../sql/side/polardb/PolardbAsyncReqRow.java | 4 +-- .../postgresql/PostgresqlAsyncReqRow.java | 4 +-- .../flink/sql/side/rdb/all/RdbAllReqRow.java | 8 +++++ .../sql/side/rdb/async/RdbAsyncReqRow.java | 13 +++++++- .../sql/side/rdb/table/RdbSideTableInfo.java | 12 ++++++++ .../side/sqlserver/SqlserverAsyncReqRow.java | 4 +-- 13 files changed, 96 insertions(+), 32 deletions(-) diff --git a/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java b/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java index 2b5223412..db49e4337 100644 --- a/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java +++ b/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java @@ -48,7 +48,7 @@ public void open(Configuration parameters) throws Exception { RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); clickhouseClientConfig.put("url", rdbSideTableInfo.getUrl()) .put("driver_class", CLICKHOUSE_DRIVER) - .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) + .put("max_pool_size", rdbSideTableInfo.getAsyncPoolSize()) .put("user", rdbSideTableInfo.getUserName()) .put("password", rdbSideTableInfo.getPassword()) .put("provider_class", DT_PROVIDER_CLASS) @@ -59,7 +59,7 @@ public void open(Configuration parameters) throws Exception { System.setProperty("vertx.disableFileCPResolving", "true"); VertxOptions vo = new VertxOptions(); vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); - vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); + vo.setWorkerPoolSize(rdbSideTableInfo.getAsyncPoolSize()); vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); setRdbSQLClient(JDBCClient.createNonShared(vertx, clickhouseClientConfig)); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java index 8c78578fb..32f065123 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java @@ -55,6 +55,8 @@ public abstract class SideTableInfo extends TableInfo implements Serializable { public static final String ASYNC_TIMEOUT_NUM_KEY = "asyncTimeoutNum"; + public static final String ASYNC_REQ_POOL_KEY = "asyncPoolSize"; + private String cacheType = "none";//None or LRU or ALL private int cacheSize = 10000; @@ -65,6 +67,11 @@ public abstract class SideTableInfo extends TableInfo implements Serializable { private int asyncTimeout=10000; + /** + * async operator req outside conn pool size, egg rdb conn pool size + */ + private int asyncPoolSize = 0; + private int asyncTimeoutNumLimit = Integer.MAX_VALUE; private boolean partitionedJoin = false; @@ -156,4 +163,27 @@ public void setAsyncTimeoutNumLimit(int asyncTimeoutNumLimit) { this.asyncTimeoutNumLimit = asyncTimeoutNumLimit; } + public int getAsyncPoolSize() { + return asyncPoolSize; + } + + public void setAsyncPoolSize(int asyncPoolSize) { + this.asyncPoolSize = asyncPoolSize; + } + + @Override + public String toString() { + return "Cache Info{" + + "cacheType='" + cacheType + '\'' + + ", cacheSize=" + cacheSize + + ", cacheTimeout=" + cacheTimeout + + ", asyncCapacity=" + asyncCapacity + + ", asyncTimeout=" + asyncTimeout + + ", asyncPoolSize=" + asyncPoolSize + + ", asyncTimeoutNumLimit=" + asyncTimeoutNumLimit + + ", partitionedJoin=" + partitionedJoin + + ", cacheMode='" + cacheMode + '\'' + + '}'; + } + } diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java index bf0b8a7a4..403360400 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java @@ -23,6 +23,8 @@ import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.util.MathUtil; +import org.apache.flink.util.Preconditions; + import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -114,6 +116,12 @@ protected void parseCacheProp(SideTableInfo sideTableInfo, Map p sideTableInfo.setAsyncTimeoutNumLimit(asyncTimeoutNum); } } + + if (props.containsKey(SideTableInfo.ASYNC_REQ_POOL_KEY.toLowerCase())) { + Integer asyncPoolSize = MathUtil.getIntegerVal(props.get(SideTableInfo.ASYNC_REQ_POOL_KEY.toLowerCase())); + Preconditions.checkArgument(asyncPoolSize > 0 && asyncPoolSize <= 20, "asyncPoolSize size limit (0,20]"); + sideTableInfo.setAsyncPoolSize(asyncPoolSize); + } } } } diff --git a/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java b/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java index 3d85c6fbd..cda2ed128 100644 --- a/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java +++ b/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java @@ -59,7 +59,7 @@ public void open(Configuration parameters) throws Exception { RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); db2lientConfig.put("url", rdbSideTableInfo.getUrl()) .put("driver_class", DB2_DRIVER) - .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) + .put("max_pool_size", rdbSideTableInfo.getAsyncPoolSize()) .put("user", rdbSideTableInfo.getUserName()) .put("password", rdbSideTableInfo.getPassword()) .put("provider_class", DT_PROVIDER_CLASS) @@ -71,7 +71,7 @@ public void open(Configuration parameters) throws Exception { VertxOptions vo = new VertxOptions(); vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); - vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); + vo.setWorkerPoolSize(rdbSideTableInfo.getAsyncPoolSize()); vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); setRdbSQLClient(JDBCClient.createNonShared(vertx, db2lientConfig)); 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 725e7c3ee..2b76ec26c 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 @@ -57,33 +57,28 @@ public ImpalaAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List>> newCache = Maps.newConcurrentMap(); 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 2d767ccd9..971285e52 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 @@ -22,6 +22,7 @@ import com.dtstack.flink.sql.enums.ECacheContentType; import com.dtstack.flink.sql.side.*; import com.dtstack.flink.sql.side.cache.CacheObj; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.side.rdb.util.SwitchUtil; import com.dtstack.flink.sql.util.DateUtil; import io.vertx.core.json.JsonArray; @@ -59,7 +60,9 @@ public class RdbAsyncReqRow extends AsyncReqRow { public final static int DEFAULT_VERTX_WORKER_POOL_SIZE = Runtime.getRuntime().availableProcessors() * 2; - public final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE + DEFAULT_VERTX_WORKER_POOL_SIZE; + 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 DEFAULT_IDLE_CONNECTION_TEST_PEROID = 60; @@ -73,6 +76,14 @@ public class RdbAsyncReqRow extends AsyncReqRow { public RdbAsyncReqRow(SideInfo sideInfo) { super(sideInfo); + init(sideInfo); + } + + protected void init(SideInfo sideInfo) { + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); + int defaultAsyncPoolSize = Math.min(MAX_DB_CONN_POOL_SIZE_LIMIT, DEFAULT_DB_CONN_POOL_SIZE); + int rdbPoolSize = rdbSideTableInfo.getAsyncPoolSize() > 0 ? rdbSideTableInfo.getAsyncPoolSize() : defaultAsyncPoolSize; + rdbSideTableInfo.setAsyncPoolSize(rdbPoolSize); } @Override diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java index 2c8c18fcd..fdb024b1d 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java @@ -101,4 +101,16 @@ public String getPassword() { public void setPassword(String password) { this.password = password; } + + @Override + public String toString() { + String cacheInfo = super.toString(); + String connectionInfo = "Rdb Side Connection Info{" + + "url='" + url + '\'' + + ", tableName='" + tableName + '\'' + + ", schema='" + schema + '\'' + + '}'; + return cacheInfo + " , " + connectionInfo; + } + } diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java index cb4240ca6..9fc2c35cd 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java @@ -51,7 +51,7 @@ public void open(Configuration parameters) throws Exception { RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); sqlserverClientConfig.put("url", rdbSideTableInfo.getUrl()) .put("driver_class", SQLSERVER_DRIVER) - .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) + .put("max_pool_size", rdbSideTableInfo.getAsyncPoolSize()) .put("user", rdbSideTableInfo.getUserName()) .put("password", rdbSideTableInfo.getPassword()) .put("provider_class", DT_PROVIDER_CLASS) @@ -64,7 +64,7 @@ public void open(Configuration parameters) throws Exception { VertxOptions vo = new VertxOptions(); vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); - vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); + vo.setWorkerPoolSize(rdbSideTableInfo.getAsyncPoolSize()); vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); setRdbSQLClient(JDBCClient.createNonShared(vertx, sqlserverClientConfig)); From 330be96a729c630506a8737ce198d5ee6a77b99e Mon Sep 17 00:00:00 2001 From: dapeng Date: Fri, 10 Apr 2020 12:56:06 +0800 Subject: [PATCH 027/109] =?UTF-8?q?=E6=B7=BB=E5=8A=A0=E8=AE=B0=E8=BF=B0?= =?UTF-8?q?=E5=92=8C=E8=B6=85=E8=BF=87=E6=AC=A1=E6=95=B0=E4=BB=BB=E5=8A=A1?= =?UTF-8?q?=E9=80=80=E5=87=BA?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../sql/table/AbstractSideTableParser.java | 6 ---- .../sql/side/rdb/async/RdbAsyncReqRow.java | 33 ++++++++++--------- 2 files changed, 18 insertions(+), 21 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java index 7df86b3cb..de46e8409 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java @@ -108,12 +108,6 @@ protected void parseCacheProp(AbstractSideTableInfo sideTableInfo, Map 0){ - sideTableInfo.setAsyncTimeoutNumLimit(asyncTimeoutNum); - } - } if(props.containsKey(AbstractSideTableInfo.ASYNC_FAIL_MAX_NUM_KEY.toLowerCase())){ Long asyncFailNum = MathUtil.getLongVal(props.get(AbstractSideTableInfo.ASYNC_FAIL_MAX_NUM_KEY.toLowerCase())); 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 c86434528..ca6dcc38b 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 @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.side.rdb.async; import com.dtstack.flink.sql.enums.ECacheContentType; +import com.dtstack.flink.sql.metric.MetricConstant; import com.dtstack.flink.sql.side.BaseAsyncReqRow; import com.dtstack.flink.sql.side.BaseSideInfo; import com.dtstack.flink.sql.side.CacheMissVal; @@ -31,6 +32,7 @@ import io.vertx.ext.sql.SQLConnection; import org.apache.calcite.sql.JoinType; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.metrics.Counter; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; @@ -77,10 +79,12 @@ public class RdbAsyncReqRow extends BaseAsyncReqRow { private final static AtomicBoolean CONN_STATUS = new AtomicBoolean(true); - private final static AtomicLong TIMOUT_NUM = new AtomicLong(0); + private final static AtomicLong FAIL_NUM = new AtomicLong(0); private Logger logger = LoggerFactory.getLogger(getClass()); + private Counter counter = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_SIDE_PARSE_ERROR_RECORDS); + public RdbAsyncReqRow(BaseSideInfo sideInfo) { super(sideInfo); } @@ -123,7 +127,12 @@ private void connectWithRetry(Map inputParams, CRow input, Resul logger.error("getConnection error", conn.cause()); } if(failCounter.get() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3L)){ - outByJoinType(resultFuture, conn.cause()); + if(FAIL_NUM.incrementAndGet() > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Long.MAX_VALUE)){ + counter.inc(); + resultFuture.completeExceptionally(conn.cause()); + } else { + dealMissKey(input, resultFuture); + } finishFlag.set(true); } conn.result().close(); @@ -135,6 +144,7 @@ private void connectWithRetry(Map inputParams, CRow input, Resul handleQuery(conn.result(), inputParams, input, resultFuture); finishFlag.set(true); } catch (Exception e) { + dealFillDataError(resultFuture, e, null); logger.error("", e); } finally { latch.countDown(); @@ -200,12 +210,13 @@ private void handleQuery(SQLConnection connection,Map inputParam JsonArray params = new JsonArray(Lists.newArrayList(inputParams.values())); connection.queryWithParams(sideInfo.getSqlCondition(), params, rs -> { if (rs.failed()) { - if(TIMOUT_NUM.incrementAndGet() > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Long.MAX_VALUE)){ - outByJoinType(resultFuture, rs.cause()); - return; + if(FAIL_NUM.incrementAndGet() > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Long.MAX_VALUE)){ + LOG.error("Cannot retrieve the data from the database", rs.cause()); + counter.inc(); + resultFuture.completeExceptionally(rs.cause()); + } else { + dealMissKey(input, resultFuture); } - LOG.error("Cannot retrieve the data from the database", rs.cause()); - resultFuture.complete(null); return; } @@ -244,12 +255,4 @@ private void handleQuery(SQLConnection connection,Map inputParam }); } - private void outByJoinType(ResultFuture resultFuture, Throwable e){ - if(sideInfo.getJoinType() == JoinType.LEFT){ - resultFuture.complete(null); - return; - } - resultFuture.completeExceptionally(e); - } - } From 2027b415a930d35417575669a31212b5f5b93523 Mon Sep 17 00:00:00 2001 From: maqi Date: Mon, 13 Apr 2020 19:08:49 +0800 Subject: [PATCH 028/109] parse where case --- .../flink/sql/parser/InsertSqlParser.java | 16 +++------- .../flink/sql/side/SidePredicatesParser.java | 21 +++++++------ .../flink/sql/util/FieldReplaceUtil.java | 17 ++++++++++- .../dtstack/flink/sql/util/TableUtils.java | 30 ++++++++++++------- 4 files changed, 52 insertions(+), 32 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index e2940c4f7..e29ac4bef 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -26,6 +26,7 @@ import org.apache.calcite.sql.parser.SqlParser; import org.apache.commons.lang3.StringUtils; import com.google.common.collect.Lists; +import org.apache.flink.table.calcite.FlinkPlannerImpl; import java.util.List; @@ -53,18 +54,9 @@ public static InsertSqlParser newInstance(){ @Override public void parseSql(String sql, SqlTree sqlTree) { - SqlParser.Config config = SqlParser - .configBuilder() - .setLex(Lex.MYSQL) - .build(); - - SqlParser sqlParser = SqlParser.create(sql,config); - SqlNode sqlNode = null; - try { - sqlNode = sqlParser.parseStmt(); - } catch (SqlParseException e) { - throw new RuntimeException("", e); - } + + FlinkPlannerImpl flinkPlanner = FlinkPlanner.getFlinkPlanner(); + SqlNode sqlNode = flinkPlanner.parse(sql); SqlParseResult sqlParseResult = new SqlParseResult(); parseNode(sqlNode, sqlParseResult); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java index 0902bf39f..99dc3507b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java @@ -138,16 +138,19 @@ private void extractPredicateInfo(SqlNode whereNode, List predica private void fillPredicateInfoToList(SqlBasicCall whereNode, List predicatesInfoList, String operatorName, SqlKind operatorKind, int fieldIndex, int conditionIndex) { - SqlIdentifier fieldFullPath = (SqlIdentifier) whereNode.getOperands()[fieldIndex]; - if (fieldFullPath.names.size() == 2) { - String ownerTable = fieldFullPath.names.get(0); - String fieldName = fieldFullPath.names.get(1); - String content = (operatorKind == SqlKind.BETWEEN) ? whereNode.getOperands()[conditionIndex].toString() + " AND " + - whereNode.getOperands()[2].toString() : whereNode.getOperands()[conditionIndex].toString(); + SqlNode sqlNode = whereNode.getOperands()[fieldIndex]; + if (sqlNode.getKind() == SqlKind.IDENTIFIER) { + SqlIdentifier fieldFullPath = (SqlIdentifier) sqlNode; + if (fieldFullPath.names.size() == 2) { + String ownerTable = fieldFullPath.names.get(0); + String fieldName = fieldFullPath.names.get(1); + String content = (operatorKind == SqlKind.BETWEEN) ? whereNode.getOperands()[conditionIndex].toString() + " AND " + + whereNode.getOperands()[2].toString() : whereNode.getOperands()[conditionIndex].toString(); - PredicateInfo predicateInfo = PredicateInfo.builder().setOperatorName(operatorName).setOperatorKind(operatorKind.toString()) - .setOwnerTable(ownerTable).setFieldName(fieldName).setCondition(content).build(); - predicatesInfoList.add(predicateInfo); + PredicateInfo predicateInfo = PredicateInfo.builder().setOperatorName(operatorName).setOperatorKind(operatorKind.toString()) + .setOwnerTable(ownerTable).setFieldName(fieldName).setCondition(content).build(); + predicatesInfoList.add(predicateInfo); + } } } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java index 10919ca5b..697954dd6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java @@ -146,7 +146,22 @@ private static SqlNode replaceNodeInfo(SqlNode groupNode, } return sqlBasicCall; - }else{ + } else if (groupNode.getKind() == CASE) { + SqlCase sqlCase = (SqlCase) groupNode; + + for (int i = 0; i < sqlCase.getWhenOperands().size(); i++) { + SqlNode sqlNode = sqlCase.getWhenOperands().getList().get(i); + SqlNode replaceNode = replaceSelectFieldName(sqlNode, oldTbName, newTbName, mappingField); + sqlCase.getWhenOperands().set(i,replaceNode); + } + + for (int i = 0; i < sqlCase.getThenOperands().size(); i++) { + SqlNode sqlNode = sqlCase.getThenOperands().getList().get(i); + SqlNode replaceNode = replaceSelectFieldName(sqlNode, oldTbName, newTbName, mappingField); + sqlCase.getThenOperands().set(i,replaceNode); + } + return sqlCase; + } else { return groupNode; } } 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 1527c2db7..579beaf63 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 @@ -568,7 +568,12 @@ private static void replaceConditionNode(SqlNode selectNode, String oldTbName, S }else if(selectNode.getKind() == OTHER){ //不处理 return; - }else{ + } else if (selectNode.getKind() == CASE) { + SqlCase sqlCase = (SqlCase) selectNode; + + sqlCase.getWhenOperands().getList().forEach(sqlNode -> replaceConditionNode(sqlNode, oldTbName, newTbName, fieldReplaceRef)); + sqlCase.getThenOperands().getList().forEach(sqlNode -> replaceConditionNode(sqlNode, oldTbName, newTbName, fieldReplaceRef)); + } else { throw new RuntimeException(String.format("not support node kind of %s to replace name now.", selectNode.getKind())); } } @@ -579,14 +584,14 @@ private static void replaceConditionNode(SqlNode selectNode, String oldTbName, S * @param fieldInfos */ public static void getConditionRefTable(SqlNode selectNode, Set fieldInfos) { - if(selectNode.getKind() == IDENTIFIER){ + if (selectNode.getKind() == IDENTIFIER) { SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; fieldInfos.add(sqlIdentifier.toString()); return; - }else if(selectNode.getKind() == LITERAL || selectNode.getKind() == LITERAL_CHAIN){//字面含义 + } else if (selectNode.getKind() == LITERAL || selectNode.getKind() == LITERAL_CHAIN) {//字面含义 return; - }else if( AGGREGATE.contains(selectNode.getKind()) + } else if (AGGREGATE.contains(selectNode.getKind()) || AVG_AGG_FUNCTIONS.contains(selectNode.getKind()) || COMPARISON.contains(selectNode.getKind()) || selectNode.getKind() == OTHER_FUNCTION @@ -616,15 +621,15 @@ public static void getConditionRefTable(SqlNode selectNode, Set fieldInf || selectNode.getKind() == TIMESTAMP_DIFF || selectNode.getKind() == LIKE - ){ + ) { SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; - for(int i=0; i fieldInf } return; - }else if(selectNode.getKind() == OTHER){ + } else if (selectNode.getKind() == OTHER) { //不处理 return; - }else{ + } else if (selectNode.getKind() == CASE) { + SqlCase sqlCase = (SqlCase) selectNode; + + sqlCase.getWhenOperands().getList().forEach(sqlNode -> getConditionRefTable(sqlNode, fieldInfos)); + sqlCase.getThenOperands().getList().forEach(sqlNode -> getConditionRefTable(sqlNode, fieldInfos)); + } else { throw new RuntimeException(String.format("not support node kind of %s to replace name now.", selectNode.getKind())); } } From a7ea7f3c8eda717d50c9e1edf8baefe883686363 Mon Sep 17 00:00:00 2001 From: maqi Date: Mon, 13 Apr 2020 21:34:38 +0800 Subject: [PATCH 029/109] add log --- core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java index fae0ba9aa..6f0dce58a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java @@ -66,6 +66,7 @@ public void open(Configuration parameters) throws Exception { super.open(parameters); initCache(); initMetric(); + LOG.info("async dim table config info: {} ", sideInfo.getSideTableInfo().toString()); } private void initCache(){ From 63f3ab158c2900deed4063c123a08f12754f1430 Mon Sep 17 00:00:00 2001 From: maqi Date: Tue, 14 Apr 2020 10:05:10 +0800 Subject: [PATCH 030/109] fix spelling and kafka params map name --- .../dtstack/flink/sql/exec/ExecuteProcessHelper.java | 10 +++++----- .../flink/sql/sink/kafka/table/KafkaSinkTableInfo.java | 8 ++++---- .../sql/source/kafka/table/KafkaSourceTableInfo.java | 8 ++++---- 3 files changed, 13 insertions(+), 13 deletions(-) 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 7ffe7d1dc..4bd33363c 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 @@ -266,7 +266,7 @@ public static void registerUserDefinedFunction(SqlTree sqlTree, List jarUrl */ public static Set registerTable(SqlTree sqlTree, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv, String localSqlPluginPath, String remoteSqlPluginPath, String pluginLoadMode, Map sideTableMap, Map registerTableCache) throws Exception { - Set pluginClassPatshSets = Sets.newHashSet(); + Set pluginClassPathSets = Sets.newHashSet(); WaterMarkerAssigner waterMarkerAssigner = new WaterMarkerAssigner(); for (AbstractTableInfo tableInfo : sqlTree.getTableInfoMap().values()) { @@ -304,7 +304,7 @@ public static Set registerTable(SqlTree sqlTree, StreamExecutionEnvironment registerTableCache.put(tableInfo.getName(), regTable); URL sourceTablePathUrl = PluginUtil.buildSourceAndSinkPathByLoadMode(tableInfo.getType(), AbstractSourceTableInfo.SOURCE_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode); - pluginClassPatshSets.add(sourceTablePathUrl); + pluginClassPathSets.add(sourceTablePathUrl); } else if (tableInfo instanceof AbstractTargetTableInfo) { TableSink tableSink = StreamSinkFactory.getTableSink((AbstractTargetTableInfo) tableInfo, localSqlPluginPath); @@ -312,18 +312,18 @@ public static Set registerTable(SqlTree sqlTree, StreamExecutionEnvironment tableEnv.registerTableSink(tableInfo.getName(), tableInfo.getFields(), flinkTypes, tableSink); URL sinkTablePathUrl = PluginUtil.buildSourceAndSinkPathByLoadMode(tableInfo.getType(), AbstractTargetTableInfo.TARGET_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode); - pluginClassPatshSets.add(sinkTablePathUrl); + pluginClassPathSets.add(sinkTablePathUrl); } else if (tableInfo instanceof AbstractSideTableInfo) { String sideOperator = ECacheType.ALL.name().equals(((AbstractSideTableInfo) tableInfo).getCacheType()) ? "all" : "async"; sideTableMap.put(tableInfo.getName(), (AbstractSideTableInfo) tableInfo); URL sideTablePathUrl = PluginUtil.buildSidePathByLoadMode(tableInfo.getType(), sideOperator, AbstractSideTableInfo.TARGET_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode); - pluginClassPatshSets.add(sideTablePathUrl); + pluginClassPathSets.add(sideTablePathUrl); } else { throw new RuntimeException("not support table type:" + tableInfo.getType()); } } - return pluginClassPatshSets; + return pluginClassPathSets; } /** diff --git a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index f1bef088e..2804c71de 100644 --- a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -58,7 +58,7 @@ public class KafkaSinkTableInfo extends AbstractTargetTableInfo { private String bootstrapServers; - public Map kafkaParam = new HashMap(); + public Map kafkaParams = new HashMap(); private String topic; @@ -73,15 +73,15 @@ public class KafkaSinkTableInfo extends AbstractTargetTableInfo { private String updateMode; public void addKafkaParam(String key, String value) { - kafkaParam.put(key, value); + kafkaParams.put(key, value); } public String getKafkaParam(String key) { - return kafkaParam.get(key); + return kafkaParams.get(key); } public Set getKafkaParamKeys() { - return kafkaParam.keySet(); + return kafkaParams.keySet(); } public String getBootstrapServers() { diff --git a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 1210aba40..62453166f 100644 --- a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -69,7 +69,7 @@ public class KafkaSourceTableInfo extends AbstractSourceTableInfo { private String fieldDelimiter; - public Map kafkaParam = new HashMap<>(); + public Map kafkaParams = new HashMap<>(); public String getBootstrapServers() { @@ -113,15 +113,15 @@ public void setTopicIsPattern(Boolean topicIsPattern) { } public void addKafkaParam(Map kafkaParam) { - kafkaParam.putAll(kafkaParam); + kafkaParams.putAll(kafkaParam); } public String getKafkaParam(String key) { - return kafkaParam.get(key); + return kafkaParams.get(key); } public Set getKafkaParamKeys() { - return kafkaParam.keySet(); + return kafkaParams.keySet(); } public String getSourceDataType() { From 86bd4b6a06a925f05955fcdb6e61263063e6316d Mon Sep 17 00:00:00 2001 From: tiezhu Date: Tue, 14 Apr 2020 11:41:04 +0800 Subject: [PATCH 031/109] fix 24602 --- .../flink/sql/side/rdb/all/RdbAllReqRow.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java index 60e3d817f..2fc1d3b23 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java @@ -83,12 +83,12 @@ protected void initCache() throws SQLException { protected void reloadCache() { //reload cacheRef and replace to old cacheRef Map>> newCache = Maps.newConcurrentMap(); - cacheRef.set(newCache); try { loadData(newCache); } catch (SQLException e) { throw new RuntimeException(e); } + cacheRef.set(newCache); LOG.info("----- rdb all cacheRef reload end:{}", Calendar.getInstance()); } @@ -112,9 +112,9 @@ public void flatMap(CRow value, Collector out) throws Exception { List> cacheList = cacheRef.get().get(cacheKey); if (CollectionUtils.isEmpty(cacheList) && sideInfo.getJoinType() == JoinType.LEFT) { out.collect(new CRow(fillData(value.row(), null), value.change())); + } else if (!CollectionUtils.isEmpty(cacheList)) { + cacheList.forEach(one -> out.collect(new CRow(fillData(value.row(), one), value.change()))); } - - cacheList.stream().forEach(one -> out.collect(new CRow(fillData(value.row(), one), value.change()))); } @Override @@ -141,8 +141,8 @@ public Row fillData(Row input, Object sideInput) { } /** - * covert flink time attribute.Type information for indicating event or processing time. - * However, it behaves like a regular SQL timestamp but is serialized as Long. + * covert flink time attribute.Type information for indicating event or processing time. + * However, it behaves like a regular SQL timestamp but is serialized as Long. * * @param entry * @param obj @@ -222,7 +222,8 @@ public int getFetchSize() { } /** - * get jdbc connection + * get jdbc connection + * * @param dbURL * @param userName * @param password From 6943b54f1b9964cbceeae35a8b2309c1fb08b249 Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 14 Apr 2020 15:54:21 +0800 Subject: [PATCH 032/109] =?UTF-8?q?fix=20=E5=88=AB=E5=90=8D=E9=97=AE?= =?UTF-8?q?=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flink/sql/side/SideInfo.java | 7 ++++--- .../dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java | 6 ++++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java index 029c86e25..d8a22e022 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java @@ -93,7 +93,7 @@ public void parseSelectFields(JoinInfo joinInfo){ if(fieldInfo.getTable().equalsIgnoreCase(sideTableName)){ String sideFieldName = sideTableInfo.getPhysicalFields().getOrDefault(fieldInfo.getFieldName(), fieldInfo.getFieldName()); fields.add(sideFieldName); - sideSelectFieldsType.put(sideTableFieldIndex, getTargetFieldType(sideFieldName)); + sideSelectFieldsType.put(sideTableFieldIndex, getTargetFieldType(fieldInfo.getFieldName())); sideFieldIndex.put(i, sideTableFieldIndex); sideFieldNameIndex.put(i, sideFieldName); sideTableFieldIndex++; @@ -132,12 +132,13 @@ public void dealOneEqualCon(SqlNode sqlNode, String sideTableName){ String leftTableName = left.getComponent(0).getSimple(); String leftField = left.getComponent(1).getSimple(); + Map physicalFields = sideTableInfo.getPhysicalFields(); String rightTableName = right.getComponent(0).getSimple(); 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 physicalFields = sideTableInfo.getPhysicalFields(); String leftTableName = left.getComponent(0).getSimple(); String leftField = left.getComponent(1).getSimple(); @@ -94,7 +96,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]; @@ -110,7 +112,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]; From 75adb0b0cbb56e57797d915a9aadfe87ee0dc635 Mon Sep 17 00:00:00 2001 From: maqi Date: Tue, 14 Apr 2020 17:04:14 +0800 Subject: [PATCH 033/109] side predicate skip where case --- .../com/dtstack/flink/sql/side/SidePredicatesParser.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java index 99dc3507b..4d6112aa8 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java @@ -30,6 +30,7 @@ import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.table.calcite.FlinkPlannerImpl; import java.util.List; @@ -147,6 +148,10 @@ private void fillPredicateInfoToList(SqlBasicCall whereNode, List String content = (operatorKind == SqlKind.BETWEEN) ? whereNode.getOperands()[conditionIndex].toString() + " AND " + whereNode.getOperands()[2].toString() : whereNode.getOperands()[conditionIndex].toString(); + if (StringUtils.containsIgnoreCase(content,SqlKind.CASE.toString())) { + return; + } + PredicateInfo predicateInfo = PredicateInfo.builder().setOperatorName(operatorName).setOperatorKind(operatorKind.toString()) .setOwnerTable(ownerTable).setFieldName(fieldName).setCondition(content).build(); predicatesInfoList.add(predicateInfo); From 9bf9a659b25dc4b3c22511e53cef7fe00e3def6b Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 14 Apr 2020 17:46:52 +0800 Subject: [PATCH 034/109] =?UTF-8?q?=E4=BF=AE=E5=A4=8Dcoalesce=E5=87=BD?= =?UTF-8?q?=E6=95=B0=E4=B8=8D=E8=83=BD=E7=94=A8=E9=97=AE=E9=A2=98=E3=80=90?= =?UTF-8?q?http://redmine.prod.dtstack.cn/issues/24635=3Fissue=5Fcount=3D2?= =?UTF-8?q?&issue=5Fposition=3D1&next=5Fissue=5Fid=3D24600=E3=80=91?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java index 10919ca5b..4b05840d6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java @@ -253,6 +253,7 @@ public static SqlNode replaceSelectFieldName(SqlNode selectNode, || selectNode.getKind() == TIMESTAMP_ADD || selectNode.getKind() == TIMESTAMP_DIFF || selectNode.getKind() == LIKE + || selectNode.getKind() == COALESCE ){ SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; From 04550b05a4b7c47d56536f6b75d86ac17f0b2bbf Mon Sep 17 00:00:00 2001 From: dapeng Date: Wed, 15 Apr 2020 14:44:58 +0800 Subject: [PATCH 035/109] =?UTF-8?q?=E5=A4=84=E7=90=86=E5=BC=82=E5=B8=B8?= =?UTF-8?q?=E7=AD=96=E7=95=A5?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/side/AbstractSideTableInfo.java | 2 -- .../flink/sql/side/BaseAsyncReqRow.java | 23 +++++++------- .../sql/side/rdb/async/RdbAsyncReqRow.java | 30 +++++-------------- 3 files changed, 21 insertions(+), 34 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java index 71a309411..19d3cc5fb 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java @@ -54,8 +54,6 @@ public abstract class AbstractSideTableInfo extends AbstractTableInfo implements public static final String ASYNC_TIMEOUT_KEY = "asyncTimeout"; - public static final String ASYNC_TIMEOUT_NUM_KEY = "asyncTimeoutNum"; - public static final String ASYNC_FAIL_MAX_NUM_KEY = "asyncFailMaxNum"; private String cacheType = "none";//None or LRU or ALL 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 bfe14c3ea..7f8478183 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 @@ -51,6 +51,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; /** * All interfaces inherit naming rules: type + "AsyncReqRow" such as == "MysqlAsyncReqRow @@ -64,10 +65,9 @@ public abstract class BaseAsyncReqRow extends RichAsyncFunction impl private static final Logger LOG = LoggerFactory.getLogger(BaseAsyncReqRow.class); private static final long serialVersionUID = 2098635244857937717L; private RuntimeContext runtimeContext; - + private final static AtomicLong FAIL_NUM = new AtomicLong(0); private static int TIMEOUT_LOG_FLUSH_NUM = 10; private int timeOutNum = 0; - protected BaseSideInfo sideInfo; protected transient Counter parseErrorRecords; @@ -125,7 +125,7 @@ protected void dealMissKey(CRow input, ResultFuture resultFuture){ Row row = fillData(input.row(), null); resultFuture.complete(Collections.singleton(new CRow(row, input.change()))); } catch (Exception e) { - dealFillDataError(resultFuture, e, input); + dealFillDataError(input, resultFuture, e); } }else{ resultFuture.complete(null); @@ -205,7 +205,7 @@ private void invokeWithCache(Map inputParams, CRow input, Result Row row = fillData(input.row(), val); resultFuture.complete(Collections.singleton(new CRow(row, input.change()))); } catch (Exception e) { - dealFillDataError(resultFuture, e, input); + dealFillDataError(input, resultFuture, e); } } else if (ECacheContentType.MultiLine == val.getType()) { try { @@ -216,7 +216,7 @@ private void invokeWithCache(Map inputParams, CRow input, Result } resultFuture.complete(rowList); } catch (Exception e) { - dealFillDataError(resultFuture, e, input); + dealFillDataError(input, resultFuture, e); } } else { resultFuture.completeExceptionally(new RuntimeException("not support cache obj type " + val.getType())); @@ -256,11 +256,14 @@ protected void cancelTimerWhenComplete(ResultFuture resultFuture, Schedule } } - protected void dealFillDataError(ResultFuture resultFuture, Exception e, Object sourceData) { - LOG.debug("source data {} join side table error ", sourceData); - LOG.debug("async buid row error..{}", e); - parseErrorRecords.inc(); - resultFuture.complete(Collections.emptyList()); + protected void dealFillDataError(CRow input, ResultFuture resultFuture, Throwable e) { + if(FAIL_NUM.incrementAndGet() > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Long.MAX_VALUE)){ + LOG.info("dealFillDataError", e); + parseErrorRecords.inc(); + resultFuture.completeExceptionally(e); + } else { + dealMissKey(input, resultFuture); + } } @Override 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 ca6dcc38b..9571d7962 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 @@ -20,7 +20,6 @@ package com.dtstack.flink.sql.side.rdb.async; import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.metric.MetricConstant; import com.dtstack.flink.sql.side.BaseAsyncReqRow; import com.dtstack.flink.sql.side.BaseSideInfo; import com.dtstack.flink.sql.side.CacheMissVal; @@ -30,9 +29,7 @@ import io.vertx.core.json.JsonArray; import io.vertx.ext.sql.SQLClient; import io.vertx.ext.sql.SQLConnection; -import org.apache.calcite.sql.JoinType; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.metrics.Counter; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; @@ -79,12 +76,8 @@ public class RdbAsyncReqRow extends BaseAsyncReqRow { private final static AtomicBoolean CONN_STATUS = new AtomicBoolean(true); - private final static AtomicLong FAIL_NUM = new AtomicLong(0); - private Logger logger = LoggerFactory.getLogger(getClass()); - private Counter counter = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_SIDE_PARSE_ERROR_RECORDS); - public RdbAsyncReqRow(BaseSideInfo sideInfo) { super(sideInfo); } @@ -98,7 +91,12 @@ protected void preInvoke(CRow input, ResultFuture resultFuture){ @Override public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { + AtomicLong networkLogCounter = new AtomicLong(0L); while (!CONN_STATUS.get()){//network is unhealth + //todo:统一计数 + if(networkLogCounter.getAndIncrement() % 1000 == 0){ + LOG.info("network unhealth to block task"); + } Thread.sleep(100); } rdbSqlClient.getConnection(conn -> { @@ -127,12 +125,7 @@ private void connectWithRetry(Map inputParams, CRow input, Resul logger.error("getConnection error", conn.cause()); } if(failCounter.get() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3L)){ - if(FAIL_NUM.incrementAndGet() > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Long.MAX_VALUE)){ - counter.inc(); - resultFuture.completeExceptionally(conn.cause()); - } else { - dealMissKey(input, resultFuture); - } + dealFillDataError(input, resultFuture, conn.cause()); finishFlag.set(true); } conn.result().close(); @@ -144,8 +137,7 @@ private void connectWithRetry(Map inputParams, CRow input, Resul handleQuery(conn.result(), inputParams, input, resultFuture); finishFlag.set(true); } catch (Exception e) { - dealFillDataError(resultFuture, e, null); - logger.error("", e); + dealFillDataError(input, resultFuture, e); } finally { latch.countDown(); } @@ -210,13 +202,7 @@ private void handleQuery(SQLConnection connection,Map inputParam JsonArray params = new JsonArray(Lists.newArrayList(inputParams.values())); connection.queryWithParams(sideInfo.getSqlCondition(), params, rs -> { if (rs.failed()) { - if(FAIL_NUM.incrementAndGet() > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Long.MAX_VALUE)){ - LOG.error("Cannot retrieve the data from the database", rs.cause()); - counter.inc(); - resultFuture.completeExceptionally(rs.cause()); - } else { - dealMissKey(input, resultFuture); - } + dealFillDataError(input, resultFuture, rs.cause()); return; } From 96936193e3e18ad7d85b536401afffd30d95c8c1 Mon Sep 17 00:00:00 2001 From: tiezhu Date: Wed, 15 Apr 2020 15:29:49 +0800 Subject: [PATCH 036/109] update doc quickStart --- README.md | 4 +- docs/quickStart.md | 224 +++++++++++++++++++++++++++++++++++++++++---- 2 files changed, 210 insertions(+), 18 deletions(-) diff --git a/README.md b/README.md index e5d4622eb..30c95552c 100644 --- a/README.md +++ b/README.md @@ -15,8 +15,8 @@ FlinkStreamSQL * 自定义create view 语法 * 自定义create function 语法 * 实现了流与维表的join - * 支持原生FLinkSQL所有的语法 - * 扩展了输入和输出的性能指标到promethus + * 支持原生FlinkSQL所有的语法 + * 扩展了输入和输出的性能指标到Task metrics ## 目录 diff --git a/docs/quickStart.md b/docs/quickStart.md index dda046062..f1607c5e7 100644 --- a/docs/quickStart.md +++ b/docs/quickStart.md @@ -8,7 +8,7 @@ ### 1.2 执行环境 * Java: JDK8及以上 -* Flink集群: 1.4,1.5,1.8(单机模式不需要安装Flink集群) +* Flink集群: 1.4,1.5,1.8,1.9,1.10(单机模式不需要安装Flink集群) * 操作系统:理论上不限 * kerberos环境需要在flink-conf.yaml配置security.kerberos.login.keytab以及security.kerberos.login.principal参数,配置案例: ``` @@ -29,7 +29,6 @@ zookeeper.sasl.login-context-name: Client ``` mvn clean package -Dmaven.test.skip - ``` ####可运行的目录结构: @@ -37,9 +36,9 @@ mvn clean package -Dmaven.test.skip | |-----bin | |--- submit.sh 任务启动脚本 -|-----lib -| |--- sql.launcher.jar 包存储路径,是任务提交的入口(需要手动移动到该目录) -|-----plugins: 插件包存储路径(mvn 打包之后会自动将jar移动到该目录下) +|-----lib: launcher包存储路径,是任务提交的入口 +| |--- sql.launcher.jar +|-----sqlplugins: 插件包存储路径(mvn 打包之后会自动将jar移动到该目录下) | |--- core.jar | |--- xxxsource | |--- xxxsink @@ -49,15 +48,208 @@ mvn clean package -Dmaven.test.skip #### 1.4.1 启动命令 +```shell script +# 脚本启动 +sh submit.sh + -mode yarn + -name flink1.10_yarnSession + -sql F:\dtstack\stressTest\flinkStreamSql\1.10_dev\sql\flink1100.sql + -localSqlPluginPath F:\dtstack\project\flinkStreamSQL\plugins + -remoteSqlPluginPath F:\dtstack\project\flinkStreamSQL\plugins + -flinkconf F:\dtstack\flink\flink-1.10.0\conf + -yarnconf F:\dtstack\flinkStreamSql\yarnConf_node1 + -flinkJarPath F:\dtstack\flink\flink-1.10.0\lib + -pluginLoadMode shipfile + -confProp {\"time.characteristic\":\"eventTime\",\"logLevel\":\"info\"} + -yarnSessionConf {\"yid\":\"application_1586851105774_0014\"} +``` +```shell script +# 通过idea启动 程序入口类LaucherMain +# Run/Debug Configurations中设置Program arguments +-mode yarnPer +-sql /home/wen/Desktop/flink_stream_sql_conf/sql/stressTest.sql +-name stressTestAll +-localSqlPluginPath /home/wen/IdeaProjects/flinkStreamSQL/plugins +-remoteSqlPluginPath /home/wen/IdeaProjects/flinkStreamSQL/plugins +-flinkconf /home/wen/Desktop/flink_stream_sql_conf/flinkConf +-yarnconf /home/wen/Desktop/flink_stream_sql_conf/yarnConf_node1 +-flinkJarPath /home/wen/Desktop/dtstack/flink-1.8.1/lib +-pluginLoadMode shipfile +-confProp {\"time.characteristic\":\"eventTime\",\"logLevel\":\"info\"} +-queue c +``` +#### 1.4.2 命令参数说明 +* **mode** + * 描述:执行模式,也就是flink集群的工作模式 + * local: 本地模式 + * standalone: 提交到独立部署模式的flink集群 + * yarn: 提交到yarn模式的flink集群,该模式下需要提前启动一个yarn-session,使用默认名"Flink session cluster" + * yarnPer: yarn per_job模式提交(即创建新flink application),默认名为flink任务名称 + * 必选:否 + * 默认值:local + +* **name** + * 描述:flink 任务对应名称。 + * 必选:是 + * 默认值:无 + +* **sql** + * 描述:待执行的flink sql所在路径 。 + * 必选:是 + * 默认值:无 + +* **localSqlPluginPath** + * 描述:本地插件根目录地址,也就是打包后产生的plugins目录。 + * 必选:是 + * 默认值:无 + +* **remoteSqlPluginPath** + * 描述:flink执行集群上的插件根目录地址(将打包好的插件存放到各个flink节点上,如果是yarn集群需要存放到所有的nodemanager上)。 + * 必选:否 + * 默认值:无 + +* **addjar** + * 描述:扩展jar路径,当前主要是UDF定义的jar; + * 必选:否 + * 默认值:无 + +* **confProp** + * 描述:一些参数设置 + * 必选:是 (如无参数填写空json即可) + * 默认值:{} + * 可选参数: + * sql.ttl.min: 最小过期时间,大于0的整数,如1d、1h(d\D:天,h\H:小时,m\M:分钟,s\s:秒) + * sql.ttl.max: 最大过期时间,大于0的整数,如2d、2h(d\D:天,h\H:小时,m\M:分钟,s\s:秒),需同时设置最小时间,且比最小时间大5分钟 + * state.backend: 任务状态后端,可选为MEMORY,FILESYSTEM,ROCKSDB,默认为flinkconf中的配置。 + * state.checkpoints.dir: FILESYSTEM,ROCKSDB状态后端文件系统存储路径,例如:hdfs://ns1/dtInsight/flink180/checkpoints。 + * state.backend.incremental: ROCKSDB状态后端是否开启增量checkpoint,默认为true。 + * sql.env.parallelism: 默认并行度设置 + * sql.max.env.parallelism: 最大并行度设置 + * time.characteristic: 可选值[ProcessingTime|IngestionTime|EventTime] + * sql.checkpoint.interval: 设置了该参数表明开启checkpoint(ms) + * sql.checkpoint.mode: 可选值[EXACTLY_ONCE|AT_LEAST_ONCE] + * sql.checkpoint.timeout: 生成checkpoint的超时时间(ms) + * sql.max.concurrent.checkpoints: 最大并发生成checkpoint数 + * sql.checkpoint.cleanup.mode: 默认是不会将checkpoint存储到外部存储,[true(任务cancel之后会删除外部存储)|false(外部存储需要手动删除)] + * flinkCheckpointDataURI: 设置checkpoint的外部存储路径,根据实际的需求设定文件路径,hdfs://, file:// + * jobmanager.memory.mb: per_job模式下指定jobmanager的内存大小(单位MB, 默认值:768) + * taskmanager.memory.mb: per_job模式下指定taskmanager的内存大小(单位MB, 默认值:768) + * taskmanager.num: per_job模式下指定taskmanager的实例数(默认1) + * taskmanager.slots:per_job模式下指定每个taskmanager对应的slot数量(默认1) + * savePointPath:任务恢复点的路径(默认无) + * allowNonRestoredState:指示保存点是否允许非还原状态的标志(默认false) + * restore.enable:是否失败重启(默认是true) + * failure.interval:衡量失败率的时间段,单位分钟(默认6m) + * delay.interval:连续两次重启尝试间的间隔,单位是秒(默认10s) + * logLevel: 日志级别动态配置(默认info) + * [prometheus 相关参数](docs/prometheus.md) per_job可指定metric写入到外部监控组件,以prometheus pushgateway举例 + + +* **flinkconf** + * 描述:flink配置文件所在的目录(单机模式下不需要),如/hadoop/flink-1.10.0/conf + * 必选:否 + * 默认值:无 + +* **yarnconf** + * 描述:Hadoop配置文件(包括hdfs和yarn)所在的目录(单机模式下不需要),如/hadoop/etc/hadoop + * 必选:否 + * 默认值:无 + +* **flinkJarPath** + * 描述:per_job 模式提交需要指定本地的flink jar存放路径 + * 必选:否 + * 默认值:无 + +* **queue** + * 描述:per_job 模式下指定的yarn queue + * 必选:否 + * 默认值:default + +* **pluginLoadMode** + * 描述:per_job 模式下的插件包加载方式。classpath:从每台机器加载插件包,shipfile:将需要插件从提交的节点上传到hdfs,不需要每台安装插件 + * 必选:否 + * 默认值:classpath + +* **yarnSessionConf** + * 描述:yarn session 模式下指定的运行的一些参数,[可参考](https://ci.apache.org/projects/flink/flink-docs-release-1.8/ops/cli.html),目前只支持指定yid + * 必选:否 + * 默认值:{} + +## 1.5 任务样例 + ``` -sh submit.sh -sql D:\sideSql.txt --name xctest --remoteSqlPluginPath /opt/dtstack/150_flinkplugin/sqlplugin --localSqlPluginPath D:\gitspace\flinkStreamSQL\plugins --addjar \["udf.jar\"\] --mode yarn --flinkconf D:\flink_home\kudu150etc --yarnconf D:\hadoop\etc\hadoopkudu --confProp \{\"time.characteristic\":\"EventTime\",\"sql.checkpoint.interval\":10000\} --yarnSessionConf \{\"yid\":\"application_1564971615273_38182\"} -``` \ No newline at end of file +# 一个kafka join all维表 sink kafka的样例 +CREATE TABLE MyTable( + id bigint, + name varchar, + address varchar +)WITH( + type = 'kafka10', + bootstrapServers = '172.16.101.224:9092', + zookeeperQuorm = '172.16.100.188:2181/kafka', + offsetReset = 'latest', + topic = 'tiezhu_test_in', + groupId = 'flink_sql', + timezone = 'Asia/Shanghai', + topicIsPattern = 'false', + parallelism = '1' +); + +CREATE TABLE sideTable( + id bigint, + school varchar, + home varchar, + PRIMARY KEY(id), + PERIOD FOR SYSTEM_TIME +)WITH( + type='mysql', + url='jdbc:mysql://172.16.8.109:3306/tiezhu', + userName='dtstack', + password='you-guess', + tableName='stressTest', + cache='ALL', + parallelism='1', + asyncCapacity='100' +); + +CREATE TABLE MyResult( + id bigint, + name varchar, + address varchar, + home varchar, + school varchar +)WITH( + type = 'kafka10', + bootstrapServers = '172.16.101.224:9092', + zookeeperQuorm = '172.16.100.188:2181/kafka', + offsetReset = 'latest', + topic = 'tiezhu_test_out', + parallelism = '1' +); + + +insert +into + MyResult + select + t1.id AS id, + t1.name AS name, + t1.address AS address, + t2.school AS school, + t2.home AS home + from + ( + select + id, + name, + address + from + MyTable + ) t1 + left join sideTable t2 + on t2.id = t2.id; +``` + +# 招聘 +1.大数据平台开发工程师,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至sishu@dtstack.com。 + From e496e3376be02a05badcc88cea7d0d2950fa9eda Mon Sep 17 00:00:00 2001 From: tiezhu Date: Wed, 15 Apr 2020 15:37:24 +0800 Subject: [PATCH 037/109] =?UTF-8?q?=E6=9B=B4=E6=96=B0=E4=B8=AA=E5=88=AB?= =?UTF-8?q?=E5=86=85=E5=AE=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/quickStart.md | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/docs/quickStart.md b/docs/quickStart.md index f1607c5e7..52fc3af4e 100644 --- a/docs/quickStart.md +++ b/docs/quickStart.md @@ -22,11 +22,8 @@ zookeeper.sasl.service-name: zookeeper zookeeper.sasl.login-context-name: Client ``` - ### 1.3 打包 - 进入项目根目录,使用maven打包: - ``` mvn clean package -Dmaven.test.skip ``` @@ -38,13 +35,13 @@ mvn clean package -Dmaven.test.skip | |--- submit.sh 任务启动脚本 |-----lib: launcher包存储路径,是任务提交的入口 | |--- sql.launcher.jar -|-----sqlplugins: 插件包存储路径(mvn 打包之后会自动将jar移动到该目录下) +|-----plugins: 插件包存储路径(mvn 打包之后会自动将jar移动到该目录下) | |--- core.jar | |--- xxxsource | |--- xxxsink | |--- xxxside ``` -### 1.4 启动 +### 1.4 快速启动 #### 1.4.1 启动命令 @@ -115,7 +112,7 @@ sh submit.sh * **confProp** * 描述:一些参数设置 - * 必选:是 (如无参数填写空json即可) + * 必选:否 * 默认值:{} * 可选参数: * sql.ttl.min: 最小过期时间,大于0的整数,如1d、1h(d\D:天,h\H:小时,m\M:分钟,s\s:秒) @@ -176,9 +173,8 @@ sh submit.sh * 默认值:{} ## 1.5 任务样例 - ``` -# 一个kafka join all维表 sink kafka的样例 +# 一个 kafka source join all维表 到 kafka sink的样例 CREATE TABLE MyTable( id bigint, name varchar, From 88d73ed3122c3f5cede49eef0abfb8cf2864e36e Mon Sep 17 00:00:00 2001 From: tiezhu Date: Thu, 16 Apr 2020 09:42:52 +0800 Subject: [PATCH 038/109] =?UTF-8?q?=E4=BF=AE=E6=94=B9config=E4=B8=AD?= =?UTF-8?q?=E9=83=A8=E5=88=86=E4=BF=A1=E6=81=AF?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/config.md | 20 +++++++++----------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/docs/config.md b/docs/config.md index 48aeefc5a..3fb6ac723 100644 --- a/docs/config.md +++ b/docs/config.md @@ -9,8 +9,8 @@ sh submit.sh -key1 val1 -key2 val2 * 描述:执行模式,也就是flink集群的工作模式 * local: 本地模式 * standalone: 提交到独立部署模式的flink集群 - * yarn: 提交到yarn模式的flink集群(即提交到已有flink集群) - * yarnPer: yarn per_job模式提交(即创建新flink application) + * yarn: 提交到yarn模式的flink集群,该模式下需要提前启动一个yarn-session,使用默认名"Flink session cluster" + * yarnPer: yarn per_job模式提交(即创建新flink application),默认名为flink任务名称 * 必选:否 * 默认值:local @@ -28,7 +28,7 @@ sh submit.sh -key1 val1 -key2 val2 * 默认值:无 * **sql** - * 描述:执行flink sql 的主体语句。 + * 描述:待执行的flink sql所在路径 。 * 必选:是 * 默认值:无 @@ -44,15 +44,13 @@ sh submit.sh -key1 val1 -key2 val2 * **addjar** * 描述:扩展jar路径,当前主要是UDF定义的jar; - * 格式:json * 必选:否 * 默认值:无 * **confProp** * 描述:一些参数设置 - * 格式: json - * 必选:是 (如无参数填写空json即可) - * 默认值:无 + * 必选:否 + * 默认值:{} * 可选参数: * sql.ttl.min: 最小过期时间,大于0的整数,如1d、1h(d\D:天,h\H:小时,m\M:分钟,s\s:秒) * sql.ttl.max: 最大过期时间,大于0的整数,如2d、2h(d\D:天,h\H:小时,m\M:分钟,s\s:秒),需同时设置最小时间,且比最小时间大5分钟 @@ -79,7 +77,7 @@ sh submit.sh -key1 val1 -key2 val2 * **flinkconf** - * 描述:flink配置文件所在的目录(单机模式下不需要),如/hadoop/flink-1.4.0/conf + * 描述:flink配置文件所在的目录(单机模式下不需要),如/hadoop/flink-1.10.0/conf * 必选:否 * 默认值:无 @@ -91,15 +89,15 @@ sh submit.sh -key1 val1 -key2 val2 * **flinkJarPath** * 描述:yarnPer 模式提交需要指定本地的flink jar存放路径 * 必选:否 - * 默认值:false + * 默认值:无 * **queue** * 描述:yarnPer 模式下指定的yarn queue * 必选:否 - * 默认值:false + * 默认值:default * **yarnSessionConf** * 描述:yarn session 模式下指定的运行的一些参数,[可参考](https://ci.apache.org/projects/flink/flink-docs-release-1.8/ops/cli.html),目前只支持指定yid * 必选:否 - * 默认值:false + * 默认值:{} \ No newline at end of file From 9b451a8ef1ef906e9efef9f5d95050cb47719713 Mon Sep 17 00:00:00 2001 From: tiezhu Date: Thu, 16 Apr 2020 09:59:32 +0800 Subject: [PATCH 039/109] =?UTF-8?q?=E4=BF=AE=E6=94=B9quickStart=E6=96=87?= =?UTF-8?q?=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/quickStart.md | 202 ++++----------------------------------------- 1 file changed, 16 insertions(+), 186 deletions(-) diff --git a/docs/quickStart.md b/docs/quickStart.md index 52fc3af4e..a51f234e3 100644 --- a/docs/quickStart.md +++ b/docs/quickStart.md @@ -11,7 +11,7 @@ * Flink集群: 1.4,1.5,1.8,1.9,1.10(单机模式不需要安装Flink集群) * 操作系统:理论上不限 * kerberos环境需要在flink-conf.yaml配置security.kerberos.login.keytab以及security.kerberos.login.principal参数,配置案例: -``` +```yaml #提交到hadoop环境一定要配置fs.hdfs.hadoopconf参数 fs.hdfs.hadoopconf: /Users/maqi/tmp/hadoopconf/hadoop_250 security.kerberos.login.use-ticket-cache: true @@ -24,7 +24,7 @@ zookeeper.sasl.login-context-name: Client ### 1.3 打包 进入项目根目录,使用maven打包: -``` +```shell script mvn clean package -Dmaven.test.skip ``` @@ -59,193 +59,23 @@ sh submit.sh -pluginLoadMode shipfile -confProp {\"time.characteristic\":\"eventTime\",\"logLevel\":\"info\"} -yarnSessionConf {\"yid\":\"application_1586851105774_0014\"} -``` -```shell script -# 通过idea启动 程序入口类LaucherMain -# Run/Debug Configurations中设置Program arguments --mode yarnPer --sql /home/wen/Desktop/flink_stream_sql_conf/sql/stressTest.sql --name stressTestAll --localSqlPluginPath /home/wen/IdeaProjects/flinkStreamSQL/plugins --remoteSqlPluginPath /home/wen/IdeaProjects/flinkStreamSQL/plugins --flinkconf /home/wen/Desktop/flink_stream_sql_conf/flinkConf --yarnconf /home/wen/Desktop/flink_stream_sql_conf/yarnConf_node1 --flinkJarPath /home/wen/Desktop/dtstack/flink-1.8.1/lib --pluginLoadMode shipfile --confProp {\"time.characteristic\":\"eventTime\",\"logLevel\":\"info\"} --queue c -``` -#### 1.4.2 命令参数说明 -* **mode** - * 描述:执行模式,也就是flink集群的工作模式 - * local: 本地模式 - * standalone: 提交到独立部署模式的flink集群 - * yarn: 提交到yarn模式的flink集群,该模式下需要提前启动一个yarn-session,使用默认名"Flink session cluster" - * yarnPer: yarn per_job模式提交(即创建新flink application),默认名为flink任务名称 - * 必选:否 - * 默认值:local - -* **name** - * 描述:flink 任务对应名称。 - * 必选:是 - * 默认值:无 -* **sql** - * 描述:待执行的flink sql所在路径 。 - * 必选:是 - * 默认值:无 - -* **localSqlPluginPath** - * 描述:本地插件根目录地址,也就是打包后产生的plugins目录。 - * 必选:是 - * 默认值:无 - -* **remoteSqlPluginPath** - * 描述:flink执行集群上的插件根目录地址(将打包好的插件存放到各个flink节点上,如果是yarn集群需要存放到所有的nodemanager上)。 - * 必选:否 - * 默认值:无 - -* **addjar** - * 描述:扩展jar路径,当前主要是UDF定义的jar; - * 必选:否 - * 默认值:无 - -* **confProp** - * 描述:一些参数设置 - * 必选:否 - * 默认值:{} - * 可选参数: - * sql.ttl.min: 最小过期时间,大于0的整数,如1d、1h(d\D:天,h\H:小时,m\M:分钟,s\s:秒) - * sql.ttl.max: 最大过期时间,大于0的整数,如2d、2h(d\D:天,h\H:小时,m\M:分钟,s\s:秒),需同时设置最小时间,且比最小时间大5分钟 - * state.backend: 任务状态后端,可选为MEMORY,FILESYSTEM,ROCKSDB,默认为flinkconf中的配置。 - * state.checkpoints.dir: FILESYSTEM,ROCKSDB状态后端文件系统存储路径,例如:hdfs://ns1/dtInsight/flink180/checkpoints。 - * state.backend.incremental: ROCKSDB状态后端是否开启增量checkpoint,默认为true。 - * sql.env.parallelism: 默认并行度设置 - * sql.max.env.parallelism: 最大并行度设置 - * time.characteristic: 可选值[ProcessingTime|IngestionTime|EventTime] - * sql.checkpoint.interval: 设置了该参数表明开启checkpoint(ms) - * sql.checkpoint.mode: 可选值[EXACTLY_ONCE|AT_LEAST_ONCE] - * sql.checkpoint.timeout: 生成checkpoint的超时时间(ms) - * sql.max.concurrent.checkpoints: 最大并发生成checkpoint数 - * sql.checkpoint.cleanup.mode: 默认是不会将checkpoint存储到外部存储,[true(任务cancel之后会删除外部存储)|false(外部存储需要手动删除)] - * flinkCheckpointDataURI: 设置checkpoint的外部存储路径,根据实际的需求设定文件路径,hdfs://, file:// - * jobmanager.memory.mb: per_job模式下指定jobmanager的内存大小(单位MB, 默认值:768) - * taskmanager.memory.mb: per_job模式下指定taskmanager的内存大小(单位MB, 默认值:768) - * taskmanager.num: per_job模式下指定taskmanager的实例数(默认1) - * taskmanager.slots:per_job模式下指定每个taskmanager对应的slot数量(默认1) - * savePointPath:任务恢复点的路径(默认无) - * allowNonRestoredState:指示保存点是否允许非还原状态的标志(默认false) - * restore.enable:是否失败重启(默认是true) - * failure.interval:衡量失败率的时间段,单位分钟(默认6m) - * delay.interval:连续两次重启尝试间的间隔,单位是秒(默认10s) - * logLevel: 日志级别动态配置(默认info) - * [prometheus 相关参数](docs/prometheus.md) per_job可指定metric写入到外部监控组件,以prometheus pushgateway举例 - - -* **flinkconf** - * 描述:flink配置文件所在的目录(单机模式下不需要),如/hadoop/flink-1.10.0/conf - * 必选:否 - * 默认值:无 - -* **yarnconf** - * 描述:Hadoop配置文件(包括hdfs和yarn)所在的目录(单机模式下不需要),如/hadoop/etc/hadoop - * 必选:否 - * 默认值:无 - -* **flinkJarPath** - * 描述:per_job 模式提交需要指定本地的flink jar存放路径 - * 必选:否 - * 默认值:无 - -* **queue** - * 描述:per_job 模式下指定的yarn queue - * 必选:否 - * 默认值:default - -* **pluginLoadMode** - * 描述:per_job 模式下的插件包加载方式。classpath:从每台机器加载插件包,shipfile:将需要插件从提交的节点上传到hdfs,不需要每台安装插件 - * 必选:否 - * 默认值:classpath - -* **yarnSessionConf** - * 描述:yarn session 模式下指定的运行的一些参数,[可参考](https://ci.apache.org/projects/flink/flink-docs-release-1.8/ops/cli.html),目前只支持指定yid - * 必选:否 - * 默认值:{} - -## 1.5 任务样例 +# mode: 任务启动的模式 +# name: 本次任务名称 +# sql: 本次任务执行sql脚本 +# localSqPluginPath: 本地插件包根目录地址 +# remoteSqlPluginPath: flink执行集群上的插件根目录地址 +# flinkconf: flink配置文件所在目录(单机模式下不需要) +# yarnconf: Hadoop配置文件(包括hdfs和yarn)所在目录 +# flinkJarPath: yarnPer模式提交需要指定本地的flink jar存放路径 +# pluginLoadMode:yarnPer模式下的插件包加载方式 +# confProp: 其他额外参数配置 +# yarnSessionConf: yarnSession模式下指定的运行参数,目前只支持指定yid ``` -# 一个 kafka source join all维表 到 kafka sink的样例 -CREATE TABLE MyTable( - id bigint, - name varchar, - address varchar -)WITH( - type = 'kafka10', - bootstrapServers = '172.16.101.224:9092', - zookeeperQuorm = '172.16.100.188:2181/kafka', - offsetReset = 'latest', - topic = 'tiezhu_test_in', - groupId = 'flink_sql', - timezone = 'Asia/Shanghai', - topicIsPattern = 'false', - parallelism = '1' -); - -CREATE TABLE sideTable( - id bigint, - school varchar, - home varchar, - PRIMARY KEY(id), - PERIOD FOR SYSTEM_TIME -)WITH( - type='mysql', - url='jdbc:mysql://172.16.8.109:3306/tiezhu', - userName='dtstack', - password='you-guess', - tableName='stressTest', - cache='ALL', - parallelism='1', - asyncCapacity='100' -); +参数具体细节请看[命令参数说明](./config.md) -CREATE TABLE MyResult( - id bigint, - name varchar, - address varchar, - home varchar, - school varchar -)WITH( - type = 'kafka10', - bootstrapServers = '172.16.101.224:9092', - zookeeperQuorm = '172.16.100.188:2181/kafka', - offsetReset = 'latest', - topic = 'tiezhu_test_out', - parallelism = '1' -); - - -insert -into - MyResult - select - t1.id AS id, - t1.name AS name, - t1.address AS address, - t2.school AS school, - t2.home AS home - from - ( - select - id, - name, - address - from - MyTable - ) t1 - left join sideTable t2 - on t2.id = t2.id; -``` +任务sql详情请看[任务样例](./demo.md) -# 招聘 +### 招聘 1.大数据平台开发工程师,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至sishu@dtstack.com。 From 28085f5e8e4c59c8d5b2d3f0db4bc136956ab919 Mon Sep 17 00:00:00 2001 From: tiezhu Date: Thu, 16 Apr 2020 10:06:43 +0800 Subject: [PATCH 040/109] =?UTF-8?q?=E4=BF=AE=E6=94=B9quickStart=E9=83=A8?= =?UTF-8?q?=E5=88=86=E5=86=85=E5=AE=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/quickStart.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/quickStart.md b/docs/quickStart.md index a51f234e3..d7b613092 100644 --- a/docs/quickStart.md +++ b/docs/quickStart.md @@ -65,7 +65,7 @@ sh submit.sh # sql: 本次任务执行sql脚本 # localSqPluginPath: 本地插件包根目录地址 # remoteSqlPluginPath: flink执行集群上的插件根目录地址 -# flinkconf: flink配置文件所在目录(单机模式下不需要) +# flinkconf: flink配置文件所在目录(local模式下不需要) # yarnconf: Hadoop配置文件(包括hdfs和yarn)所在目录 # flinkJarPath: yarnPer模式提交需要指定本地的flink jar存放路径 # pluginLoadMode:yarnPer模式下的插件包加载方式 From 393c98e5cd089f58ac678866d37228e7d2d27fbe Mon Sep 17 00:00:00 2001 From: dapeng Date: Thu, 16 Apr 2020 10:50:38 +0800 Subject: [PATCH 041/109] =?UTF-8?q?=E5=8E=BB=E9=99=A4=E5=A4=9A=E4=BD=99?= =?UTF-8?q?=E6=97=A5=E5=BF=97logger=E5=92=8C=E9=87=87=E7=94=A8=E5=85=A8?= =?UTF-8?q?=E5=B1=80=E8=AE=A1=E6=95=B0=E5=99=A8?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/side/BaseAsyncReqRow.java | 6 ++---- .../dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java | 8 ++------ 2 files changed, 4 insertions(+), 10 deletions(-) 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 7f8478183..0e59f4590 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 @@ -51,7 +51,6 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; /** * All interfaces inherit naming rules: type + "AsyncReqRow" such as == "MysqlAsyncReqRow @@ -65,7 +64,6 @@ public abstract class BaseAsyncReqRow extends RichAsyncFunction impl private static final Logger LOG = LoggerFactory.getLogger(BaseAsyncReqRow.class); private static final long serialVersionUID = 2098635244857937717L; private RuntimeContext runtimeContext; - private final static AtomicLong FAIL_NUM = new AtomicLong(0); private static int TIMEOUT_LOG_FLUSH_NUM = 10; private int timeOutNum = 0; protected BaseSideInfo sideInfo; @@ -257,9 +255,9 @@ protected void cancelTimerWhenComplete(ResultFuture resultFuture, Schedule } protected void dealFillDataError(CRow input, ResultFuture resultFuture, Throwable e) { - if(FAIL_NUM.incrementAndGet() > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Long.MAX_VALUE)){ + parseErrorRecords.inc(); + if(parseErrorRecords.getCount() > sideInfo.getSideTableInfo().getAsyncFailMaxNum(Long.MAX_VALUE)){ LOG.info("dealFillDataError", e); - parseErrorRecords.inc(); resultFuture.completeExceptionally(e); } else { dealMissKey(input, resultFuture); 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 9571d7962..b34e8297c 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 @@ -76,13 +76,10 @@ public class RdbAsyncReqRow extends BaseAsyncReqRow { private final static AtomicBoolean CONN_STATUS = new AtomicBoolean(true); - private Logger logger = LoggerFactory.getLogger(getClass()); - public RdbAsyncReqRow(BaseSideInfo sideInfo) { super(sideInfo); } - @Override protected void preInvoke(CRow input, ResultFuture resultFuture){ @@ -93,7 +90,6 @@ public void handleAsyncInvoke(Map inputParams, CRow input, Resul AtomicLong networkLogCounter = new AtomicLong(0L); while (!CONN_STATUS.get()){//network is unhealth - //todo:统一计数 if(networkLogCounter.getAndIncrement() % 1000 == 0){ LOG.info("network unhealth to block task"); } @@ -122,7 +118,7 @@ private void connectWithRetry(Map inputParams, CRow input, Resul try { if(conn.failed()){ if(failCounter.getAndIncrement() % 1000 == 0){ - logger.error("getConnection error", conn.cause()); + LOG.error("getConnection error", conn.cause()); } if(failCounter.get() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3L)){ dealFillDataError(input, resultFuture, conn.cause()); @@ -146,7 +142,7 @@ private void connectWithRetry(Map inputParams, CRow input, Resul try { latch.wait(); } catch (InterruptedException e) { - logger.error("", e); + LOG.error("", e); } } From d394f209f7dc44f4880066db0ed56ee0ff6eca8b Mon Sep 17 00:00:00 2001 From: tiezhu Date: Thu, 16 Apr 2020 10:55:18 +0800 Subject: [PATCH 042/109] =?UTF-8?q?quickStart=20=E6=B7=BB=E5=8A=A0?= =?UTF-8?q?=E5=9B=9B=E7=A7=8D=E6=A8=A1=E5=BC=8F=E5=91=BD=E4=BB=A4?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/quickStart.md | 48 +++++++++++++++++++++++++++++++++------------- 1 file changed, 35 insertions(+), 13 deletions(-) diff --git a/docs/quickStart.md b/docs/quickStart.md index d7b613092..f2dcc39ab 100644 --- a/docs/quickStart.md +++ b/docs/quickStart.md @@ -46,7 +46,27 @@ mvn clean package -Dmaven.test.skip #### 1.4.1 启动命令 ```shell script -# 脚本启动 +# local模式命令 +sh submit.sh + -mode local + -name local_test + -sql F:\dtstack\stressTest\flinkStreamSql\stressTest.sql + -localSqlPluginPath F:\dtstack\project\flinkStreamSQL\plugins + +#standalone模式命令 +sh submit.sh + -mode standalone + -sql F:\dtstack\flinkStreamSql\tiezhu\twodimjoin.sql + -name wtz_standalone_flinkStreamSql + -localSqlPluginPath F:\dtstack\project\flinkStreamSQL\plugins + -remoteSqlPluginPath /home/admin/dtstack/flinkStreamSQL/plugins + -flinkconf F:\dtstack\flinkStreamSql\localhost\flinkConf + -yarnconf F:\dtstack\flinkStreamSql\localhost\hadoop + -flinkJarPath F:\Java\flink-1.8.2-bin-scala_2.12\flink-1.8.2\lib + -pluginLoadMode shipfile + -confProp {\"time.characteristic\":\"eventTime\",\"logLevel\":\"info\"} + +# yarn模式命令 sh submit.sh -mode yarn -name flink1.10_yarnSession @@ -57,20 +77,22 @@ sh submit.sh -yarnconf F:\dtstack\flinkStreamSql\yarnConf_node1 -flinkJarPath F:\dtstack\flink\flink-1.10.0\lib -pluginLoadMode shipfile - -confProp {\"time.characteristic\":\"eventTime\",\"logLevel\":\"info\"} -yarnSessionConf {\"yid\":\"application_1586851105774_0014\"} -# mode: 任务启动的模式 -# name: 本次任务名称 -# sql: 本次任务执行sql脚本 -# localSqPluginPath: 本地插件包根目录地址 -# remoteSqlPluginPath: flink执行集群上的插件根目录地址 -# flinkconf: flink配置文件所在目录(local模式下不需要) -# yarnconf: Hadoop配置文件(包括hdfs和yarn)所在目录 -# flinkJarPath: yarnPer模式提交需要指定本地的flink jar存放路径 -# pluginLoadMode:yarnPer模式下的插件包加载方式 -# confProp: 其他额外参数配置 -# yarnSessionConf: yarnSession模式下指定的运行参数,目前只支持指定yid +# yarnPer模式命令 +sh submit.sh + -mode yarnPer + -sql /home/wen/Desktop/flink_stream_sql_conf/sql/Test01.sql + -name TestAll + -localSqlPluginPath /home/wen/IdeaProjects/flinkStreamSQL/plugins + -remoteSqlPluginPath /home/wen/IdeaProjects/flinkStreamSQL/plugins + -flinkconf /home/wen/Desktop/flink_stream_sql_conf/flinkConf + -yarnconf /home/wen/Desktop/flink_stream_sql_conf/yarnConf_node1 + -flinkJarPath /home/wen/Desktop/dtstack/flink-1.8.1/lib + -pluginLoadMode shipfile + -confProp {\"time.characteristic\":\"eventTime\",\"logLevel\":\"info\"} + -queue c + ``` 参数具体细节请看[命令参数说明](./config.md) From d68751dfe9422d1f659256396663b6eb1ef8f3c0 Mon Sep 17 00:00:00 2001 From: tiezhu Date: Thu, 16 Apr 2020 11:00:06 +0800 Subject: [PATCH 043/109] =?UTF-8?q?=E4=BF=AE=E6=94=B9quickStart=E6=8E=92?= =?UTF-8?q?=E7=89=88?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/quickStart.md | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/docs/quickStart.md b/docs/quickStart.md index f2dcc39ab..a5bf1fa9a 100644 --- a/docs/quickStart.md +++ b/docs/quickStart.md @@ -23,6 +23,7 @@ zookeeper.sasl.login-context-name: Client ``` ### 1.3 打包 + 进入项目根目录,使用maven打包: ```shell script mvn clean package -Dmaven.test.skip @@ -41,19 +42,19 @@ mvn clean package -Dmaven.test.skip | |--- xxxsink | |--- xxxside ``` -### 1.4 快速启动 - -#### 1.4.1 启动命令 +### 1.4 快速启动命令 +#### local模式命令 ```shell script -# local模式命令 sh submit.sh -mode local -name local_test -sql F:\dtstack\stressTest\flinkStreamSql\stressTest.sql -localSqlPluginPath F:\dtstack\project\flinkStreamSQL\plugins +``` -#standalone模式命令 +#### standalone模式命令 +```shell script sh submit.sh -mode standalone -sql F:\dtstack\flinkStreamSql\tiezhu\twodimjoin.sql @@ -65,8 +66,10 @@ sh submit.sh -flinkJarPath F:\Java\flink-1.8.2-bin-scala_2.12\flink-1.8.2\lib -pluginLoadMode shipfile -confProp {\"time.characteristic\":\"eventTime\",\"logLevel\":\"info\"} +``` -# yarn模式命令 +#### yarn模式命令 +```shell script sh submit.sh -mode yarn -name flink1.10_yarnSession @@ -78,8 +81,10 @@ sh submit.sh -flinkJarPath F:\dtstack\flink\flink-1.10.0\lib -pluginLoadMode shipfile -yarnSessionConf {\"yid\":\"application_1586851105774_0014\"} +``` -# yarnPer模式命令 +#### yarnPer模式命令 +```shell script sh submit.sh -mode yarnPer -sql /home/wen/Desktop/flink_stream_sql_conf/sql/Test01.sql @@ -92,7 +97,6 @@ sh submit.sh -pluginLoadMode shipfile -confProp {\"time.characteristic\":\"eventTime\",\"logLevel\":\"info\"} -queue c - ``` 参数具体细节请看[命令参数说明](./config.md) From 92baf834cd0a75230e5b5ff11fc5e0456b496070 Mon Sep 17 00:00:00 2001 From: maqi Date: Thu, 16 Apr 2020 11:56:54 +0800 Subject: [PATCH 044/109] add rdbSideParams.md --- docs/plugin/rdbSideParams.md | 41 ++++++++++++++++++++++++++++++++++++ 1 file changed, 41 insertions(+) create mode 100644 docs/plugin/rdbSideParams.md diff --git a/docs/plugin/rdbSideParams.md b/docs/plugin/rdbSideParams.md new file mode 100644 index 000000000..26666bd5b --- /dev/null +++ b/docs/plugin/rdbSideParams.md @@ -0,0 +1,41 @@ +## 关系型数据库维表参数 + +适用于`MYSQL`,`ORACLE`,`SQLSERVER`,`POSTGRESQL`,`DB2`,`POLARDB`,`CLICKHOUSE`,`IMPALA`维表插件 + +### 维表参数 + +|参数名称|含义|是否必填|默认值| +|----|---|---|----| +| type | 维表类型, 例如:mysql |是|| +| url | 连接数据库 jdbcUrl |是|| +| userName | 连接用户名 |是|| +| password | 连接密码|是|| +| tableName| 表名称|是|| +| schema| 表空间|否|| +| cache | 维表缓存策略(NONE/LRU/ALL)|否|LRU| +| partitionedJoin | 是否在維表join之前先根据设定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| + +### 缓存策略 + +- NONE:不做内存缓存。每条流数据触发一次维表查询操作。 +- ALL: 任务启动时,一次性加载所有数据到内存,并进行缓存。适用于维表数据量较小的情况。 +- LRU: 任务执行时,根据维表关联条件使用异步算子加载维表数据,并进行缓存。 + +#### ALL全量维表参数 + +|参数名称|含义|默认值| +|----|---|----| +| cacheTTLMs | 缓存周期刷新时间 |60,单位s| + +#### LRU异步维表参数 + +|参数名称|含义|默认值| +|----|---|----| +| cacheTTLMs | LRU缓存写入后超时时间 |60,单位s| +| cacheSize | LRU缓存大小 |10000| +| cacheMode | 异步请求处理有序还是无序,可选:ordered,unordered |ordered| +| asyncCapacity | 异步线程容量 |100| +| asyncTimeout | 异步处理超时时间 |10000,单位毫秒| +| asyncPoolSize | 异步查询DB最大线程池,上限20 |min(20,Runtime.getRuntime().availableProcessors() * 2)| + + From 4566ece8e09a97af3e16e773f8298e018007b062 Mon Sep 17 00:00:00 2001 From: maqi Date: Thu, 16 Apr 2020 14:27:11 +0800 Subject: [PATCH 045/109] oracle mysql side table doc --- docs/plugin/mysqlSide.md | 149 ++++++++++++++++++++++++----------- docs/plugin/oracleSide.md | 133 +++++++++++++++++++++++-------- docs/plugin/rdbSideParams.md | 1 + 3 files changed, 203 insertions(+), 80 deletions(-) diff --git a/docs/plugin/mysqlSide.md b/docs/plugin/mysqlSide.md index 6a0e0aea9..96e997e9e 100644 --- a/docs/plugin/mysqlSide.md +++ b/docs/plugin/mysqlSide.md @@ -1,11 +1,15 @@ ## 1.格式: + + 通过建表语句中的` PERIOD FOR SYSTEM_TIME`将表标识为维表,其中`PRIMARY KEY(keyInfo)`中的keyInfo,表示用来和源表进行关联的字段, + 维表JOIN的条件必须与`keyInfo`字段一致。 + ``` CREATE TABLE tableName( colName cloType, ... PRIMARY KEY(keyInfo), - PERIOD FOR SYSTEM_TIME + PERIOD FOR SYSTEM_TIME )WITH( type='mysql', url='jdbcUrl', @@ -24,62 +28,117 @@ mysql-5.6.35 ## 3.表结构定义 - - |参数名称|含义| - |----|---| - | tableName | mysql表名称| - | colName | 列名称| - | colType | 列类型 [colType支持的类型](docs/colType.md)| - | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| - | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| + + [关系型数据库维表参数](docs/rdbSideParams.md) -## 4.参数 +## 4.样例 - |参数名称|含义|是否必填|默认值| - |----|---|---|----| - | type | 表明维表的类型 mysql |是|| - | url | 连接mysql数据库 jdbcUrl |是|| - | userName | mysql连接用户名 |是|| - | password | mysql连接密码|是|| - | tableName | mysql表名称|是|| - | tableName | mysql 的表名称|是|| - | cache | 维表缓存策略(NONE/LRU)|否|NONE| - | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| - - ---------- - > 缓存策略 - * NONE: 不做内存缓存 - * LRU: - * cacheSize: 缓存的条目数量 - * cacheTTLMs:缓存的过期时间(ms) - * cacheMode: (unordered|ordered)异步加载是有序还是无序,默认有序。 - * asyncCapacity:异步请求容量,默认1000 - * asyncTimeout:异步请求超时时间,默认10000毫秒 +### ALL全量维表定义 +``` + // 定义全量维表 +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , + PERIOD FOR SYSTEM_TIME + )WITH( + type ='mysql', + url ='jdbc:mysql://172.16.10.204:3306/mqtest', + userName ='dtstack', + password ='1abc123', + tableName ='test_mysql_10', + cache ='ALL', + cacheTTLMs ='60000', + parallelism ='2' + ); -## 5.样例 ``` -create table sideTable( - channel varchar, - xccount int, - PRIMARY KEY(channel), +### LRU异步维表定义 + +``` +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , PERIOD FOR SYSTEM_TIME )WITH( - type='mysql', - url='jdbc:mysql://172.16.8.104:3306/test?charset=utf8', - userName='dtstack', - password='abc123', - tableName='sidetest', + type ='mysql', + url ='jdbc:mysql://172.16.10.204:3306/mqtest', + userName ='dtstack', + password ='1abc123', + tableName ='yctest_mysql_10', + partitionedJoin ='false', cache ='LRU', cacheSize ='10000', cacheTTLMs ='60000', - cacheMode='unordered', - asyncCapacity='1000', - asyncTimeout='10000' - parallelism ='1', - partitionedJoin='false' + asyncPoolSize ='3', + parallelism ='2' ); +``` + +### MySQL异步维表关联 ``` +CREATE TABLE MyTable( + id int, + name varchar + )WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='cannan_yctest01', + timezone='Asia/Shanghai', + enableKeyPartitions ='false', + topicIsPattern ='false', + parallelism ='1' + ); + +CREATE TABLE MyResult( + id INT, + name VARCHAR + )WITH( + type ='mysql', + url ='jdbc:mysql://172.16.10.204:3306/mqtest', + userName ='dtstack', + password ='1abc123', + tableName ='yctest_mysql_mq', + updateMode ='append', + parallelism ='1', + batchSize ='100', + batchWaitInterval ='1000' + ); + +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , + PERIOD FOR SYSTEM_TIME + )WITH( + type ='mysql', + url ='jdbc:mysql://172.16.10.204:3306/mqtest', + userName ='dtstack', + password ='1abc123', + tableName ='yctest_mysql_10', + partitionedJoin ='false', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + asyncPoolSize ='3', + parallelism ='1' + ); +insert +into + MyResult + select + m.id, + s.name + from + MyTable m + join + sideTable s + on m.id=s.id; +``` \ No newline at end of file diff --git a/docs/plugin/oracleSide.md b/docs/plugin/oracleSide.md index 47b8e36c7..1cb4e4cce 100644 --- a/docs/plugin/oracleSide.md +++ b/docs/plugin/oracleSide.md @@ -1,5 +1,13 @@ ## 1.格式: + + 通过建表语句中的` PERIOD FOR SYSTEM_TIME`将表标识为维表,其中`PRIMARY KEY(keyInfo)`中的keyInfo,表示用来和源表进行关联的字段, + 维表JOIN的条件必须与`keyInfo`字段一致。 + + 此外, 针对oracle char类型自动补齐的特性,我们允许为char指定长度,确保维表查询时能够匹配数据。 + + + 注意:Oracle维表使用的字段大小写,需要和Oracle中定义的保持一致。 ``` CREATE TABLE tableName( colName cloType, @@ -22,43 +30,35 @@ # 2.支持版本 10g 11g -## 3.表结构定义 - - |参数名称|含义| - |----|---| - | tableName | oracle表名称| - | colName | 列名称| - | colType | 列类型 [colType支持的类型](docs/colType.md)| - | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| - | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| - -## 4.参数 - - |参数名称|含义|是否必填|默认值| - |----|---|---|----| - | type | 表明维表的类型 oracle |是|| - | url | 连接oracle数据库 jdbcUrl |是|| - | userName | oracle连接用户名 |是|| - | password | oracle连接密码|是|| - | tableName | oracle表名称|是|| - | schema | oracle 的schema|否|当前登录用户| - | cache | 维表缓存策略(NONE/LRU)|否|NONE| - | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| - - ---------- - > 缓存策略 - * NONE: 不做内存缓存 - * LRU: - * cacheSize: 缓存的条目数量 - * cacheTTLMs:缓存的过期时间(ms) - * cacheMode: (unordered|ordered)异步加载是有序还是无序,默认有序。 - * asyncCapacity:异步请求容量,默认1000 - * asyncTimeout:异步请求超时时间,默认10000毫秒 - -## 5.样例 +## 3. 表结构定义 + [关系型数据库维表参数](docs/rdbSideParams.md) + +## 4.样例 + +### ALL全量维表定义 +``` +CREATE TABLE sideTable( + ID char(20), // oracle定义了char(20) + NAME varchar, + PRIMARY KEY (ID), + PERIOD FOR SYSTEM_TIME + )WITH( + type='oracle', + url = 'jdbc:oracle:thin:@172.16.8.178:1521:xe', + userName = 'system', + password = 'oracle', + tableName = 'SIDETEST1', + schema = 'dtstack', + cache = 'ALL', + cacheTTLMs ='60000' + ); +``` + +### LRU异步维表定义 + ``` create table sideTable( - channel varchar, + channel char, xccount int, PRIMARY KEY(channel), PERIOD FOR SYSTEM_TIME @@ -79,7 +79,70 @@ create table sideTable( schema = 'MQTEST' ); +``` + +### Oracle异步维表关联 + +```aidl +CREATE TABLE MyTable( + id varchar, + name varchar + --ts timestamp, + --tsDate Date + )WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest01', + timezone='Asia/Shanghai', + topicIsPattern ='false', + parallelism ='1' + ); + + +CREATE TABLE sideTable( + ID char(20), + NAME varchar, + PRIMARY KEY (ID), + PERIOD FOR SYSTEM_TIME + )WITH( + type='oracle', + url = 'jdbc:oracle:thin:@172.16.8.178:1521:xe', + userName = 'system', + password = 'oracle', + tableName = 'SIDETEST1', + --schema = 'dtstack', + cache = 'LRU', + asyncPoolSize ='3' + ); + + +CREATE TABLE MyResult( + NAME varchar, + ID char(20), + PRIMARY KEY (ID) + )WITH( + --type ='console', + type='oracle', + url = 'jdbc:oracle:thin:@172.16.8.178:1521:xe', + userName = 'system', + password = 'oracle', + tableName = 'SINK_TEST', + batchSize='1' + ); + + +INSERT INTO MyResult +SELECT + s.ID as ID, + m.name as NAME +FROM MyTable m +LEFT JOIN + sideTable s +ON + m.id=s.ID ``` diff --git a/docs/plugin/rdbSideParams.md b/docs/plugin/rdbSideParams.md index 26666bd5b..34d02978e 100644 --- a/docs/plugin/rdbSideParams.md +++ b/docs/plugin/rdbSideParams.md @@ -14,6 +14,7 @@ | schema| 表空间|否|| | cache | 维表缓存策略(NONE/LRU/ALL)|否|LRU| | partitionedJoin | 是否在維表join之前先根据设定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| +| parallelism | 处理后的数据流并行度|否|| ### 缓存策略 From b264114417d7044274972d3b3309abc612a02d30 Mon Sep 17 00:00:00 2001 From: maqi Date: Thu, 16 Apr 2020 14:30:45 +0800 Subject: [PATCH 046/109] link rdbSideParams.md --- docs/plugin/mysqlSide.md | 2 +- docs/plugin/oracleSide.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/plugin/mysqlSide.md b/docs/plugin/mysqlSide.md index 96e997e9e..f28f09518 100644 --- a/docs/plugin/mysqlSide.md +++ b/docs/plugin/mysqlSide.md @@ -29,7 +29,7 @@ ## 3.表结构定义 - [关系型数据库维表参数](docs/rdbSideParams.md) + [关系型数据库维表参数](docs/plugin/rdbSideParams.md) ## 4.样例 diff --git a/docs/plugin/oracleSide.md b/docs/plugin/oracleSide.md index 1cb4e4cce..ee7e03331 100644 --- a/docs/plugin/oracleSide.md +++ b/docs/plugin/oracleSide.md @@ -31,7 +31,7 @@ 10g 11g ## 3. 表结构定义 - [关系型数据库维表参数](docs/rdbSideParams.md) + [关系型数据库维表参数](docs/plugin/rdbSideParams.md) ## 4.样例 From 1e2db49880757b2820544e0bf9a91081652fc95a Mon Sep 17 00:00:00 2001 From: tiezhu Date: Thu, 16 Apr 2020 15:52:06 +0800 Subject: [PATCH 047/109] =?UTF-8?q?update=20redis=E7=BB=B4=E8=A1=A8?= =?UTF-8?q?=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/config.md | 2 +- docs/plugin/redisSide.md | 46 +++++++++++++++++++++++++--------------- 2 files changed, 30 insertions(+), 18 deletions(-) diff --git a/docs/config.md b/docs/config.md index 3fb6ac723..9aa8df994 100644 --- a/docs/config.md +++ b/docs/config.md @@ -73,7 +73,7 @@ sh submit.sh -key1 val1 -key2 val2 * savePointPath:任务恢复点的路径(默认无) * allowNonRestoredState:指示保存点是否允许非还原状态的标志(默认false) * logLevel: 日志级别动态配置(默认info) - * [prometheus 相关参数](docs/prometheus.md) per_job可指定metric写入到外部监控组件,以prometheus pushgateway举例 + * [prometheus 相关参数](./prometheus.md) per_job可指定metric写入到外部监控组件,以prometheus pushgateway举例 * **flinkconf** diff --git a/docs/plugin/redisSide.md b/docs/plugin/redisSide.md index f9b5e41f9..6aa335fea 100644 --- a/docs/plugin/redisSide.md +++ b/docs/plugin/redisSide.md @@ -22,29 +22,31 @@ redis5.0 ## 3.表结构定义 +注意:redis中没有表和schema的概念,参数中tableName是指符合命名规则的key,具体规则请看[缓存redis的存储命名规则] |参数名称|含义| |----|---| -| tableName | 注册到flink的表名称(可选填;不填默认和redis对应的表名称相同)| -| colName | 列名称,维表列名格式 表名:主键名:主键值:列名]| +| tableName | 注册到flinkStreamSql的表名称(可选填;不填默认和redis对应的"表"名称相同)| +| colName | 列名称,对应redis对应"表"的field| | colType | 列类型,当前只支持varchar| -| PRIMARY KEY |主键,多个字段做为联合主键时以逗号分隔 -| PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| +| PRIMARY KEY |主键,多个字段做为联合主键时以逗号分隔| +| PERIOD FOR SYSTEM_TIME | 关键字,表明该定义的表为维表信息| ## 4.参数 +参数详细说明请看[参数详细说明]() + |参数名称|含义|是否必填|默认值| |----|---|---|----| | type | 表明维表的类型[hbase|mysql|redis]|是|| | url | redis 的地址;格式ip:port[,ip:port]|是|| -| password | redis 的密码 |是|| +| password | redis 的密码 |否|空| | redisType | redis模式(1 单机,2 哨兵, 3 集群)| 是 | | masterName | 主节点名称(哨兵模式下为必填项) | 否 | -| database | reids 的数据库地址|否|| -| tableName | redis 的表名称|是|| +| database | reids 的数据库地址|否|0| +| tableName | redis 的“表”名称|是|| | cache | 维表缓存策略(NONE/LRU/ALL)|否|NONE| | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| - -------------- > 缓存策略 * NONE: 不做内存缓存 @@ -55,27 +57,37 @@ ## 5.样例 ``` -create table sideTable( - channel varchar, - xccount varchar, - PRIMARY KEY(channel), +CREATE TABLE MyRedis( + id varchar, + message varchar, + PRIMARY KEY(id), PERIOD FOR SYSTEM_TIME - )WITH( +)WITH( type='redis', url='172.16.10.79:6379', password='abc123', database='0', redisType = '1', - tableName='sidetest', + tableName = 'sideTable', cache = 'LRU', cacheTTLMs='10000' - ); +); ``` -## 6.缓存redis的存储结构规则 -``` +## 6.redis的存储命名规则 + redis使用散列类型 hash 数据结构,key=tableName_primaryKey1_primaryKey2,value={column1=value1, column2=value2} 如果以班级class表为例,id和name作为联合主键,那么redis的结构为 + +在样例中,tableName为sideTable,主键为id,column为id,message,所以对应的redis数据插入语句为 + +数据在redis中对应的数据存储情况为: +``` +192.168.80.105:6379> hgetall sideTable_5 +1) "id" +2) "5" +3) "message" +4) "redis" ``` From bbf37e1df45ee7f589522177dc5f3eabdfe5dbc5 Mon Sep 17 00:00:00 2001 From: maqi Date: Thu, 16 Apr 2020 16:05:08 +0800 Subject: [PATCH 048/109] oracle mysql side doc --- docs/plugin/mysqlSide.md | 14 +++++++++++++- docs/plugin/oracleSide.md | 13 ++++++++++++- docs/plugin/{rdbSideParams.md => sideParams.md} | 11 ++++------- 3 files changed, 29 insertions(+), 9 deletions(-) rename docs/plugin/{rdbSideParams.md => sideParams.md} (76%) diff --git a/docs/plugin/mysqlSide.md b/docs/plugin/mysqlSide.md index f28f09518..834183ed2 100644 --- a/docs/plugin/mysqlSide.md +++ b/docs/plugin/mysqlSide.md @@ -29,7 +29,19 @@ ## 3.表结构定义 - [关系型数据库维表参数](docs/plugin/rdbSideParams.md) + [维表参数信息](docs/plugin/sideParams.md) + + mysql独有的参数配置: + + +|参数名称|含义|是否必填|默认值| +|----|---|---|----| +| type | 维表类型, mysql |是|| +| url | 连接数据库 jdbcUrl |是|| +| userName | 连接用户名 |是|| +| password | 连接密码|是|| + + ## 4.样例 diff --git a/docs/plugin/oracleSide.md b/docs/plugin/oracleSide.md index ee7e03331..b681536f8 100644 --- a/docs/plugin/oracleSide.md +++ b/docs/plugin/oracleSide.md @@ -31,7 +31,18 @@ 10g 11g ## 3. 表结构定义 - [关系型数据库维表参数](docs/plugin/rdbSideParams.md) + [维表参数](docs/plugin/sideParams.md) + + + oracle独有的参数配置: + + |参数名称|含义|是否必填|默认值| + |----|---|---|----| + | type | 维表类型, oracle |是|| + | url | 连接数据库 jdbcUrl |是|| + | userName | 连接用户名 |是|| + | password | 连接密码|是|| + | schema| 表空间|否|| ## 4.样例 diff --git a/docs/plugin/rdbSideParams.md b/docs/plugin/sideParams.md similarity index 76% rename from docs/plugin/rdbSideParams.md rename to docs/plugin/sideParams.md index 34d02978e..6059dcab8 100644 --- a/docs/plugin/rdbSideParams.md +++ b/docs/plugin/sideParams.md @@ -1,17 +1,14 @@ -## 关系型数据库维表参数 +## 维表参数 -适用于`MYSQL`,`ORACLE`,`SQLSERVER`,`POSTGRESQL`,`DB2`,`POLARDB`,`CLICKHOUSE`,`IMPALA`维表插件 ### 维表参数 +维表需要的基本属性,每个插件还需要提供连接所需的基本信息。 + |参数名称|含义|是否必填|默认值| |----|---|---|----| | type | 维表类型, 例如:mysql |是|| -| url | 连接数据库 jdbcUrl |是|| -| userName | 连接用户名 |是|| -| password | 连接密码|是|| | tableName| 表名称|是|| -| schema| 表空间|否|| | cache | 维表缓存策略(NONE/LRU/ALL)|否|LRU| | partitionedJoin | 是否在維表join之前先根据设定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| | parallelism | 处理后的数据流并行度|否|| @@ -37,6 +34,6 @@ | cacheMode | 异步请求处理有序还是无序,可选:ordered,unordered |ordered| | asyncCapacity | 异步线程容量 |100| | asyncTimeout | 异步处理超时时间 |10000,单位毫秒| -| asyncPoolSize | 异步查询DB最大线程池,上限20 |min(20,Runtime.getRuntime().availableProcessors() * 2)| +| asyncPoolSize | 异步查询DB最大线程池,上限20。适用于MYSQL,ORACLE,SQLSERVER,POSTGRESQL,DB2,POLARDB,CLICKHOUSE,IMPALA维表插件|min(20,Runtime.getRuntime().availableProcessors() * 2)| From dd7987e420d5f4d52008491acd8edfb3c63ab4e5 Mon Sep 17 00:00:00 2001 From: tiezhu Date: Thu, 16 Apr 2020 16:21:14 +0800 Subject: [PATCH 049/109] =?UTF-8?q?redisSide=20=E6=B7=BB=E5=8A=A0=E7=A4=BA?= =?UTF-8?q?=E4=BE=8B?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/redisSide.md | 99 ++++++++++++++++++++++++++++++++++++---- 1 file changed, 89 insertions(+), 10 deletions(-) diff --git a/docs/plugin/redisSide.md b/docs/plugin/redisSide.md index 6aa335fea..3f4483e65 100644 --- a/docs/plugin/redisSide.md +++ b/docs/plugin/redisSide.md @@ -1,5 +1,5 @@ -## 1.格式: +## 1.格式 ``` CREATE TABLE tableName( colName cloType, @@ -15,7 +15,9 @@ redisType = '1', cache ='LRU', cacheSize ='10000', - cacheTTLMs ='60000' + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' ); ``` ## 2.支持版本 @@ -34,7 +36,7 @@ ## 4.参数 -参数详细说明请看[参数详细说明]() +参数详细说明请看[参数详细说明](./sideParams.md) |参数名称|含义|是否必填|默认值| |----|---|---|----| @@ -48,14 +50,9 @@ | cache | 维表缓存策略(NONE/LRU/ALL)|否|NONE| | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| -------------- -> 缓存策略 - * NONE: 不做内存缓存 - * LRU: - * cacheSize: 缓存的条目数量 - * cacheTTLMs:缓存的过期时间(ms) - * ALL: 缓存全量表数据 ## 5.样例 +### LRU维表示例 ``` CREATE TABLE MyRedis( id varchar, @@ -70,10 +67,92 @@ CREATE TABLE MyRedis( redisType = '1', tableName = 'sideTable', cache = 'LRU', - cacheTTLMs='10000' + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' +); +``` +### ALL维表示例 +``` +CREATE TABLE MyRedis( + id varchar, + message varchar, + PRIMARY KEY(id), + PERIOD FOR SYSTEM_TIME +)WITH( + type='redis', + url='172.16.10.79:6379', + password='abc123', + database='0', + redisType = '1', + tableName = 'sideTable', + cache = 'ALL', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' +); +``` +### redis异步维表关联完整案例 +``` +CREATE TABLE MyTable( + id varchar, + name varchar, + address varchar +)WITH( + type = 'kafka10', + bootstrapServers = '172.16.101.224:9092', + zookeeperQuorm = '172.16.100.188:2181/kafka', + offsetReset = 'latest', + topic = 'tiezhu_test_in2', + groupId = 'flink_sql', + timezone = 'Asia/Shanghai', + topicIsPattern = 'false', + parallelism = '1' ); +CREATE TABLE MyRedis( + id varchar, + message varchar, + PRIMARY KEY(id), + PERIOD FOR SYSTEM_TIME +)WITH( + type='redis', + url='172.16.10.79:6379', + password='abc123', + database='0', + redisType = '1', + tableName = 'sideTable', + cache = 'LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' +); + +CREATE TABLE MyResult( + id varchar, + name varchar, + address varchar, + message varchar +)WITH( + type = 'console' +); + +insert into MyResult + select + t1.id AS id, + t1.name AS name, + t1.address AS address, + t2.message AS message + from + ( + select id, name, address + from MyTable + ) t1 join MyRedis t2 + on t1.id = t2.id; ``` + ## 6.redis的存储命名规则 redis使用散列类型 hash 数据结构,key=tableName_primaryKey1_primaryKey2,value={column1=value1, column2=value2} From 38e34dc546ee421530915d5c2b387304ac6b8fdf Mon Sep 17 00:00:00 2001 From: tiezhu Date: Fri, 17 Apr 2020 11:06:19 +0800 Subject: [PATCH 050/109] =?UTF-8?q?update=20hbase=E7=BB=B4=E8=A1=A8?= =?UTF-8?q?=E9=83=A8=E5=88=86=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/hbaseSide.md | 140 ++++++++++++++++++++++++++++++++------- 1 file changed, 117 insertions(+), 23 deletions(-) diff --git a/docs/plugin/hbaseSide.md b/docs/plugin/hbaseSide.md index 0c4e545f9..3d08084d0 100644 --- a/docs/plugin/hbaseSide.md +++ b/docs/plugin/hbaseSide.md @@ -1,5 +1,5 @@ -## 1.格式: +## 1.格式 ``` CREATE TABLE tableName( columnFamily:columnName type as alias, @@ -31,7 +31,9 @@ | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| | PRIMARY KEY(keyInfo) | 维表主键定义;hbase 维表rowkey的构造方式;可选择的构造包括 md5(alias + alias), '常量',也包括上述方式的自由组合 | -## 3.参数 +## 4.参数 + +参数详细说明请看[参数详细说明](./sideParams.md) |参数名称|含义|是否必填|默认值| |----|---|---|----| @@ -43,31 +45,123 @@ | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| -------------- -> 缓存策略 - * NONE: 不做内存缓存 - * LRU: - * cacheSize: 缓存的条目数量 - * cacheTTLMs:缓存的过期时间(ms) -## 4.样例 +## 5.样例 +### LRU维表示例 +``` +CREATE TABLE sideTable ( + wtz:message varchar as message, + wtz:info varchar as info , + PRIMARY KEY (rowkey), + PERIOD FOR SYSTEM_TIME +) WITH ( + type = 'hbase', + zookeeperQuorum = '192.168.80.105:2181,192.168.80.106:2181,192.168.80.107:2181', + zookeeperParent = '/hbase', + tableName = 'testFlinkStreamSql', + parallelism = '1', + cache = 'LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' +); +``` + +### ALL维表示例 +``` +CREATE TABLE sideTable ( + wtz:message varchar as message, + wtz:info varchar as info , + PRIMARY KEY (rowkey), + PERIOD FOR SYSTEM_TIME +) WITH ( + type = 'hbase', + zookeeperQuorum = '192.168.80.105:2181,192.168.80.106:2181,192.168.80.107:2181', + zookeeperParent = '/hbase', + tableName = 'testFlinkStreamSql', + parallelism = '1', + cache = 'ALL', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' +); ``` -CREATE TABLE sideTable( - cf:name varchar as name, - cf:info int as info, - PRIMARY KEY(md5(name) + 'test'), - PERIOD FOR SYSTEM_TIME - )WITH( - type ='hbase', - zookeeperQuorum ='rdos1:2181', - zookeeperParent ='/hbase', - tableName ='workerinfo', - cache ='LRU', - cacheSize ='10000', - cacheTTLMs ='60000', - parallelism ='1', - partitionedJoin='true' + +### hbase异步维表关联完整案例 +``` +CREATE TABLE MyTable( + id varchar, + name varchar, + address varchar +)WITH( + type = 'kafka10', + bootstrapServers = '172.16.101.224:9092', + zookeeperQuorm = '172.16.100.188:2181/kafka', + offsetReset = 'latest', + topic = 'tiezhu_test_in', + groupId = 'flink_sql', + timezone = 'Asia/Shanghai', + topicIsPattern = 'false', + parallelism = '1' +); + +CREATE TABLE MyResult( + id varchar, + name varchar, + address varchar, + message varchar, + info varchar +)WITH( + type = 'console' ); + + CREATE TABLE sideTable ( + wtz:message varchar as message, + wtz:info varchar as info , + PRIMARY KEY (rowkey), + PERIOD FOR SYSTEM_TIME +) WITH ( + type = 'hbase', + zookeeperQuorum = '192.168.80.105:2181,192.168.80.106:2181,192.168.80.107:2181', + zookeeperParent = '/hbase', + tableName = 'testFlinkStreamSql', + parallelism = '1', + cache = 'LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' +); +insert +into + MyResult + select + a.name, + a.id, + a.address, + b.message, + b.info + from + MyTable a + left join + sideTable b + on a.id=b.rowkey; ``` +## 6.hbase中数据存储形式 + +在hbase中,数据是以列簇的形式存储,其中rowKey作为主键,按字典排序。 +在样例中,wtz为列族名,message, info为列名,数据在hbase中的存储情况为: +``` +hbase(main):002:0> scan 'testFlinkStreamSql' +ROW COLUMN+CELL + 0 column=wtz:info, timestamp=1587089266719, value=hadoop + 0 column=wtz:message, timestamp=1587089245780, value=hbase + 1 column=wtz:info, timestamp=1587088818432, value=flink + 1 column=wtz:message, timestamp=1587088796633, value=dtstack + 2 column=wtz:info, timestamp=1587088858564, value=sql + 2 column=wtz:message, timestamp=1587088840507, value=stream +``` From 774da5c35d3abc07abf90aa9494d44531013b98b Mon Sep 17 00:00:00 2001 From: tiezhu Date: Fri, 17 Apr 2020 11:07:57 +0800 Subject: [PATCH 051/109] =?UTF-8?q?=E4=BF=AE=E6=94=B9hbase=E7=BB=B4?= =?UTF-8?q?=E8=A1=A8=E6=8E=92=E7=89=88?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/hbaseSide.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/plugin/hbaseSide.md b/docs/plugin/hbaseSide.md index 3d08084d0..2ab50f914 100644 --- a/docs/plugin/hbaseSide.md +++ b/docs/plugin/hbaseSide.md @@ -107,13 +107,13 @@ CREATE TABLE MyTable( ); CREATE TABLE MyResult( - id varchar, - name varchar, - address varchar, - message varchar, - info varchar + id varchar, + name varchar, + address varchar, + message varchar, + info varchar )WITH( - type = 'console' + type = 'console' ); CREATE TABLE sideTable ( From 4bbe75178dc71115df4236078078b9d7526199f8 Mon Sep 17 00:00:00 2001 From: tiezhu Date: Fri, 17 Apr 2020 11:36:25 +0800 Subject: [PATCH 052/109] =?UTF-8?q?update=20hbase,=20redis=E7=BB=B4?= =?UTF-8?q?=E8=A1=A8=E6=8E=92=E7=89=88?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/hbaseSide.md | 33 +++++++++-------- docs/plugin/redisSide.md | 77 ++++++++++++++++++++-------------------- 2 files changed, 57 insertions(+), 53 deletions(-) diff --git a/docs/plugin/hbaseSide.md b/docs/plugin/hbaseSide.md index 2ab50f914..cc5fa319c 100644 --- a/docs/plugin/hbaseSide.md +++ b/docs/plugin/hbaseSide.md @@ -88,6 +88,24 @@ CREATE TABLE sideTable ( ); ``` +## 6.hbase完整样例 + +### 数据说明 + +在hbase中,数据是以列簇的形式存储,其中rowKey作为主键,按字典排序。 + +在样例中,wtz为列族名,message, info为列名,数据在hbase中的存储情况为: +``` +hbase(main):002:0> scan 'testFlinkStreamSql' +ROW COLUMN+CELL + 0 column=wtz:info, timestamp=1587089266719, value=hadoop + 0 column=wtz:message, timestamp=1587089245780, value=hbase + 1 column=wtz:info, timestamp=1587088818432, value=flink + 1 column=wtz:message, timestamp=1587088796633, value=dtstack + 2 column=wtz:info, timestamp=1587088858564, value=sql + 2 column=wtz:message, timestamp=1587088840507, value=stream +``` + ### hbase异步维表关联完整案例 ``` CREATE TABLE MyTable( @@ -150,18 +168,3 @@ into on a.id=b.rowkey; ``` -## 6.hbase中数据存储形式 - -在hbase中,数据是以列簇的形式存储,其中rowKey作为主键,按字典排序。 - -在样例中,wtz为列族名,message, info为列名,数据在hbase中的存储情况为: -``` -hbase(main):002:0> scan 'testFlinkStreamSql' -ROW COLUMN+CELL - 0 column=wtz:info, timestamp=1587089266719, value=hadoop - 0 column=wtz:message, timestamp=1587089245780, value=hbase - 1 column=wtz:info, timestamp=1587088818432, value=flink - 1 column=wtz:message, timestamp=1587088796633, value=dtstack - 2 column=wtz:info, timestamp=1587088858564, value=sql - 2 column=wtz:message, timestamp=1587088840507, value=stream -``` diff --git a/docs/plugin/redisSide.md b/docs/plugin/redisSide.md index 3f4483e65..c63c542cd 100644 --- a/docs/plugin/redisSide.md +++ b/docs/plugin/redisSide.md @@ -93,6 +93,25 @@ CREATE TABLE MyRedis( partitionedJoin='false' ); ``` + +## 6.redis完整样例 + +### redis数据说明 + +redis使用散列类型 hash 数据结构,key=tableName_primaryKey1_primaryKey2,value={column1=value1, column2=value2} +如果以班级class表为例,id和name作为联合主键,那么redis的结构为 + +在样例中,tableName为sideTable,主键为id,column为id,message,所以对应的redis数据插入语句为 + +数据在redis中对应的数据存储情况为: +``` +192.168.80.105:6379> hgetall sideTable_5 +1) "id" +2) "5" +3) "message" +4) "redis" +``` + ### redis异步维表关联完整案例 ``` CREATE TABLE MyTable( @@ -112,12 +131,12 @@ CREATE TABLE MyTable( ); CREATE TABLE MyRedis( - id varchar, - message varchar, - PRIMARY KEY(id), - PERIOD FOR SYSTEM_TIME + id varchar, + message varchar, + PRIMARY KEY(id), + PERIOD FOR SYSTEM_TIME )WITH( - type='redis', + type='redis', url='172.16.10.79:6379', password='abc123', database='0', @@ -131,42 +150,24 @@ CREATE TABLE MyRedis( ); CREATE TABLE MyResult( - id varchar, - name varchar, - address varchar, - message varchar + id varchar, + name varchar, + address varchar, + message varchar )WITH( - type = 'console' + type = 'console' ); insert into MyResult - select - t1.id AS id, - t1.name AS name, - t1.address AS address, - t2.message AS message - from - ( - select id, name, address - from MyTable - ) t1 join MyRedis t2 - on t1.id = t2.id; -``` - -## 6.redis的存储命名规则 - -redis使用散列类型 hash 数据结构,key=tableName_primaryKey1_primaryKey2,value={column1=value1, column2=value2} -如果以班级class表为例,id和name作为联合主键,那么redis的结构为 - -在样例中,tableName为sideTable,主键为id,column为id,message,所以对应的redis数据插入语句为 - -数据在redis中对应的数据存储情况为: + select + t1.id AS id, + t1.name AS name, + t1.address AS address, + t2.message AS message + from( + select id, name, address + from MyTable + ) t1 join MyRedis t2 + on t1.id = t2.id; ``` -192.168.80.105:6379> hgetall sideTable_5 -1) "id" -2) "5" -3) "message" -4) "redis" -``` - From 38b9b388b6b517631080d1f17135a175528bfe54 Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 21 Apr 2020 11:21:46 +0800 Subject: [PATCH 053/109] =?UTF-8?q?=E7=BB=B4=E8=A1=A8=E8=A1=A8=E5=90=8D?= =?UTF-8?q?=E6=9B=BF=E6=8D=A2=E9=80=BB=E8=BE=91?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/util/FieldReplaceUtil.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java index f92643d5f..69bdc97b2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java @@ -181,12 +181,12 @@ public static SqlIdentifier createNewIdentify(SqlIdentifier sqlIdentifier, return sqlIdentifier; } + sqlIdentifier = sqlIdentifier.setName(0, newTbName); + String mappingFieldName = mappingField.get(fieldName); if(mappingFieldName == null){ return sqlIdentifier; } - - sqlIdentifier = sqlIdentifier.setName(0, newTbName); sqlIdentifier = sqlIdentifier.setName(1, mappingFieldName); return sqlIdentifier; } From ca8d08de02594d5abfe8efa6cc687219e0a274ad Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 21 Apr 2020 12:53:24 +0800 Subject: [PATCH 054/109] =?UTF-8?q?=E4=BF=AE=E5=A4=8Dcase=20when=20?= =?UTF-8?q?=E4=B8=AD=E6=B2=A1=E6=9C=89=E8=A7=A3=E6=9E=90else=E5=AD=97?= =?UTF-8?q?=E6=AE=B5?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/side/JoinNodeDealer.java | 2 +- .../java/com/dtstack/flink/sql/util/FieldReplaceUtil.java | 4 ++++ .../main/java/com/dtstack/flink/sql/util/TableUtils.java | 6 ++++++ 3 files changed, 11 insertions(+), 1 deletion(-) 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 f072e2591..544e8ebb0 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 @@ -233,7 +233,6 @@ public Set extractField(SqlNode sqlNode, Set fieldFromJoinCondition = extractSelectFieldFromJoinCondition(joinFieldSet, fromTableNameSet, tableRef); Set extractGroupByField = extractFieldFromGroupByList(parentGroupByList, fromTableNameSet, tableRef); - extractSelectField.addAll(extractCondition); extractSelectField.addAll(fieldFromJoinCondition); extractSelectField.addAll(extractGroupByField); @@ -575,6 +574,7 @@ private void extractSelectField(SqlNode selectNode, || selectNode.getKind() == TIMESTAMP_ADD || selectNode.getKind() == TIMESTAMP_DIFF || selectNode.getKind() == LIKE + || selectNode.getKind() == COALESCE ){ SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java index 69bdc97b2..775e0a985 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FieldReplaceUtil.java @@ -160,6 +160,10 @@ private static SqlNode replaceNodeInfo(SqlNode groupNode, SqlNode replaceNode = replaceSelectFieldName(sqlNode, oldTbName, newTbName, mappingField); sqlCase.getThenOperands().set(i,replaceNode); } + if(sqlCase.getElseOperand() != null){ + SqlNode replaceNode = replaceSelectFieldName(sqlCase.getElseOperand(), oldTbName, newTbName, mappingField); + sqlCase.setOperand(3, replaceNode); + } return sqlCase; } else { return groupNode; 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 579beaf63..4a4973901 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 @@ -151,6 +151,7 @@ public static void extractSelectFieldToFieldInfo(SqlNode fieldNode, String fromN || fieldNode.getKind() == TIMESTAMP_ADD || fieldNode.getKind() == TIMESTAMP_DIFF || fieldNode.getKind() == LIKE + || fieldNode.getKind() == COALESCE ) { SqlBasicCall sqlBasicCall = (SqlBasicCall) fieldNode; for (int i = 0; i < sqlBasicCall.getOperands().length; i++) { @@ -363,6 +364,7 @@ public static void replaceSelectFieldTable(SqlNode selectNode, || selectNode.getKind() == TIMESTAMP_ADD || selectNode.getKind() == TIMESTAMP_DIFF || selectNode.getKind() == LIKE + || selectNode.getKind() == COALESCE ){ SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; @@ -548,6 +550,7 @@ private static void replaceConditionNode(SqlNode selectNode, String oldTbName, S || selectNode.getKind() == TIMESTAMP_ADD || selectNode.getKind() == TIMESTAMP_DIFF || selectNode.getKind() == LIKE + || selectNode.getKind() == COALESCE ){ SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; @@ -573,6 +576,7 @@ private static void replaceConditionNode(SqlNode selectNode, String oldTbName, S sqlCase.getWhenOperands().getList().forEach(sqlNode -> replaceConditionNode(sqlNode, oldTbName, newTbName, fieldReplaceRef)); sqlCase.getThenOperands().getList().forEach(sqlNode -> replaceConditionNode(sqlNode, oldTbName, newTbName, fieldReplaceRef)); + replaceConditionNode(sqlCase.getElseOperand(), oldTbName, newTbName, fieldReplaceRef); } else { throw new RuntimeException(String.format("not support node kind of %s to replace name now.", selectNode.getKind())); } @@ -620,6 +624,7 @@ public static void getConditionRefTable(SqlNode selectNode, Set fieldInf || selectNode.getKind() == TIMESTAMP_ADD || selectNode.getKind() == TIMESTAMP_DIFF || selectNode.getKind() == LIKE + || selectNode.getKind() == COALESCE ) { SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; @@ -645,6 +650,7 @@ public static void getConditionRefTable(SqlNode selectNode, Set fieldInf sqlCase.getWhenOperands().getList().forEach(sqlNode -> getConditionRefTable(sqlNode, fieldInfos)); sqlCase.getThenOperands().getList().forEach(sqlNode -> getConditionRefTable(sqlNode, fieldInfos)); + getConditionRefTable(sqlCase.getElseOperand(), fieldInfos); } else { throw new RuntimeException(String.format("not support node kind of %s to replace name now.", selectNode.getKind())); } From b231974e816aafa5d8d685921e1404b7a1211249 Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 21 Apr 2020 16:06:35 +0800 Subject: [PATCH 055/109] fix --- docs/plugin/mysqlSink.md | 73 +++++++++++++++++++++++++++++++++------- docs/plugin/redisSink.md | 5 +++ 2 files changed, 66 insertions(+), 12 deletions(-) diff --git a/docs/plugin/mysqlSink.md b/docs/plugin/mysqlSink.md index 300086e21..8dabea047 100644 --- a/docs/plugin/mysqlSink.md +++ b/docs/plugin/mysqlSink.md @@ -22,7 +22,7 @@ CREATE TABLE tableName( |参数名称|含义| |----|---| -| tableName| mysql表名称| +| tableName| 结果表名称| | colName | 列名称| | colType | 列类型 [colType支持的类型](docs/colType.md)| @@ -33,21 +33,70 @@ CREATE TABLE tableName( |type |表名 输出表类型[mysq|hbase|elasticsearch]|是|| |url | 连接mysql数据库 jdbcUrl |是|| |userName | mysql连接用户名 |是|| -| password | mysql连接密码|是|| -| tableName | mysql表名称|是|| -| parallelism | 并行度设置|否|1| - -## 5.样例: +|password | mysql连接密码|是|| +|tableName | mysql表名称|是|| +|parallelism | 并行度设置|否|1| +|batchNum | flush的大小|否|100| +|batchWaitInterval | flush的时间间隔,单位ms|否|1000| +|allReplace| true:新值替换旧值|否|false| +|updateMode| APPEND:不回撤数据,只下发增量数据,UPSERT:先删除回撤数据,然后更新|否|| + +## 5.完整样例: ``` +CREATE TABLE MyTable( + id int, + channel varchar, + pv varchar, + xctime varchar, + name varchar + )WITH( + type ='kafka10', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='es_test', + timezone='Asia/Shanghai', + topicIsPattern ='false', + parallelism ='1' + ); + CREATE TABLE MyResult( - channel VARCHAR, - pv VARCHAR + pv VARCHAR, + channel VARCHAR )WITH( type ='mysql', - url ='jdbc:mysql://172.16.8.104:3306/test?charset=utf8', + url ='jdbc:mysql://172.16.10.134:3306/test', userName ='dtstack', password ='abc123', - tableName ='pv2', + tableName ='myresult', parallelism ='1' - ) - ``` \ No newline at end of file + ); + + +insert +into + MyResult + select + channel, + pv + from + MyTable + ``` + +## 6.结果表数据示例: +``` +mysql> desc myresult; ++---------+--------------+------+-----+---------+-------+ +| Field | Type | Null | Key | Default | Extra | ++---------+--------------+------+-----+---------+-------+ +| channel | varchar(255) | YES | | NULL | | +| pv | varchar(11) | YES | | NULL | | ++---------+--------------+------+-----+---------+-------+ +2 rows in set (0.00 sec) + +mysql> select * from myresult limit 1; ++---------+------+ +| channel | pv | ++---------+------+ +| aa | mq6 | +``` \ No newline at end of file diff --git a/docs/plugin/redisSink.md b/docs/plugin/redisSink.md index 6a754e5c6..b43cb7377 100644 --- a/docs/plugin/redisSink.md +++ b/docs/plugin/redisSink.md @@ -41,6 +41,11 @@ redis5.0 | database | reids 的数据库地址|否|| | tableName | redis 的表名称|是|| | parallelism | 并行度设置|否|1| +|timeout| 连接超时时间|否|10000| +|maxTotal|最大连接数|否|8| +|maxIdle|最大空闲连接数|否|8| +|minIdle|最小空闲连接数|否||0| +|masterName| 哨兵模式下的masterName|否|| ## 5.样例: From ccbc0a1207e4554021d214e9e40caeb156898e92 Mon Sep 17 00:00:00 2001 From: tiezhu Date: Wed, 22 Apr 2020 16:04:18 +0800 Subject: [PATCH 056/109] =?UTF-8?q?update=20hbaseSide=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/hbaseSide.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/plugin/hbaseSide.md b/docs/plugin/hbaseSide.md index cc5fa319c..25af638d6 100644 --- a/docs/plugin/hbaseSide.md +++ b/docs/plugin/hbaseSide.md @@ -29,7 +29,7 @@ | columnFamily:columnName | hbase中的列族名称和列名称 | | alias | hbase 中的列对应到flink中注册的列名称 | | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| -| PRIMARY KEY(keyInfo) | 维表主键定义;hbase 维表rowkey的构造方式;可选择的构造包括 md5(alias + alias), '常量',也包括上述方式的自由组合 | +| PRIMARY KEY(keyInfo) | 维表主键定义;hbase 维表rowkey的构造方式:可选择的构造包括 md5(alias + alias), '常量',也包括上述方式的自由组合 | ## 4.参数 @@ -105,7 +105,8 @@ ROW COLUMN+CELL 2 column=wtz:info, timestamp=1587088858564, value=sql 2 column=wtz:message, timestamp=1587088840507, value=stream ``` - +在hbase中,rowKey是一个二进制码流,可以为任意字符串,flinkStreamSql读取rowKey并通过rowKey唯一确定数据,对rowKey没有任何限制,对rowKey可选择的构造包括 md5(alias + alias), '常量',也可以它们的自由组合。 +在本次案例中,rowKey为了简单,设置成了"0,1,2"这样的数值型字符,若有必要,也可以设计得更为复杂。 ### hbase异步维表关联完整案例 ``` CREATE TABLE MyTable( From 2b0af472c7d46e792c43db9070a7691fa23d1512 Mon Sep 17 00:00:00 2001 From: dapeng Date: Thu, 23 Apr 2020 11:02:30 +0800 Subject: [PATCH 057/109] =?UTF-8?q?=E4=BF=AE=E5=A4=8D=E7=BB=B4=E8=A1=A8?= =?UTF-8?q?=E6=95=B0=E6=8D=AE=E9=94=99=E4=B9=B1=E9=97=AE=E9=A2=98http://re?= =?UTF-8?q?dmine.prod.dtstack.cn/issues/24996?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 971285e52..52a27fcc3 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 @@ -88,7 +88,7 @@ protected void init(SideInfo sideInfo) { @Override public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exception { - CRow copyCrow = new CRow(input.row(), input.change()); + CRow copyCrow = new CRow(Row.copy(input.row()), input.change()); JsonArray inputParams = new JsonArray(); for (Integer conValIndex : sideInfo.getEqualValIndex()) { Object equalObj = copyCrow.row().getField(conValIndex); From 73ed644f80fef243dc97c5bbde86edc114e2b57a Mon Sep 17 00:00:00 2001 From: dapeng Date: Thu, 23 Apr 2020 16:09:26 +0800 Subject: [PATCH 058/109] =?UTF-8?q?=E4=B9=B1=E5=BA=8F=EF=BC=8C=E5=88=AB?= =?UTF-8?q?=E5=90=8D=E5=92=8Chase?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flink/sql/side/SideInfo.java | 5 ++--- .../com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java | 2 +- .../flink/sql/side/hbase/table/HbaseSideParser.java | 1 + .../dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java | 7 +++---- 4 files changed, 7 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java index d8a22e022..4b53f77f7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java @@ -132,13 +132,12 @@ public void dealOneEqualCon(SqlNode sqlNode, String sideTableName){ String leftTableName = left.getComponent(0).getSimple(); String leftField = left.getComponent(1).getSimple(); - Map physicalFields = sideTableInfo.getPhysicalFields(); String rightTableName = right.getComponent(0).getSimple(); String rightField = right.getComponent(1).getSimple(); if(leftTableName.equalsIgnoreCase(sideTableName)){ - equalFieldList.add(physicalFields.get(leftField)); + equalFieldList.add(leftField); int equalFieldIndex = -1; for(int i=0; i resultFuture) throws Exception { - CRow inputCopy = new CRow(input.row(), input.change()); + CRow inputCopy = new CRow(Row.copy(input.row()), input.change()); Map refData = Maps.newHashMap(); for (int i = 0; i < sideInfo.getEqualValIndex().size(); i++) { Integer conValIndex = sideInfo.getEqualValIndex().get(i); diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java index 7b627da4d..3e9ab7f45 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java @@ -100,6 +100,7 @@ private void dealField(Matcher matcher, TableInfo tableInfo){ sideTableInfo.addFieldClass(fieldClass); sideTableInfo.addFieldType(fieldType); sideTableInfo.putAliasNameRef(aliasStr, fieldName); + sideTableInfo.addPhysicalMappings(aliasStr, fieldName); } 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 f3830793e..7aa83b3c8 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 @@ -87,7 +87,6 @@ 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(); @@ -96,7 +95,7 @@ public void dealOneEqualCon(SqlNode sqlNode, String sideTableName) { String rightField = right.getComponent(1).getSimple(); if (leftTableName.equalsIgnoreCase(sideTableName)) { - equalFieldList.add(physicalFields.get(leftField)); + equalFieldList.add(leftField); int equalFieldIndex = -1; for (int i = 0; i < rowTypeInfo.getFieldNames().length; i++) { String fieldName = rowTypeInfo.getFieldNames()[i]; @@ -112,7 +111,7 @@ public void dealOneEqualCon(SqlNode sqlNode, String sideTableName) { } else if (rightTableName.equalsIgnoreCase(sideTableName)) { - equalFieldList.add(physicalFields.get(rightField)); + equalFieldList.add(rightField); int equalFieldIndex = -1; for (int i = 0; i < rowTypeInfo.getFieldNames().length; i++) { String fieldName = rowTypeInfo.getFieldNames()[i]; @@ -144,7 +143,7 @@ public String getSelectFromStatement(String tableName, List selectFields .collect(Collectors.joining(", ")); String whereClause = conditionFields.stream() - .map(f -> quoteIdentifier(f) + sqlJoinCompareOperate.get(conditionFields.indexOf(f)) + wrapperPlaceholder(f)) + .map(f -> quoteIdentifier(sideTableInfo.getPhysicalFields().getOrDefault(f, f)) + sqlJoinCompareOperate.get(conditionFields.indexOf(f)) + wrapperPlaceholder(f)) .collect(Collectors.joining(" AND ")); String predicateClause = predicateInfoes.stream() From 93844cff640be6676305ceeeddef3ea3ea655f79 Mon Sep 17 00:00:00 2001 From: maqi Date: Thu, 23 Apr 2020 16:29:57 +0800 Subject: [PATCH 059/109] pgsink sqlserversink doc --- docs/plugin/mysqlSink.md | 2 +- docs/plugin/postgresqlSink.md | 101 ++++++++++++++++++++++++++++------ docs/plugin/sqlserverSink.md | 96 ++++++++++++++++++++++++++------ 3 files changed, 164 insertions(+), 35 deletions(-) diff --git a/docs/plugin/mysqlSink.md b/docs/plugin/mysqlSink.md index 8dabea047..254f11f2a 100644 --- a/docs/plugin/mysqlSink.md +++ b/docs/plugin/mysqlSink.md @@ -36,7 +36,7 @@ CREATE TABLE tableName( |password | mysql连接密码|是|| |tableName | mysql表名称|是|| |parallelism | 并行度设置|否|1| -|batchNum | flush的大小|否|100| +|batchSize | flush的大小|否|100| |batchWaitInterval | flush的时间间隔,单位ms|否|1000| |allReplace| true:新值替换旧值|否|false| |updateMode| APPEND:不回撤数据,只下发增量数据,UPSERT:先删除回撤数据,然后更新|否|| diff --git a/docs/plugin/postgresqlSink.md b/docs/plugin/postgresqlSink.md index 6d7d87834..c04ff0ec0 100644 --- a/docs/plugin/postgresqlSink.md +++ b/docs/plugin/postgresqlSink.md @@ -3,7 +3,8 @@ CREATE TABLE tableName( colName colType, ... - colNameX colType + colNameX colType, + primary key (colName) )WITH( type ='postgresql', url ='jdbcUrl', @@ -25,29 +26,95 @@ CREATE TABLE tableName( | tableName| 在 sql 中使用的名称;即注册到flink-table-env上的名称| | colName | 列名称| | colType | 列类型 [colType支持的类型](docs/colType.md)| +| primary key | updateMode为UPSERT时,需要指定的主键信息| + ## 4.参数: |参数名称|含义|是否必填|默认值| |----|----|----|----| -| type |表明 输出表类型[postgresql]|是|| -| url | 连接postgresql数据库 jdbcUrl |是|| -| userName | postgresql连接用户名 |是|| -| password | postgresql连接密码|是|| -| tableName | postgresqll表名称|是|| -| parallelism | 并行度设置|否|1| +|type |结果表插件类型,必须为postgresql|是|| +|url | 连接postgresql数据库 jdbcUrl |是|| +|userName |postgresql连接用户名 |是|| +|password | postgresql连接密码|是|| +|tableName | postgresql表名称|是|| +|schema | postgresql表空间|否|| +|parallelism | 并行度设置|否|1| +|batchSize | flush的大小|否|100| +|batchWaitInterval | flush的时间间隔,单位ms|否|1000| +|allReplace| true:新值替换旧值|否|false| +|updateMode| APPEND:不回撤数据,只下发增量数据,UPSERT:先删除回撤数据,然后更新|否|| ## 5.样例: + +回溯流删除 + ``` + +CREATE TABLE source1 ( + id int, + name VARCHAR +)WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest03', + timezone='Asia/Shanghai', + topicIsPattern ='false' + ); + + + +CREATE TABLE source2( + id int, + address VARCHAR +)WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest04', + timezone='Asia/Shanghai', + topicIsPattern ='false' +); + + CREATE TABLE MyResult( - channel VARCHAR, - pv VARCHAR - )WITH( - type ='postgresql', - url ='jdbc:postgresql://localhost:9001/test?sslmode=disable', - userName ='dtstack', - password ='abc123', - tableName ='pv2', - parallelism ='1' - ) + id int, + name VARCHAR, + address VARCHAR, + primary key (id) +)WITH( + type='postgresql', + url='jdbc:postgresql://172.16.8.193:5432/DTstack', + userName='root', + password='123456', + tableName='userInfo', + schema = 'aaa', + updateMode = 'upsert', + batchSize = '1' +); + +insert into MyResult +select + s1.id, + s1.name, + s2.address +from + source1 s1 +left join + source2 s2 +on + s1.id = s2.id + + + ``` + + +数据结果: + +向Topic mqTest03 发送数据 {"name":"maqi","id":1001} 插入 (1001,"maqi",null) + +向Topic mqTest04 发送数据 {"address":"hz","id":1001} 删除 (1001,"maqi",null) 插入 (1001,"maqi","hz") diff --git a/docs/plugin/sqlserverSink.md b/docs/plugin/sqlserverSink.md index 8ecb14308..0ab9431e9 100644 --- a/docs/plugin/sqlserverSink.md +++ b/docs/plugin/sqlserverSink.md @@ -3,7 +3,8 @@ CREATE TABLE tableName( colName colType, ... - colNameX colType + colNameX colType, + primary key (colName) )WITH( type ='sqlserver', url ='jdbcUrl', @@ -25,29 +26,90 @@ CREATE TABLE tableName( | tableName| sqlserver表名称| | colName | 列名称| | colType | 列类型 [colType支持的类型](docs/colType.md)| +| primary key | updateMode为UPSERT时,需要指定的主键信息| ## 4.参数: |参数名称|含义|是否必填|默认值| |----|----|----|----| -|type |表名 输出表类型[mysq|hbase|elasticsearch]|是|| +|type |结果表插件类型,必须为sqlserver|是|| |url | 连接sqlserver数据库 jdbcUrl |是|| -|userName | sqlserver连接用户名 |是|| -| password | sqlserver连接密码|是|| -| tableName | sqlserver表名称|是|| -| parallelism | 并行度设置|否|1| +|userName |sqlserver连接用户名 |是|| +|password | sqlserver连接密码|是|| +|tableName | sqlserver表名称|是|| +|schema | sqlserver表空间|否|| +|parallelism | 并行度设置|否|1| +|batchSize | flush的大小|否|100| +|batchWaitInterval | flush的时间间隔,单位ms|否|1000| +|allReplace| true:新值替换旧值|否|false| +|updateMode| APPEND:不回撤数据,只下发增量数据,UPSERT:先删除回撤数据,然后更新|否|| + ## 5.样例: + +回溯流删除 ``` + +CREATE TABLE source1 ( + id int, + name VARCHAR +)WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest03', + timezone='Asia/Shanghai', + topicIsPattern ='false' + ); + + +CREATE TABLE source2( + id int, + address VARCHAR +)WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest04', + timezone='Asia/Shanghai', + topicIsPattern ='false' +); + + CREATE TABLE MyResult( - channel VARCHAR, - pv VARCHAR - )WITH( - type ='sqlserver', - url ='jdbc:jtds:sqlserver://172.16.8.104:1433;DatabaseName=mytest', - userName ='dtstack', - password ='abc123', - tableName ='pv2', - parallelism ='1' - ) - ``` \ No newline at end of file + id int, + name VARCHAR, + address VARCHAR, + primary key (id) +)WITH( + type='sqlserver', + url='jdbc:jtds:sqlserver://172.16.8.149:1433;DatabaseName=DTstack', + userName='sa', + password='Dtstack2018', + tableName='user', + schema = 'aaa', + updateMode = 'upsert', + batchSize = '1' +); + +insert into MyResult +select + s1.id, + s1.name, + s2.address +from + source1 s1 +left join + source2 s2 +on + s1.id = s2.id + + ``` + +数据结果: + +向Topic mqTest03 发送数据 {"name":"maqi","id":1001} 插入 (1001,"maqi",null) + +向Topic mqTest04 发送数据 {"address":"hz","id":1001} 删除 (1001,"maqi",null) 插入 (1001,"maqi","hz") From 83571b419945d3540c4ca3d7104318673cbb8d5a Mon Sep 17 00:00:00 2001 From: gkd Date: Thu, 23 Apr 2020 20:23:52 +0800 Subject: [PATCH 060/109] postgresql-side clickhouse-sise --- docs/plugin/clickhouseSide.md | 170 ++++++++++++++++++++++++---------- docs/plugin/postgresqlSide.md | 162 +++++++++++++++++++++++--------- 2 files changed, 239 insertions(+), 93 deletions(-) diff --git a/docs/plugin/clickhouseSide.md b/docs/plugin/clickhouseSide.md index 57e4eef6b..df050ca8b 100644 --- a/docs/plugin/clickhouseSide.md +++ b/docs/plugin/clickhouseSide.md @@ -1,6 +1,9 @@ - ## 1.格式: -``` + +通过建表语句中的` PERIOD FOR SYSTEM_TIME`将表标识为维表,其中`PRIMARY KEY(keyInfo)`中的keyInfo,表示用来和源表进行关联的字段, + 维表JOIN的条件必须与`keyInfo`字段一致。 + +```sql CREATE TABLE tableName( colName cloType, ... @@ -21,65 +24,132 @@ ``` # 2.支持版本 + 19.14.x、19.15.x、19.16.x - + ## 3.表结构定义 - - |参数名称|含义| - |----|---| - | tableName | clickhouse表名称| - | colName | 列名称| - | colType | 列类型 [colType支持的类型](docs/colType.md)| - | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| - | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| - -## 4.参数 - - |参数名称|含义|是否必填|默认值| - |----|---|---|----| - | type | 表明维表的类型 clickhouse |是|| - | url | 连接clickhouse数据库 jdbcUrl |是|| - | userName | clickhouse连接用户名 |是|| - | password | clickhouse连接密码|是|| - | tableName | clickhouse表名称|是|| - | tableName | clickhouse 的表名称|是|| - | cache | 维表缓存策略(NONE/LRU)|否|NONE| - | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| - - ---------- - > 缓存策略 - * NONE: 不做内存缓存 - * LRU: - * cacheSize: 缓存的条目数量 - * cacheTTLMs:缓存的过期时间(ms) - * cacheMode: (unordered|ordered)异步加载是有序还是无序,默认有序。 - * asyncCapacity:异步请求容量,默认1000 - * asyncTimeout:异步请求超时时间,默认10000毫秒 - -## 5.样例 + + [维表参数信息](docs/plugin/sideParams.md) + +clinkhose独有的参数信息 + +| 参数名称 | 含义 | 是否必填 | 默认值 | +| -------- | --------------------- | -------- | ------ | +| type | 维表类型, clinkhouse | 是 | | +| url | 连接数据库 jdbcUrl | 是 | | +| userName | 连接用户名 | 是 | | +| password | 连接密码 | 是 | | + +## 4.样例 + +------ + +## ALL全量维表定义 + +```sql +// 定义全量维表 +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id), + PERIOD FOR SYSTEM_TIME + )WITH( + type ='clickhouse', + url ='jdbc:clickhouse://172.16.8.104:3306/zftest', + userName ='dtstack', + password ='abc123', + tableName ='all_test_clinkhousesql', + cache ='ALL', + cacheTTLMs ='60000', + parallelism ='1' + ); ``` -create table sideTable( - channel varchar, - xccount int, - PRIMARY KEY(channel), + +### LRU异步维表定义 + +``` +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , PERIOD FOR SYSTEM_TIME )WITH( - type='clickhouse', - url='jdbc:clickhouse://172.16.8.104:3306/test?charset=utf8', - userName='dtstack', - password='abc123', - tableName='sidetest', + type ='clickhousesql', + url ='jdbc:clickhousesql://172.16.8.104:3306/zftest', + userName ='dtstack', + password ='abc123', + tableName ='lru_test_clickhousesql', + partitionedJoin ='false', cache ='LRU', cacheSize ='10000', cacheTTLMs ='60000', - cacheMode='unordered', - asyncCapacity='1000', - asyncTimeout='10000' + asyncPoolSize ='3', + parallelism ='1' + ); +``` + +### ClickHouseSQL异步维表关联 + +```sql +CREATE TABLE MyTable( + id int, + name varchar + )WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='cannan_zftest01', + timezone='Asia/Shanghai', + enableKeyPartitions ='false', + topicIsPattern ='false', + parallelism ='1' + ); + +CREATE TABLE MyResult( + id INT, + name VARCHAR + )WITH( + type ='clickhousesql', + url ='jdbc:clickhousesql://172.16.8.104:3306/zftest', + userName ='dtstack', + password ='abc123', + tableName ='test_clickhouse_zf', + updateMode ='append', parallelism ='1', - partitionedJoin='false' + batchSize ='100', + batchWaitInterval ='1000' ); +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , + PERIOD FOR SYSTEM_TIME + )WITH( + type ='clickhousesql', + url ='jdbc:clickhousesql://172.16.8.104:3306/zftest', + userName ='dtstack', + password ='abc123', + tableName ='test_clickhouse_10', + partitionedJoin ='false', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + asyncPoolSize ='3', + parallelism ='1' + ); +insert +into + MyResult + select + m.id, + s.name + from + MyTable m + join + sideTable s + on m.id=s.id; ``` - diff --git a/docs/plugin/postgresqlSide.md b/docs/plugin/postgresqlSide.md index 6928042f6..b50a2de52 100644 --- a/docs/plugin/postgresqlSide.md +++ b/docs/plugin/postgresqlSide.md @@ -1,5 +1,8 @@ - ## 1.格式: + +通过建表语句中的` PERIOD FOR SYSTEM_TIME`将表标识为维表,其中`PRIMARY KEY(keyInfo)`中的keyInfo,表示用来和源表进行关联的字段, + 维表JOIN的条件必须与`keyInfo`字段一致。 + ``` CREATE TABLE tableName( colName cloType, @@ -21,59 +24,132 @@ ``` # 2.支持版本 + postgresql-8.2+ - + ## 3.表结构定义 - - |参数名称|含义| - |----|---| - | tableName | 注册到flink的表名称| - | colName | 列名称| - | colType | 列类型 [colType支持的类型](docs/colType.md)| - | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| - | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| - -## 4.参数 - - |参数名称|含义|是否必填|默认值| - |----|---|---|----| - | type | 表明维表的类型[postgresql] |是|| - | url | 连接postgresql数据库 jdbcUrl |是|| - | userName | postgresql连接用户名 |是|| - | password | postgresql连接密码|是|| - | tableName | postgresql表名称|是|| - | cache | 维表缓存策略(NONE/LRU/ALL)|否|NONE| - | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| - - ---------- - > 缓存策略 - * NONE: 不做内存缓存 - * LRU: - * cacheSize: 缓存的条目数量 - * cacheTTLMs:缓存的过期时间(ms) - - -## 5.样例 + + [维表参数信息](docs/plugin/sideParams.md) + +postgresql独有的参数配置 + +| 参数名称 | 含义 | 是否必填 | 默认值 | +| -------- | ------------------ | -------- | ------ | +| type | 维表类型, mysql | 是 | | +| url | 连接数据库 jdbcUrl | 是 | | +| userName | 连接用户名 | 是 | | +| password | 连接密码 | 是 | | + +## 4.样例 + +### ALL全量维表定义 + +```sql + // 定义全量维表 +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , + PERIOD FOR SYSTEM_TIME + )WITH( + type ='postgresql', + url ='jdbc:postgresql://172.16.10.194:5432/zftest', + userName ='dtstack', + password ='abc123', + tableName ='all_test_postgresql', + cache ='ALL', + cacheTTLMs ='60000', + parallelism ='1' + ); ``` -create table sideTable( - channel varchar, - xccount int, - PRIMARY KEY(channel), + +### LRU异步维表定义 + +```sql +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , PERIOD FOR SYSTEM_TIME )WITH( - type='postgresql', - url='jdbc:postgresql://localhost:9001/test?sslmode=disable', - userName='dtstack', - password='abc123', - tableName='sidetest', + type ='postgresql', + url ='jdbc:postgresql://172.16.10.194:5432/zftest', + userName ='dtstack', + password ='abc123', + tableName ='lru_test_postgresql', + partitionedJoin ='false', cache ='LRU', cacheSize ='10000', cacheTTLMs ='60000', + asyncPoolSize ='3', + parallelism ='1' + ); + +``` + +### PostgreSQL异步维表关联 + +```sql +CREATE TABLE MyTable( + id int, + name varchar + )WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='cannan_zftest01', + timezone='Asia/Shanghai', + enableKeyPartitions ='false', + topicIsPattern ='false', + parallelism ='1' + ); + +CREATE TABLE MyResult( + id INT, + name VARCHAR + )WITH( + type ='postgresql', + url ='jdbc:postgresql://172.16.10.194:5432/zftest', + userName ='dtstack', + password ='abc123', + tableName ='test_postgresql_zf', + updateMode ='append', parallelism ='1', - partitionedJoin='false' + batchSize ='100', + batchWaitInterval ='1000' ); +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , + PERIOD FOR SYSTEM_TIME + )WITH( + type ='postgresql', + url ='jdbc:postgresql://172.16.10.194:5432/zftest', + userName ='dtstack', + password ='abc123', + tableName ='test_postgresql_10', + partitionedJoin ='false', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + asyncPoolSize ='3', + parallelism ='1' + ); -``` +insert +into + MyResult + select + m.id, + s.name + from + MyTable m + join + sideTable s + on m.id=s.id; +``` From e4bedb2dac3b135d331900a64ed0ef7759322e46 Mon Sep 17 00:00:00 2001 From: maqi Date: Fri, 24 Apr 2020 15:41:47 +0800 Subject: [PATCH 061/109] ck impala sink doc --- docs/plugin/clickhouseSink.md | 93 ++++++++++++++++++++++++++++++----- docs/plugin/impalaColType.md | 14 ++++++ docs/plugin/impalaSink.md | 4 +- 3 files changed, 98 insertions(+), 13 deletions(-) create mode 100644 docs/plugin/impalaColType.md diff --git a/docs/plugin/clickhouseSink.md b/docs/plugin/clickhouseSink.md index 780ac77b0..6b4a724be 100644 --- a/docs/plugin/clickhouseSink.md +++ b/docs/plugin/clickhouseSink.md @@ -24,7 +24,7 @@ CREATE TABLE tableName( |----|---| | tableName| clickhouse表名称| | colName | 列名称| -| colType | 列类型 [colType支持的类型](docs/colType.md)| +| colType | clickhouse基本数据类型,不包括Array,Tuple,Nested等| ## 4.参数: @@ -36,18 +36,87 @@ CREATE TABLE tableName( | password | clickhouse 连接密码|是|| | tableName | clickhouse 表名称|是|| | parallelism | 并行度设置|否|1| +|updateMode| 只支持APPEND模式,过滤掉回撤数据||| + ## 5.样例: + + ``` + +CREATE TABLE source1 ( + id int, + name VARCHAR +)WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest03', + timezone='Asia/Shanghai', + topicIsPattern ='false' + ); + + + +CREATE TABLE source2( + id int, + address VARCHAR +)WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest04', + timezone='Asia/Shanghai', + topicIsPattern ='false' +); + + CREATE TABLE MyResult( - channel VARCHAR, - pv VARCHAR - )WITH( - type ='clickhouse', - url ='jdbc:clickhouse://172.16.8.104:3306/test?charset=utf8', - userName ='dtstack', - password ='abc123', - tableName ='pv2', - parallelism ='1' - ) - ``` \ No newline at end of file + id int, + name VARCHAR, + address VARCHAR +)WITH( + type='clickhouse', + url='jdbc:clickhouse://172.16.10.168:8123/tudou?charset=utf8', + userName='dtstack', + password='abc123', + tableName='MyResult', + updateMode = 'append' +); + +insert into MyResult +select + s1.id, + s1.name, + s2.address +from + source1 s1 +left join + source2 s2 +on + s1.id = s2.id + + + ``` + + +**Clickhouse建表语句** + + + ```aidl +CREATE TABLE tudou.MyResult (`id` Int64, `name` String, `address` String) + ENGINE = MergeTree PARTITION BY address ORDER BY id SETTINGS index_granularity = 8192 +``` + + + + +数据结果: + +向Topic mqTest03 发送数据 {"name":"maqi","id":1001} 插入 (1001,"maqi",null) + +向Topic mqTest04 发送数据 {"address":"hz","id":1001} 插入 (1001,"maqi","hz") + + \ No newline at end of file diff --git a/docs/plugin/impalaColType.md b/docs/plugin/impalaColType.md new file mode 100644 index 000000000..4b84b3aa3 --- /dev/null +++ b/docs/plugin/impalaColType.md @@ -0,0 +1,14 @@ +| 支持的类型 | java对应类型 | +| ------ | ----- | +| boolean | Boolean | +| char | Character | +| double | Double| +| float | Float| +| tinyint | Byte | +| smallint | Short| +| int | Integer | +| bigint | Long | +| decimal |BigDecimal| +| string | String | +| varchar | String | +| timestamp | Timestamp | \ No newline at end of file diff --git a/docs/plugin/impalaSink.md b/docs/plugin/impalaSink.md index af3baa282..398ec6ef2 100644 --- a/docs/plugin/impalaSink.md +++ b/docs/plugin/impalaSink.md @@ -24,7 +24,7 @@ CREATE TABLE tableName( |----|---| | tableName| 在 sql 中使用的名称;即注册到flink-table-env上的名称| | colName | 列名称| -| colType | 列类型 [colType支持的类型](docs/colType.md)| +| colType | 列类型 [colType支持的类型](docs/plugin/impalaColType.md)| ## 4.参数: @@ -45,6 +45,8 @@ CREATE TABLE tableName( | enablePartition | 是否支持分区 |否|false| | partitionFields | 分区字段名|否,enablePartition='true'时为必填|| | parallelism | 并行度设置|否|1| +| parallelism | 并行度设置|否|1| +|updateMode| 只支持APPEND模式,过滤掉回撤数据||| ## 5.样例: From f1e1d28f6caff24952e8dc747104bd5c20cce147 Mon Sep 17 00:00:00 2001 From: dapeng Date: Fri, 24 Apr 2020 16:02:51 +0800 Subject: [PATCH 062/109] =?UTF-8?q?where=20=E6=9D=A1=E4=BB=B6=E5=9C=A8?= =?UTF-8?q?=E7=BB=B4=E8=A1=A8=E4=B8=AD=E4=B8=8B=E5=8F=91?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flink/sql/side/SidePredicatesParser.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java index 4d6112aa8..7b89b25fd 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java @@ -101,6 +101,8 @@ private void parseSql(SqlNode sqlNode, Map sideTableMap, parseSql(unionLeft, sideTableMap, tabMapping); parseSql(unionRight, sideTableMap, tabMapping); break; + default: + break; } } @@ -128,10 +130,10 @@ private void extractPredicateInfo(SqlNode whereNode, List predica // 跳过函数 if ((((SqlBasicCall) whereNode).getOperands()[0] instanceof SqlIdentifier) - && (((SqlBasicCall) whereNode).getOperands()[1].getKind() != SqlKind.OTHER_FUNCTION)) { + && (((SqlBasicCall) whereNode).getOperands()[1].getKind() == SqlKind.LITERAL)) { fillPredicateInfoToList((SqlBasicCall) whereNode, predicatesInfoList, operatorName, operatorKind, 0, 1); } else if ((((SqlBasicCall) whereNode).getOperands()[1] instanceof SqlIdentifier) - && (((SqlBasicCall) whereNode).getOperands()[0].getKind() != SqlKind.OTHER_FUNCTION)) { + && (((SqlBasicCall) whereNode).getOperands()[0].getKind() == LITERAL)) { fillPredicateInfoToList((SqlBasicCall) whereNode, predicatesInfoList, operatorName, operatorKind, 1, 0); } } From 9a2a8915b67ca3a0b0c992420fa82a42bd6a8c41 Mon Sep 17 00:00:00 2001 From: maqi Date: Sun, 26 Apr 2020 16:52:05 +0800 Subject: [PATCH 063/109] db2 upsert --- .../dtstack/flink/sql/sink/db/DbDialect.java | 80 +++++++++++++++++++ .../com/dtstack/flink/sql/sink/db/DbSink.java | 1 + kafka09/kafka09-sink/pom.xml | 1 + kafka09/kafka09-source/pom.xml | 1 + 4 files changed, 83 insertions(+) diff --git a/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/DbDialect.java b/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/DbDialect.java index 9c7985d8f..49a3613bc 100644 --- a/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/DbDialect.java +++ b/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/DbDialect.java @@ -19,8 +19,13 @@ package com.dtstack.flink.sql.sink.db; import com.dtstack.flink.sql.sink.rdb.dialect.JDBCDialect; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.commons.lang3.StringUtils; +import java.util.Arrays; +import java.util.List; import java.util.Optional; +import java.util.stream.Collectors; /** * Date: 2020/1/19 @@ -43,4 +48,79 @@ public String quoteIdentifier(String identifier) { return identifier; } + @Override + public Optional getUpsertStatement(String schema, String tableName, String[] fieldNames, String[] uniqueKeyFields, boolean allReplace) { + tableName = DtStringUtil.getTableFullPath(schema, tableName); + StringBuilder sb = new StringBuilder(); + sb.append("MERGE INTO " + tableName + " T1 USING " + + "(" + buildValuesStatement(fieldNames) + ") T2 (" + + buildFiledNameStatement(fieldNames) + + ") ON (" + + buildConnectionConditions(uniqueKeyFields) + ") "); + + String updateSql = buildUpdateConnection(fieldNames, uniqueKeyFields, allReplace); + + if (StringUtils.isNotEmpty(updateSql)) { + sb.append(" WHEN MATCHED THEN UPDATE SET "); + sb.append(updateSql); + } + + sb.append(" WHEN NOT MATCHED THEN " + + "INSERT (" + Arrays.stream(fieldNames).map(this::quoteIdentifier).collect(Collectors.joining(",")) + ") VALUES (" + + Arrays.stream(fieldNames).map(col -> "T2." + quoteIdentifier(col)).collect(Collectors.joining(",")) + ")"); + return Optional.of(sb.toString()); + } + + /** + * build T1."A"=T2."A" or T1."A"=nvl(T2."A",T1."A") + * @param fieldNames + * @param uniqueKeyFields + * @param allReplace + * @return + */ + private String buildUpdateConnection(String[] fieldNames, String[] uniqueKeyFields, boolean allReplace) { + List uniqueKeyList = Arrays.asList(uniqueKeyFields); + return Arrays.stream(fieldNames) + .filter(col -> !uniqueKeyList.contains(col)) + .map(col -> buildConnectString(allReplace, col)) + .collect(Collectors.joining(",")); + } + + private String buildConnectString(boolean allReplace, String col) { + return allReplace ? quoteIdentifier("T1") + "." + quoteIdentifier(col) + " = " + quoteIdentifier("T2") + "." + quoteIdentifier(col) : + quoteIdentifier("T1") + "." + quoteIdentifier(col) + " =NVL(" + quoteIdentifier("T2") + "." + quoteIdentifier(col) + "," + + quoteIdentifier("T1") + "." + quoteIdentifier(col) + ")"; + } + + + private String buildConnectionConditions(String[] uniqueKeyFields) { + return Arrays.stream(uniqueKeyFields).map(col -> "T1." + quoteIdentifier(col) + "=T2." + quoteIdentifier(col)).collect(Collectors.joining(",")); + } + + /** + * build sql part e.g: VALUES('1001','zs','sss') + * + * @param column destination column + * @return + */ + public String buildValuesStatement(String[] column) { + StringBuilder sb = new StringBuilder("VALUES("); + String collect = Arrays.stream(column) + .map(col -> " ? ") + .collect(Collectors.joining(", ")); + + return sb.append(collect).append(")").toString(); + } + + /** + * build sql part e.g: id, name, address + * @param column + * @return + */ + public String buildFiledNameStatement(String[] column) { + return Arrays.stream(column) + .collect(Collectors.joining(", ")); + } + + } diff --git a/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/DbSink.java b/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/DbSink.java index 9942d4438..573a0896a 100644 --- a/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/DbSink.java +++ b/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/DbSink.java @@ -19,6 +19,7 @@ public JDBCUpsertOutputFormat getOutputFormat() { .setDialect(jdbcDialect) .setUsername(userName) .setPassword(password) + .setSchema(schema) .setTableName(tableName) .build(); diff --git a/kafka09/kafka09-sink/pom.xml b/kafka09/kafka09-sink/pom.xml index 8f52d7bc8..e01c63cc3 100644 --- a/kafka09/kafka09-sink/pom.xml +++ b/kafka09/kafka09-sink/pom.xml @@ -37,6 +37,7 @@ shade + false org.slf4j diff --git a/kafka09/kafka09-source/pom.xml b/kafka09/kafka09-source/pom.xml index deef866c9..dde2352a1 100644 --- a/kafka09/kafka09-source/pom.xml +++ b/kafka09/kafka09-source/pom.xml @@ -35,6 +35,7 @@ shade + false org.slf4j From 2e4721b6770c731015214633a181559a93bb655d Mon Sep 17 00:00:00 2001 From: maqi Date: Sun, 26 Apr 2020 17:35:58 +0800 Subject: [PATCH 064/109] db2 doc --- docs/plugin/db2Side.md | 162 ++++++++++++++++++++++++++-------- docs/plugin/db2Sink.md | 120 +++++++++++++++++++++---- docs/plugin/mysqlSink.md | 5 +- docs/plugin/postgresqlSink.md | 2 +- 4 files changed, 233 insertions(+), 56 deletions(-) diff --git a/docs/plugin/db2Side.md b/docs/plugin/db2Side.md index 8381d6206..af0c29c02 100644 --- a/docs/plugin/db2Side.md +++ b/docs/plugin/db2Side.md @@ -1,5 +1,7 @@ ## 1.格式: + 通过建表语句中的` PERIOD FOR SYSTEM_TIME`将表标识为维表,其中`PRIMARY KEY(keyInfo)`中的keyInfo,表示用来和源表进行关联的字段, + 维表JOIN的条件必须与`keyInfo`字段一致。 ``` CREATE TABLE tableName( colName cloType, @@ -25,55 +27,145 @@ ## 3.表结构定义 + + [维表参数信息](docs/plugin/sideParams.md) + db2独有的参数配置: + |参数名称|含义| |----|---| - | tableName | db2表名称| - | colName | 列名称| - | colType | 列类型 [colType支持的类型](docs/colType.md)| - | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| - | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| +| type | 维表类型, db2 |是|| +| url | 连接数据库 jdbcUrl |是|| +| userName | 连接用户名 |是|| +| password | 连接密码|是|| +| schema | 表所属scheam|否|| -## 4.参数 - - |参数名称|含义|是否必填|默认值| - |----|---|---|----| - | type | 表明维表的类型 db2 |是|| - | url | 连接mysql数据库 jdbcUrl |是|| - | userName | db2连接用户名 |是|| - | password | db2连接密码|是|| - | tableName | db2表名称|是|| - | cache | 维表缓存策略(NONE/LRU)|否|NONE| - | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| - - ---------- - > 缓存策略 - * NONE: 不做内存缓存 - * LRU: - * cacheSize: 缓存的条目数量 - * cacheTTLMs:缓存的过期时间(ms) - - -## 5.样例 + + +## 4.样例 + +### ALL全量维表定义 ``` -create table sideTable( - channel varchar, - xccount int, - PRIMARY KEY(channel), + // 定义全量维表 +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , PERIOD FOR SYSTEM_TIME )WITH( type='db2', - url='jdbc:db2://172.16.8.104:50000/test?charset=utf8', - userName='dtstack', + url='jdbc:db2://172.16.10.251:50000/mqTest', + userName='DB2INST1', password='abc123', - tableName='sidetest', + tableName='USER_INFO2', + schema = 'DTSTACK' + cache ='ALL', + cacheTTLMs ='60000', + parallelism ='2' + ); + +``` +### LRU异步维表定义 + +``` +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , + PERIOD FOR SYSTEM_TIME + )WITH( + type='db2', + url='jdbc:db2://172.16.10.251:50000/mqTest', + userName='DB2INST1', + password='abc123', + tableName='USER_INFO2', + schema = 'DTSTACK' + partitionedJoin ='false', cache ='LRU', cacheSize ='10000', cacheTTLMs ='60000', - parallelism ='1', - partitionedJoin='false' + asyncPoolSize ='3', + parallelism ='2' + ); + +``` + + +### DB2异步维表关联输出到Console +``` + +CREATE TABLE source1 ( + id int, + name VARCHAR +)WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest03', + timezone='Asia/Shanghai', + topicIsPattern ='false' ); + +CREATE TABLE source2( + id int, + address VARCHAR, + PERIOD FOR SYSTEM_TIME +)WITH( + type='db2', + url='jdbc:db2://172.16.10.251:50000/mqTest', + userName='DB2INST1', + password='abc123', + tableName='USER_INFO2', + schema = 'DTSTACK', + batchSize = '1' +); + + +CREATE TABLE MyResult( + id int, + name VARCHAR, + address VARCHAR, + primary key (id) +)WITH( + type='console' +); + +insert into MyResult +select + s1.id, + s1.name, + s2.address +from + source1 s1 +left join + source2 s2 +on + s1.id = s2.id + ``` +DB2维表字段信息 + +```aidl +-- DTSTACK.USER_INFO2 definition + +CREATE TABLE "DTSTACK "."USER_INFO2" ( + "ID" INTEGER , + "NAME" VARCHAR(50 OCTETS) , + "ADDRESS" VARCHAR(50 OCTETS) ) + IN "USERSPACE1" + ORGANIZE BY ROW + ; + +GRANT CONTROL ON TABLE "DTSTACK "."USER_INFO2" TO USER "DB2INST1" +; +``` + +维表数据:(1001,maqi,hz) + +源表数据:{"name":"maqi","id":1001} + +输出结果: (1001,maqi,hz) \ No newline at end of file diff --git a/docs/plugin/db2Sink.md b/docs/plugin/db2Sink.md index 0fcbdd3d3..9544e44f1 100644 --- a/docs/plugin/db2Sink.md +++ b/docs/plugin/db2Sink.md @@ -3,7 +3,8 @@ CREATE TABLE tableName( colName colType, ... - colNameX colType + colNameX colType, + [primary key (colName)] )WITH( type ='db2', url ='jdbcUrl', @@ -25,29 +26,112 @@ CREATE TABLE tableName( | tableName| db2表名称| | colName | 列名称| | colType | 列类型 [colType支持的类型](docs/colType.md)| +| primary key | updateMode为UPSERT时,需要指定的主键信息,不需要和数据库一致| + ## 4.参数: |参数名称|含义|是否必填|默认值| |----|----|----|----| -| type |表名 输出表类型[mysq|hbase|elasticsearch]|是|| -| url | 连接db2数据库 jdbcUrl |是|| -| userName | db2连接用户名 |是|| -| password | db2连接密码|是|| -| tableName | db2表名称|是|| -| parallelism | 并行度设置|否|1| +|type |结果表插件类型,必须为db2|是|| +|url | 连接db2数据库 jdbcUrl |是|| +|userName |db2连接用户名 |是|| +|password | db2连接密码|是|| +|tableName | db2表名称|是|| +|schema | db2表空间|否|| +|parallelism | 并行度设置|否|1| +|batchSize | flush的大小|否|100| +|batchWaitInterval | flush的时间间隔,单位ms|否|1000| +|allReplace| true:新值替换旧值|否|false| +|updateMode| APPEND:不回撤数据,只下发增量数据,UPSERT:先删除回撤数据,然后更新|否|结果表设置主键则为UPSERT| ## 5.样例: + +回溯流删除 + ``` + +CREATE TABLE source1 ( + id int, + name VARCHAR +)WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest03', + timezone='Asia/Shanghai', + topicIsPattern ='false' + ); + + + +CREATE TABLE source2( + id int, + address VARCHAR +)WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest04', + timezone='Asia/Shanghai', + topicIsPattern ='false' +); + + CREATE TABLE MyResult( - channel VARCHAR, - pv VARCHAR - )WITH( - type ='db2', - url ='jdbc:db2://172.16.8.104:50000/test?charset=utf8', - userName ='dtstack', - password ='abc123', - tableName ='pv2', - parallelism ='1' - ) - ``` \ No newline at end of file + id int, + name VARCHAR, + address VARCHAR, + primary key (id) +)WITH( + type='db2', + url='jdbc:db2://172.16.10.251:50000/mqTest', + userName='DB2INST1', + password='abc123', + tableName='USER_INFO2', + schema = 'DTSTACK', + updateMode = 'upsert', // 设置 primary key则默认为upsert + batchSize = '1' +); + +insert into MyResult +select + s1.id, + s1.name, + s2.address +from + source1 s1 +left join + source2 s2 +on + s1.id = s2.id + + + + ``` + + + DB2结果表建表语句: + + ```aidl +CREATE TABLE "DTSTACK "."USER_INFO2" ( + "ID" INTEGER , + "NAME" VARCHAR(50 OCTETS) , + "ADDRESS" VARCHAR(50 OCTETS) ) + IN "USERSPACE1" + ORGANIZE BY ROW + ; + +GRANT CONTROL ON TABLE "DTSTACK "."USER_INFO2" TO USER "DB2INST1" +; +``` + + + +数据结果: + +向Topic mqTest03 发送数据 {"name":"maqi","id":1001} 插入 (1001,"maqi",null) + +向Topic mqTest04 发送数据 {"address":"hz","id":1001} 删除 (1001,"maqi",null) 插入 (1001,"maqi","hz") \ No newline at end of file diff --git a/docs/plugin/mysqlSink.md b/docs/plugin/mysqlSink.md index 254f11f2a..ffc62792c 100644 --- a/docs/plugin/mysqlSink.md +++ b/docs/plugin/mysqlSink.md @@ -3,7 +3,8 @@ CREATE TABLE tableName( colName colType, ... - colNameX colType + colNameX colType, + [primary key (colName)] )WITH( type ='mysql', url ='jdbcUrl', @@ -39,7 +40,7 @@ CREATE TABLE tableName( |batchSize | flush的大小|否|100| |batchWaitInterval | flush的时间间隔,单位ms|否|1000| |allReplace| true:新值替换旧值|否|false| -|updateMode| APPEND:不回撤数据,只下发增量数据,UPSERT:先删除回撤数据,然后更新|否|| +|updateMode| APPEND:不回撤数据,只下发增量数据,UPSERT:先删除回撤数据,然后更新|否|结果表设置主键则为UPSERT| ## 5.完整样例: ``` diff --git a/docs/plugin/postgresqlSink.md b/docs/plugin/postgresqlSink.md index c04ff0ec0..eb055cc04 100644 --- a/docs/plugin/postgresqlSink.md +++ b/docs/plugin/postgresqlSink.md @@ -43,7 +43,7 @@ CREATE TABLE tableName( |batchSize | flush的大小|否|100| |batchWaitInterval | flush的时间间隔,单位ms|否|1000| |allReplace| true:新值替换旧值|否|false| -|updateMode| APPEND:不回撤数据,只下发增量数据,UPSERT:先删除回撤数据,然后更新|否|| +|updateMode| APPEND:不回撤数据,只下发增量数据,UPSERT:先删除回撤数据,然后更新|否|结果表设置主键则为UPSERT| ## 5.样例: From ed3aa19336fbfd1ca68d5e3f9db25cfcc7df83a2 Mon Sep 17 00:00:00 2001 From: maqi Date: Sun, 26 Apr 2020 17:57:54 +0800 Subject: [PATCH 065/109] db2 side fix --- .../com/dtstack/flink/sql/side/db2/Db2AllSideInfo.java | 7 +++++++ .../com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java | 7 ++----- .../com/dtstack/flink/sql/side/db2/Db2AsyncSideInfo.java | 8 ++++++++ 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/db2/db2-side/db2-all-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AllSideInfo.java b/db2/db2-side/db2-all-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AllSideInfo.java index 282ee440c..b40cfd772 100644 --- a/db2/db2-side/db2-all-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AllSideInfo.java +++ b/db2/db2-side/db2-all-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AllSideInfo.java @@ -22,6 +22,8 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.flink.api.java.typeutils.RowTypeInfo; import java.util.List; @@ -38,4 +40,9 @@ public class Db2AllSideInfo extends RdbAllSideInfo { public Db2AllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } + + @Override + public String getTableName(RdbSideTableInfo rdbSideTableInfo) { + return DtStringUtil.getTableFullPath(rdbSideTableInfo.getSchema(), rdbSideTableInfo.getTableName()); + } } diff --git a/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java b/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java index cda2ed128..ed9fb025b 100644 --- a/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java +++ b/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java @@ -29,8 +29,6 @@ import io.vertx.ext.jdbc.JDBCClient; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.List; @@ -43,8 +41,7 @@ */ public class Db2AsyncReqRow extends RdbAsyncReqRow { - - private static final Logger LOG = LoggerFactory.getLogger(Db2AsyncReqRow.class); + private final static String DB2_PREFERRED_TEST_QUERY_SQL = "select 1 from sysibm.dual"; private final static String DB2_DRIVER = "com.ibm.db2.jcc.DB2Driver"; @@ -63,7 +60,7 @@ public void open(Configuration parameters) throws Exception { .put("user", rdbSideTableInfo.getUserName()) .put("password", rdbSideTableInfo.getPassword()) .put("provider_class", DT_PROVIDER_CLASS) - .put("preferred_test_query", PREFERRED_TEST_QUERY_SQL) + .put("preferred_test_query", DB2_PREFERRED_TEST_QUERY_SQL) .put("idle_connection_test_period", DEFAULT_IDLE_CONNECTION_TEST_PEROID) .put("test_connection_on_checkin", DEFAULT_TEST_CONNECTION_ON_CHECKIN); diff --git a/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncSideInfo.java b/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncSideInfo.java index e53dd7f27..dbb8230e7 100644 --- a/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncSideInfo.java +++ b/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncSideInfo.java @@ -22,6 +22,8 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.flink.api.java.typeutils.RowTypeInfo; import java.util.List; @@ -38,4 +40,10 @@ public class Db2AsyncSideInfo extends RdbAsyncSideInfo { public Db2AsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } + + @Override + public String getTableName(RdbSideTableInfo rdbSideTableInfo) { + return DtStringUtil.getTableFullPath(rdbSideTableInfo.getSchema(), rdbSideTableInfo.getTableName()); + } + } From 588071175fc04ea8daa9d183de0ebfbd339ad194 Mon Sep 17 00:00:00 2001 From: tiezhu Date: Mon, 27 Apr 2020 13:53:53 +0800 Subject: [PATCH 066/109] =?UTF-8?q?=E6=9B=B4=E6=96=B0cassandra-side?= =?UTF-8?q?=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/cassandraSide.md | 139 +++++++++++++++++++++++++++++------ 1 file changed, 115 insertions(+), 24 deletions(-) diff --git a/docs/plugin/cassandraSide.md b/docs/plugin/cassandraSide.md index f5210a900..5b8c955d5 100644 --- a/docs/plugin/cassandraSide.md +++ b/docs/plugin/cassandraSide.md @@ -1,5 +1,8 @@ ## 1.格式: + + 通过建表语句中的` PERIOD FOR SYSTEM_TIME`将表标识为维表,其中`PRIMARY KEY(keyInfo)`中的keyInfo,表示用来和源表进行关联的字段, + 维表JOIN的条件必须与`keyInfo`字段一致。 ``` CREATE TABLE tableName( colName cloType, @@ -22,15 +25,15 @@ ``` # 2.支持版本 - cassandra-3.6.x + cassandra-3.x ## 3.表结构定义 |参数名称|含义| |----|---| - | tableName | 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同)| + | tableName | 注册到flink的表名称(可选填;不填默认和cassandra对应的表名称相同)| | colName | 列名称| - | colType | 列类型 [colType支持的类型](docs/colType.md)| + | colType | 列类型| | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| @@ -46,40 +49,128 @@ | database | cassandra表名称|是|| | cache | 维表缓存策略(NONE/LRU)|否|NONE| | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| - | maxRequestsPerConnection | 每个连接最多允许64个并发请求|否|NONE| - | coreConnectionsPerHost | 和Cassandra集群里的每个机器都至少有2个连接|否|NONE| - | maxConnectionsPerHost | 和Cassandra集群里的每个机器都最多有6个连接|否|NONE| - | maxQueueSize | Cassandra队列大小|否|NONE| - | readTimeoutMillis | Cassandra读超时|否|NONE| - | connectTimeoutMillis | Cassandra连接超时|否|NONE| - | poolTimeoutMillis | Cassandra线程池超时|否|NONE| + | maxRequestsPerConnection | 每个连接最多允许64个并发请求|否|1| + | coreConnectionsPerHost | Cassandra集群里的每个机器都最少连接数|否|8| + | maxConnectionsPerHost | Cassandra集群里的每个机器都最多连接数|否|32768| + | maxQueueSize | Cassandra队列大小|否|100000| + | readTimeoutMillis | Cassandra读超时|否|60000| + | connectTimeoutMillis | Cassandra连接超时|否|60000| + | poolTimeoutMillis | Cassandra线程池超时|否|60000| ---------- > 缓存策略 - * NONE: 不做内存缓存 - * LRU: - * cacheSize: 缓存的条目数量 - * cacheTTLMs:缓存的过期时间(ms) +- NONE:不做内存缓存。每条流数据触发一次维表查询操作。 +- ALL: 任务启动时,一次性加载所有数据到内存,并进行缓存。适用于维表数据量较小的情况。 +- LRU: 任务执行时,根据维表关联条件使用异步算子加载维表数据,并进行缓存。 -## 5.样例 +## 5.维表定义样例 + +### ALL全量维表定义 ``` -create table sideTable( - CHANNEL varchar, - XCCOUNT int, - PRIMARY KEY(channel), +CREATE TABLE sideTable( + id bigint, + school varchar, + home varchar, + PRIMARY KEY(id), PERIOD FOR SYSTEM_TIME - )WITH( +)WITH( + type='mysql', + url='jdbc:mysql://172.16.8.109:3306/tiezhu', + userName='dtstack', + password='abc123', + tableName='stressTest', + cache='ALL', + parallelism='1' +); +``` +### LRU异步维表定义 +``` +CREATE TABLE sideTable( + id bigint, + message varchar, + PRIMARY KEY(id), + PERIOD FOR SYSTEM_TIME +)WITH( type ='cassandra', - address ='172.21.32.1:9042,172.21.32.1:9042', - database ='test', - tableName ='sidetest', + address ='192.168.80.106:9042, 192.168.80.107:9042', + database ='tiezhu', + tableName ='stu', + userName='cassandra', + password='cassandra', cache ='LRU', parallelism ='1', partitionedJoin='false' - ); +); +``` +## 6.完整样例 +``` +CREATE TABLE MyTable( + id bigint, + name varchar, + address varchar +)WITH( + type = 'kafka10', + bootstrapServers = '172.16.101.224:9092', + zookeeperQuorm = '172.16.100.188:2181/kafka', + offsetReset = 'latest', + topic = 'tiezhu_test_in2', + timezone = 'Asia/Shanghai', + topicIsPattern = 'false', + parallelism = '1' +); +CREATE TABLE sideTable( + id bigint, + message varchar, + PRIMARY KEY(id), + PERIOD FOR SYSTEM_TIME +)WITH( + type ='cassandra', + address ='192.168.80.106:9042, 192.168.80.107:9042', + database ='tiezhu', + tableName ='stu', + userName='cassandra', + password='cassandra', + cache ='LRU', + parallelism ='1', + partitionedJoin='false' +); + +CREATE TABLE MyResult( + id bigint, + name varchar, + address varchar, + message varchar + )WITH( + type ='cassandra', + address ='192.168.80.106:9042,192.168.80.107:9042', + userName='cassandra', + password='cassandra', + database ='tiezhu', + tableName ='stu_out', + parallelism ='1' +); +insert +into + MyResult + select + t1.id AS id, + t1.name AS name, + t1.address AS address, + t2.message AS message + from + ( + select + id, + name, + address + from + MyTable + ) t1 + join sideTable t2 + on t1.id = t2.id; ``` From ab67cfb197b762dc9f3740abc4dd28e546bd4875 Mon Sep 17 00:00:00 2001 From: tiezhu Date: Mon, 27 Apr 2020 13:56:25 +0800 Subject: [PATCH 067/109] =?UTF-8?q?=E6=9B=B4=E6=96=B0cassandra-side?= =?UTF-8?q?=E6=96=87=E6=A1=A3=E4=B8=AD=E5=8F=82=E6=95=B0=E8=AF=B4=E6=98=8E?= =?UTF-8?q?=E5=86=85=E5=AE=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/cassandraSide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/plugin/cassandraSide.md b/docs/plugin/cassandraSide.md index 5b8c955d5..048caed62 100644 --- a/docs/plugin/cassandraSide.md +++ b/docs/plugin/cassandraSide.md @@ -49,8 +49,8 @@ | database | cassandra表名称|是|| | cache | 维表缓存策略(NONE/LRU)|否|NONE| | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| - | maxRequestsPerConnection | 每个连接最多允许64个并发请求|否|1| - | coreConnectionsPerHost | Cassandra集群里的每个机器都最少连接数|否|8| + | maxRequestsPerConnection | 每个连接允许的并发请求数|否|1| + | coreConnectionsPerHost | 每台主机连接的核心数|否|8| | maxConnectionsPerHost | Cassandra集群里的每个机器都最多连接数|否|32768| | maxQueueSize | Cassandra队列大小|否|100000| | readTimeoutMillis | Cassandra读超时|否|60000| From cdc91af4738687d173ba715a2a6e835e7b4bd20f Mon Sep 17 00:00:00 2001 From: tiezhu Date: Mon, 27 Apr 2020 14:03:25 +0800 Subject: [PATCH 068/109] =?UTF-8?q?=E6=9B=B4=E6=96=B0cassandra-sink?= =?UTF-8?q?=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/cassandraSink.md | 125 +++++++++++++++++++++++++++++------ 1 file changed, 106 insertions(+), 19 deletions(-) diff --git a/docs/plugin/cassandraSink.md b/docs/plugin/cassandraSink.md index 6d6dd2301..8702acb7a 100644 --- a/docs/plugin/cassandraSink.md +++ b/docs/plugin/cassandraSink.md @@ -17,7 +17,7 @@ CREATE TABLE tableName( ``` ## 2.支持版本 - cassandra-3.6.x + cassandra-3.x ## 3.表结构定义 @@ -27,7 +27,7 @@ CREATE TABLE tableName( | colName | 列名称| | colType | 列类型 [colType支持的类型](docs/colType.md)| -## 4.参数: +## 4.参数 |参数名称|含义|是否必填|默认值| |----|----|----|----| @@ -38,26 +38,113 @@ CREATE TABLE tableName( |tableName | cassandra表名称|是|| |database | cassandra表名称|是|| |parallelism | 并行度设置|否|1| -|maxRequestsPerConnection | 每个连接最多允许64个并发请求|否|NONE| -|coreConnectionsPerHost | 和Cassandra集群里的每个机器都至少有2个连接|否|NONE| -|maxConnectionsPerHost | 和Cassandra集群里的每个机器都最多有6个连接|否|NONE| -|maxQueueSize | Cassandra队列大小|否|NONE| -|readTimeoutMillis | Cassandra读超时|否|NONE| -|connectTimeoutMillis | Cassandra连接超时|否|NONE| -|poolTimeoutMillis | Cassandra线程池超时|否|NONE| +| maxRequestsPerConnection | 每个连接允许的并发请求数|否|1| +| coreConnectionsPerHost | 每台主机连接的核心数|否|8| +| maxConnectionsPerHost | Cassandra集群里的每个机器都最多连接数|否|32768| +| maxQueueSize | Cassandra队列大小|否|100000| +| readTimeoutMillis | Cassandra读超时|否|60000| +| connectTimeoutMillis | Cassandra连接超时|否|60000| +| poolTimeoutMillis | Cassandra线程池超时|否|60000| -## 5.样例: +## 5.完整样例: ``` +CREATE TABLE MyTable( + id bigint, + name varchar, + address varchar +)WITH( + type = 'kafka10', + bootstrapServers = '172.16.101.224:9092', + zookeeperQuorm = '172.16.100.188:2181/kafka', + offsetReset = 'latest', + topic = 'tiezhu_test_in2', + timezone = 'Asia/Shanghai', + topicIsPattern = 'false', + parallelism = '1' +); + +CREATE TABLE sideTable( + id bigint, + message varchar, + PRIMARY KEY(id), + PERIOD FOR SYSTEM_TIME +)WITH( + type ='cassandra', + address ='192.168.80.106:9042, 192.168.80.107:9042', + database ='tiezhu', + tableName ='stu', + userName='cassandra', + password='cassandra', + cache ='LRU', + parallelism ='1', + partitionedJoin='false' +); + CREATE TABLE MyResult( - channel VARCHAR, - pv VARCHAR + id bigint, + name varchar, + address varchar, + message varchar )WITH( type ='cassandra', - address ='172.21.32.1:9042,172.21.32.1:9042', - userName ='dtstack', - password ='abc123', - database ='test', - tableName ='pv', + address ='192.168.80.106:9042,192.168.80.107:9042', + userName='cassandra', + password='cassandra', + database ='tiezhu', + tableName ='stu_out', parallelism ='1' - ) - ``` \ No newline at end of file +); + +insert +into + MyResult + select + t1.id AS id, + t1.name AS name, + t1.address AS address, + t2.message AS message + from + ( + select + id, + name, + address + from + MyTable + ) t1 + join sideTable t2 + on t1.id = t2.id; + ``` +### 6.结果表数据展示 +``` +cqlsh:tiezhu> desc stu_out + +CREATE TABLE tiezhu.stu_out ( + id int PRIMARY KEY, + address text, + message text, + name text +) WITH bloom_filter_fp_chance = 0.01 + AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'} + AND comment = '' + AND compaction = {'class': 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 'max_threshold': '32', 'min_threshold': '4'} + AND compression = {'chunk_length_in_kb': '64', 'class': 'org.apache.cassandra.io.compress.LZ4Compressor'} + AND crc_check_chance = 1.0 + AND dclocal_read_repair_chance = 0.1 + AND default_time_to_live = 0 + AND gc_grace_seconds = 864000 + AND max_index_interval = 2048 + AND memtable_flush_period_in_ms = 0 + AND min_index_interval = 128 + AND read_repair_chance = 0.0 + AND speculative_retry = '99PERCENTILE'; + + +cqlsh:tiezhu> select * from stu_out limit 1; + + id | address | message | name +----+------------+------------------+---------- + 23 | hangzhou23 | flinkStreamSql23 | tiezhu23 + +(1 rows) +``` \ No newline at end of file From ccd5edefa8ba01b640159201dd833cf8ae78fcf2 Mon Sep 17 00:00:00 2001 From: tiezhu Date: Mon, 27 Apr 2020 14:39:06 +0800 Subject: [PATCH 069/109] =?UTF-8?q?=E6=B7=BB=E5=8A=A0kafka-sink=E6=96=87?= =?UTF-8?q?=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/kafkaSink.md | 152 +++++++++++++++++++++++++++++++++++++++ pom.xml | 32 ++++----- 2 files changed, 168 insertions(+), 16 deletions(-) create mode 100644 docs/plugin/kafkaSink.md diff --git a/docs/plugin/kafkaSink.md b/docs/plugin/kafkaSink.md new file mode 100644 index 000000000..8b50c9e3f --- /dev/null +++ b/docs/plugin/kafkaSink.md @@ -0,0 +1,152 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + ... + colNameX colType, + [primary key (colName)] + )WITH( + type ='kafka09', + bootstrapServers ='ip:port,ip:port...', + zookeeperQuorum ='ip:port,ip:port/zkparent', + offsetReset ='latest', + topic ='topicName', + groupId='test', + parallelism ='parllNum', + timezone='Asia/Shanghai', + sourcedatatype ='json' #可不设置 + ); +``` + +## 2.支持版本 + kafka09,kafka10,kafka11及以上版本 + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName| 结果表名称| +| colName | 列名称| +| colType | 列类型 [colType支持的类型](docs/colType.md)| + +## 4.参数: + +|参数名称|含义|是否必填|默认值| +|----|----|----|----| +|type |表名的输出表类型[kafka09|kafka10|kafka11|kafka]|是|| +|groupId | 需要读取的 groupId 名称|否|| +|bootstrapServers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| +|zookeeperQuorum | kafka zk地址信息(多个之间用逗号分隔)|是|| +|topic | 需要读取的 topic 名称|是|| +|topicIsPattern | topic是否是正则表达式格式(true|false) |否| false +|offsetReset | 读取的topic 的offset初始位置[latest|earliest|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| +|parallelism | 并行度设置|否|1| +|sourcedatatype | 数据类型|否|json| +|timezone|时区设置[timezone支持的参数](../timeZone.md)|否|'Asia/Shanghai' +**kafka相关参数可以自定义,使用kafka.开头即可。** +``` +kafka.consumer.id +kafka.socket.timeout.ms +kafka.fetch.message.max.bytes +kafka.num.consumer.fetchers +kafka.auto.commit.enable +kafka.auto.commit.interval.ms +kafka.queued.max.message.chunks +kafka.rebalance.max.retries +kafka.fetch.min.bytes +kafka.fetch.wait.max.ms +kafka.rebalance.backoff.ms +kafka.refresh.leader.backoff.ms +kafka.consumer.timeout.ms +kafka.exclude.internal.topics +kafka.partition.assignment.strategy +kafka.client.id +kafka.zookeeper.session.timeout.ms +kafka.zookeeper.connection.timeout.ms +kafka.zookeeper.sync.time.ms +kafka.offsets.storage +kafka.offsets.channel.backoff.ms +kafka.offsets.channel.socket.timeout.ms +kafka.offsets.commit.max.retries +kafka.dual.commit.enabled +kafka.partition.assignment.strategy +kafka.socket.receive.buffer.bytes +kafka.fetch.min.bytes +``` + +## 5.完整样例: +``` +CREATE TABLE MyTable( + id bigint, + name varchar, + address varchar +)WITH( + type = 'kafka10', + bootstrapServers = '172.16.101.224:9092', + zookeeperQuorm = '172.16.100.188:2181/kafka', + offsetReset = 'latest', + topic = 'tiezhu_test_in2', + groupId = 'flink_sql', + timezone = 'Asia/Shanghai', + topicIsPattern = 'false', + parallelism = '1' +); + +CREATE TABLE sideTable( + id bigint, + school varchar, + home varchar, + PRIMARY KEY(id), + PERIOD FOR SYSTEM_TIME +)WITH( + type='mysql', + url='jdbc:mysql://172.16.8.109:3306/tiezhu', + userName='dtstack', + password='abc123', + tableName='stressTest', + cache='ALL', + parallelism='1' +); + +CREATE TABLE MyResult( + id bigint, + name varchar, + address varchar, + home varchar, + school varchar +)WITH( + type = 'kafka10', + bootstrapServers = '172.16.101.224:9092', + zookeeperQuorm = '172.16.100.188:2181/kafka', + offsetReset = 'latest', + topic = 'tiezhu_test_out2', + parallelism = '1' +); + +insert +into + MyResult + select + t1.id AS id, + t1.name AS name, + t1.address AS address, + t2.school AS school, + t2.home AS home + from( + select + id, + name, + address + from + MyTable + ) t1 + join sideTable t2 + on t1.id = t2.id; + ``` + +## 6.结果表数据示例: +``` +[root@node002 bin]# ./kafka-console-consumer.sh --bootstrap-server 172.16.101.224:9092 --topic tiezhu_test_out2 +{"id":122,"name":"tiezhu122","address":"hangzhou122","home":"ganzhou122","school":" ecjtu122"} +{"id":123,"name":"tiezhu123","address":"hangzhou123","home":"ganzhou123","school":" ecjtu123"} +``` \ No newline at end of file diff --git a/pom.xml b/pom.xml index 2794f9004..b5b674379 100644 --- a/pom.xml +++ b/pom.xml @@ -16,24 +16,24 @@ kafka11 kafka mysql - hbase - elasticsearch5 - mongo - redis5 + + + + launcher rdb - sqlserver - oracle - cassandra - kudu - postgresql - serversocket - console - clickhouse - impala - db2 - polardb - elasticsearch6 + + + + + + + + + + + + From eb7502a01b1c5c2a8d9633e851f7008afbb0e743 Mon Sep 17 00:00:00 2001 From: gkd Date: Mon, 27 Apr 2020 16:18:02 +0800 Subject: [PATCH 070/109] =?UTF-8?q?=E4=BC=98=E5=8C=96=E6=96=87=E6=A1=A3db2?= =?UTF-8?q?side=20impalaside=20sqlserverside?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/db2Side.md | 141 +++++++++++++++++++++------- docs/plugin/impalaSide.md | 173 +++++++++++++++++++++++++---------- docs/plugin/sqlserverSide.md | 146 ++++++++++++++++++++++------- 3 files changed, 344 insertions(+), 116 deletions(-) diff --git a/docs/plugin/db2Side.md b/docs/plugin/db2Side.md index 8381d6206..88f0bfc0e 100644 --- a/docs/plugin/db2Side.md +++ b/docs/plugin/db2Side.md @@ -1,6 +1,9 @@ - ## 1.格式: -``` + +通过建表语句中的` PERIOD FOR SYSTEM_TIME`将表标识为维表,其中`PRIMARY KEY(keyInfo)`中的keyInfo,表示用来和源表进行关联的字段, + 维表JOIN的条件必须与`keyInfo`字段一致。 + +```sql CREATE TABLE tableName( colName cloType, ... @@ -20,41 +23,48 @@ ); ``` -# 2.支持版本 +## 2.支持版本 + db2 9.X - + ## 3.表结构定义 - - |参数名称|含义| - |----|---| - | tableName | db2表名称| - | colName | 列名称| - | colType | 列类型 [colType支持的类型](docs/colType.md)| - | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| - | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| - + + [维表参数信息](docs/plugin/sideParams.md) + +db2独有的参数 + +| 参数名称 | 含义 | 是否必填 | 默认值 | +| -------- | ------------------ | -------- | ------ | +| type | 维表类型, db2 | 是 | | +| url | 连接数据库 jdbcUrl | 是 | | +| userName | 连接用户名 | 是 | | +| password | 连接密码 | 是 | | + ## 4.参数 - |参数名称|含义|是否必填|默认值| - |----|---|---|----| - | type | 表明维表的类型 db2 |是|| - | url | 连接mysql数据库 jdbcUrl |是|| - | userName | db2连接用户名 |是|| - | password | db2连接密码|是|| - | tableName | db2表名称|是|| - | cache | 维表缓存策略(NONE/LRU)|否|NONE| - | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| - - ---------- - > 缓存策略 - * NONE: 不做内存缓存 - * LRU: - * cacheSize: 缓存的条目数量 - * cacheTTLMs:缓存的过期时间(ms) - - -## 5.样例 +### ALL全量维表定义 + +```sql +// 定义全量维表 +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , + PERIOD FOR SYSTEM_TIME + )WITH( + type ='db2', + url ='jdbc:db2://172.16.8.104:50000/test?charset=utf8', + userName ='dtstack', + password ='abc123', + tableName ='all_test_db2', + cache ='ALL', + cacheTTLMs ='60000', + parallelism ='1' + ); ``` +### LRU异步维表定义 + +```sql create table sideTable( channel varchar, xccount int, @@ -65,15 +75,78 @@ create table sideTable( url='jdbc:db2://172.16.8.104:50000/test?charset=utf8', userName='dtstack', password='abc123', - tableName='sidetest', + tableName='lru_test_db2', cache ='LRU', cacheSize ='10000', cacheTTLMs ='60000', parallelism ='1', partitionedJoin='false' ); +``` +### Db2异步维表关联 -``` +```sql +CREATE TABLE MyTable( + id int, + name varchar + )WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='cannan_yctest01', + timezone='Asia/Shanghai', + enableKeyPartitions ='false', + topicIsPattern ='false', + parallelism ='1' + ); +CREATE TABLE MyResult( + id INT, + name VARCHAR + )WITH( + type='db2', + url='jdbc:db2://172.16.8.104:50000/test?charset=utf8', + userName='dtstack', + password='abc123', + tableName ='test_db2_zf', + updateMode ='append', + parallelism ='1', + batchSize ='100', + batchWaitInterval ='1000' + ); + +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , + PERIOD FOR SYSTEM_TIME + )WITH( + type='db2', + url='jdbc:db2://172.16.8.104:50000/test?charset=utf8', + userName='dtstack', + password='abc123', + tableName ='test_db2_10', + partitionedJoin ='false', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + asyncPoolSize ='3', + parallelism ='1' + ); + +insert +into + MyResult + select + m.id, + s.name + from + MyTable m + join + sideTable s + on m.id=s.id; + +``` diff --git a/docs/plugin/impalaSide.md b/docs/plugin/impalaSide.md index 5c7479b04..500219b7f 100644 --- a/docs/plugin/impalaSide.md +++ b/docs/plugin/impalaSide.md @@ -1,6 +1,6 @@ - ## 1.格式: -``` + +```sql CREATE TABLE tableName( colName cloType, ... @@ -20,51 +20,65 @@ ); ``` -# 2.支持版本 +## 2.支持版本 + 2.10.0-cdh5.13.0 - + ## 3.表结构定义 - - |参数名称|含义| - |----|---| - | tableName | 注册到flink的表名称| - | colName | 列名称| - | colType | 列类型 [colType支持的类型](docs/colType.md)| - | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| - | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| - -## 4.参数 - - |参数名称|含义|是否必填|默认值| - |----|---|---|----| - | type | 表明维表的类型[impala] |是|| - | url | 连接postgresql数据库 jdbcUrl |是|| - | userName | postgresql连接用户名 |是|| - | password | postgresql连接密码|是|| - | tableName | postgresql表名称|是|| - | authMech | 身份验证机制 (0, 1, 2, 3), 暂不支持kerberos |是|0| - | principal | kerberos用于登录的principal(authMech=1时独有) |authMech=1为必填| - | keyTabFilePath | keytab文件的路径(authMech=1时独有) |authMech=1为必填 || - | krb5FilePath | krb5.conf文件路径(authMech=1时独有) |authMech=1为必填|| - | krbServiceName | Impala服务器的Kerberos principal名称(authMech=1时独有) |authMech=1为必填|| - | krbRealm | Kerberos的域名(authMech=1时独有) |否| HADOOP.COM | - | enablePartition | 是否支持分区|否|false| - | partitionfields | 分区字段名|否,enablePartition='true'时为必填|| - | partitionFieldTypes | 分区字段类型 |否,enablePartition='true'时为必填|| - | partitionValues | 分区值|否,enablePartition='true'时为必填|| - | cache | 维表缓存策略(NONE/LRU/ALL)|否|NONE| - | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| - - ---------- - > 缓存策略 - * NONE: 不做内存缓存 - * LRU: - * cacheSize: 缓存的条目数量 - * cacheTTLMs:缓存的过期时间(ms) - - -## 5.样例 + + [维表参数信息](docs/plugin/sideParams.md) + +impala独有的参数配置 + +| 参数名称 | 含义 | 是否必填 | 默认值 | +| ------------------- | ------------------------------------------------------------ | --------------------------------- | ---------- | +| type | 表明维表的类型[impala] | 是 | | +| url | 连接postgresql数据库 jdbcUrl | 是 | | +| userName | postgresql连接用户名 | 是 | | +| password | postgresql连接密码 | 是 | | +| tableName | postgresql表名称 | 是 | | +| authMech | 身份验证机制 (0, 1, 2, 3), 暂不支持kerberos | 是 | 0 | +| principal | kerberos用于登录的principal(authMech=1时独有) | authMech=1为必填 | | +| keyTabFilePath | keytab文件的路径(authMech=1时独有) | authMech=1为必填 | | +| krb5FilePath | krb5.conf文件路径(authMech=1时独有) | authMech=1为必填 | | +| krbServiceName | Impala服务器的Kerberos principal名称(authMech=1时独有) | authMech=1为必填 | | +| krbRealm | Kerberos的域名(authMech=1时独有) | 否 | HADOOP.COM | +| enablePartition | 是否支持分区 | 否 | false | +| partitionfields | 分区字段名 | 否,enablePartition='true'时为必填 | | +| partitionFieldTypes | 分区字段类型 | 否,enablePartition='true'时为必填 | | +| partitionValues | 分区值 | 否,enablePartition='true'时为必填 | | +| cache | 维表缓存策略(NONE/LRU/ALL) | 否 | NONE | +| partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量) | 否 | false | + +## 4.样例 + +### ALL全量维表定义 + +```sql + // 定义全量维表 +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , + PERIOD FOR SYSTEM_TIME + )WITH( + type ='mysql', + url ='jdbc:impala://localhost:21050/mqtest', + userName ='dtstack', + password ='1abc123', + tableName ='test_impala_all', + authMech='3', + cache ='ALL', + cacheTTLMs ='60000', + parallelism ='2', + partitionedJoin='false' + ); + ``` + +### LRU异步维表定义 + +```sql create table sideTable( channel varchar, xccount int, @@ -87,11 +101,77 @@ create table sideTable( ``` -## 6.分区样例 +### MySQL异步维表关联 -注:分区字段放在最后面,如下,name是分区字段,放在channel,xccount字段的后面 +```sql +CREATE TABLE MyTable( + id int, + name varchar + )WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='cannan_yctest01', + timezone='Asia/Shanghai', + enableKeyPartitions ='false', + topicIsPattern ='false', + parallelism ='1' + ); + +CREATE TABLE MyResult( + id INT, + name VARCHAR + )WITH( + type='impala', + url='jdbc:impala://localhost:21050/mytest', + userName='dtstack', + password='abc123', + tableName ='test_impala_zf', + updateMode ='append', + parallelism ='1', + batchSize ='100', + batchWaitInterval ='1000' + ); + +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , + PERIOD FOR SYSTEM_TIME + )WITH( + type='impala', + url='jdbc:impala://localhost:21050/mytest', + userName='dtstack', + password='abc123', + tableName ='test_impala_10', + partitionedJoin ='false', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + asyncPoolSize ='3', + parallelism ='1' + ); + +insert +into + MyResult + select + m.id, + s.name + from + MyTable m + join + sideTable s + on m.id=s.id; ``` + +### 分区样例 + +注:分区字段放在最后面,如下,name是分区字段,放在channel,xccount字段的后面 + +```sql create table sideTable( channel varchar, xccount int, @@ -118,4 +198,3 @@ create table sideTable( ``` - diff --git a/docs/plugin/sqlserverSide.md b/docs/plugin/sqlserverSide.md index 1cab43cbd..4d50ed619 100644 --- a/docs/plugin/sqlserverSide.md +++ b/docs/plugin/sqlserverSide.md @@ -1,6 +1,9 @@ - ## 1.格式: -``` + +通过建表语句中的` PERIOD FOR SYSTEM_TIME`将表标识为维表,其中`PRIMARY KEY(keyInfo)`中的keyInfo,表示用来和源表进行关联的字段, + 维表JOIN的条件必须与`keyInfo`字段一致。 + +```sql CREATE TABLE tableName( colName cloType, ... @@ -20,41 +23,49 @@ ); ``` -# 2.支持版本 +## 2.支持版本 + mysql-5.6.35 - + ## 3.表结构定义 - - |参数名称|含义| - |----|---| - | tableName | sqlserver表名称| - | colName | 列名称| - | colType | 列类型 [colType支持的类型](docs/colType.md)| - | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| - | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| - -## 4.参数 - - |参数名称|含义|是否必填|默认值| - |----|---|---|----| - | type | 表明维表的类型 sqlserver |是|| - | url | 连接mysql数据库 jdbcUrl |是|| - | userName | sqlserver连接用户名 |是|| - | password | sqlserver连接密码|是|| - | tableName | sqlserver表名称|是|| - | tableName | sqlserver 的表名称|是|| - | cache | 维表缓存策略(NONE/LRU)|否|NONE| - | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| - - ---------- - > 缓存策略 - * NONE: 不做内存缓存 - * LRU: - * cacheSize: 缓存的条目数量 - * cacheTTLMs:缓存的过期时间(ms) - -## 5.样例 + + [维表参数信息](docs/plugin/sideParams.md) + +sqlserver独有的参数配置 + +| 参数名称 | 含义 | 是否必填 | 默认值 | +| -------- | -------------------- | -------- | ------ | +| type | 维表类型, sqlserver | 是 | | +| url | 连接数据库 jdbcUrl | 是 | | +| userName | 连接用户名 | 是 | | +| password | 连接密码 | 是 | | + +## 4.样例 + +### ALL全量维表定义 + +```sql + // 定义全量维表 +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , + PERIOD FOR SYSTEM_TIME + )WITH( + type ='sqlserver', + url ='jdbc:jtds:sqlserver://172.16.8.104:1433;DatabaseName=mytest', + userName ='dtstack', + password ='abc123', + tableName ='all_test_sqlserver', + cache ='ALL', + cacheTTLMs ='60000', + parallelism ='1' + ); ``` + +### LRU异步维表定义 + +```sql create table sideTable( channel varchar, xccount int, @@ -65,7 +76,7 @@ create table sideTable( url='jdbc:jtds:sqlserver://172.16.8.104:1433;DatabaseName=mytest', userName='dtstack', password='abc123', - tableName='sidetest', + tableName=',ru_test_sqlserver', cache ='LRU', cacheSize ='10000', cacheTTLMs ='60000', @@ -76,4 +87,69 @@ create table sideTable( ``` +### PostSQL异步维表关联 + +```sql +CREATE TABLE MyTable( + id int, + name varchar + )WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='cannan_yctest01', + timezone='Asia/Shanghai', + enableKeyPartitions ='false', + topicIsPattern ='false', + parallelism ='1' + ); + +CREATE TABLE MyResult( + id INT, + name VARCHAR + )WITH( + type='sqlserver', + url='jdbc:jtds:sqlserver://172.16.8.104:1433;DatabaseName=mytest', + userName='dtstack', + password='abc123', + tableName ='test_sqlserver_zf', + updateMode ='append', + parallelism ='1', + batchSize ='100', + batchWaitInterval ='1000' + ); + +CREATE TABLE sideTable( + id INT, + name VARCHAR, + PRIMARY KEY(id) , + PERIOD FOR SYSTEM_TIME + )WITH( + type='sqlserver', + url='jdbc:jtds:sqlserver://172.16.8.104:1433;DatabaseName=mytest', + userName='dtstack', + password='abc123', + tableName ='test_sqlserver_10', + partitionedJoin ='false', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + asyncPoolSize ='3', + parallelism ='1' + ); + +insert +into + MyResult + select + m.id, + s.name + from + MyTable m + join + sideTable s + on m.id=s.id; + +``` From 4e34155b0cdecccec1e5ff9a6fcbe88163b6e38d Mon Sep 17 00:00:00 2001 From: gkd Date: Mon, 27 Apr 2020 16:28:04 +0800 Subject: [PATCH 071/109] =?UTF-8?q?=E9=87=8D=E6=9E=84=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/db2Side.md | 115 +---------------------------------- docs/plugin/impalaSide.md | 1 - docs/plugin/sqlserverSide.md | 1 + 3 files changed, 4 insertions(+), 113 deletions(-) diff --git a/docs/plugin/db2Side.md b/docs/plugin/db2Side.md index 4fc7541ec..af0c29c02 100644 --- a/docs/plugin/db2Side.md +++ b/docs/plugin/db2Side.md @@ -1,9 +1,5 @@ -## 1.格式: - -通过建表语句中的` PERIOD FOR SYSTEM_TIME`将表标识为维表,其中`PRIMARY KEY(keyInfo)`中的keyInfo,表示用来和源表进行关联的字段, - 维表JOIN的条件必须与`keyInfo`字段一致。 -```sql +## 1.格式: 通过建表语句中的` PERIOD FOR SYSTEM_TIME`将表标识为维表,其中`PRIMARY KEY(keyInfo)`中的keyInfo,表示用来和源表进行关联的字段, 维表JOIN的条件必须与`keyInfo`字段一致。 ``` @@ -26,24 +22,10 @@ ); ``` -## 2.支持版本 - +# 2.支持版本 db2 9.X - + ## 3.表结构定义 - - [维表参数信息](docs/plugin/sideParams.md) - -db2独有的参数 - -| 参数名称 | 含义 | 是否必填 | 默认值 | -| -------- | ------------------ | -------- | ------ | -| type | 维表类型, db2 | 是 | | -| url | 连接数据库 jdbcUrl | 是 | | -| userName | 连接用户名 | 是 | | -| password | 连接密码 | 是 | | - -## 4.参数 [维表参数信息](docs/plugin/sideParams.md) @@ -58,35 +40,9 @@ db2独有的参数 | schema | 表所属scheam|否|| -### ALL全量维表定义 - -```sql -// 定义全量维表 -CREATE TABLE sideTable( - id INT, - name VARCHAR, - PRIMARY KEY(id) , - PERIOD FOR SYSTEM_TIME - )WITH( - type ='db2', - url ='jdbc:db2://172.16.8.104:50000/test?charset=utf8', - userName ='dtstack', - password ='abc123', - tableName ='all_test_db2', - cache ='ALL', - cacheTTLMs ='60000', - parallelism ='1' - ); -``` -### LRU异步维表定义 ## 4.样例 -```sql -create table sideTable( - channel varchar, - xccount int, - PRIMARY KEY(channel), ### ALL全量维表定义 ``` // 定义全量维表 @@ -100,7 +56,6 @@ CREATE TABLE sideTable( url='jdbc:db2://172.16.10.251:50000/mqTest', userName='DB2INST1', password='abc123', - tableName='lru_test_db2', tableName='USER_INFO2', schema = 'DTSTACK' cache ='ALL', @@ -131,7 +86,6 @@ CREATE TABLE sideTable( asyncPoolSize ='3', parallelism ='2' ); -``` ``` @@ -168,57 +122,7 @@ CREATE TABLE source2( batchSize = '1' ); -### Db2异步维表关联 - -```sql -CREATE TABLE MyTable( - id int, - name varchar - )WITH( - type ='kafka11', - bootstrapServers ='172.16.8.107:9092', - zookeeperQuorum ='172.16.8.107:2181/kafka', - offsetReset ='latest', - topic ='cannan_yctest01', - timezone='Asia/Shanghai', - enableKeyPartitions ='false', - topicIsPattern ='false', - parallelism ='1' - ); - -CREATE TABLE MyResult( - id INT, - name VARCHAR - )WITH( - type='db2', - url='jdbc:db2://172.16.8.104:50000/test?charset=utf8', - userName='dtstack', - password='abc123', - tableName ='test_db2_zf', - updateMode ='append', - parallelism ='1', - batchSize ='100', - batchWaitInterval ='1000' - ); -CREATE TABLE sideTable( - id INT, - name VARCHAR, - PRIMARY KEY(id) , - PERIOD FOR SYSTEM_TIME - )WITH( - type='db2', - url='jdbc:db2://172.16.8.104:50000/test?charset=utf8', - userName='dtstack', - password='abc123', - tableName ='test_db2_10', - partitionedJoin ='false', - cache ='LRU', - cacheSize ='10000', - cacheTTLMs ='60000', - asyncPoolSize ='3', - parallelism ='1' - ); CREATE TABLE MyResult( id int, name VARCHAR, @@ -259,19 +163,6 @@ GRANT CONTROL ON TABLE "DTSTACK "."USER_INFO2" TO USER "DB2INST1" ; ``` -insert -into - MyResult - select - m.id, - s.name - from - MyTable m - join - sideTable s - on m.id=s.id; - -``` 维表数据:(1001,maqi,hz) 源表数据:{"name":"maqi","id":1001} diff --git a/docs/plugin/impalaSide.md b/docs/plugin/impalaSide.md index 500219b7f..dce959388 100644 --- a/docs/plugin/impalaSide.md +++ b/docs/plugin/impalaSide.md @@ -98,7 +98,6 @@ create table sideTable( partitionedJoin='false' ); - ``` ### MySQL异步维表关联 diff --git a/docs/plugin/sqlserverSide.md b/docs/plugin/sqlserverSide.md index 4d50ed619..f95d8bba5 100644 --- a/docs/plugin/sqlserverSide.md +++ b/docs/plugin/sqlserverSide.md @@ -63,6 +63,7 @@ CREATE TABLE sideTable( ); ``` + ### LRU异步维表定义 ```sql From d6f72c937960e0047ea03fcdbc07b5d7fc459aa1 Mon Sep 17 00:00:00 2001 From: maqi Date: Mon, 27 Apr 2020 16:51:49 +0800 Subject: [PATCH 072/109] mongo doc --- docs/plugin/mongoSide.md | 142 +++++++++++++++++++++++++++++---------- docs/plugin/mongoSink.md | 75 ++++++++++++++++++--- 2 files changed, 170 insertions(+), 47 deletions(-) diff --git a/docs/plugin/mongoSide.md b/docs/plugin/mongoSide.md index 90ffa2582..92362a752 100644 --- a/docs/plugin/mongoSide.md +++ b/docs/plugin/mongoSide.md @@ -26,53 +26,123 @@ ## 3.表结构定义 - |参数名称|含义| - |----|---| - | tableName | 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同)| - | colName | 列名称| - | colType | 列类型 [colType支持的类型](docs/colType.md)| - | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| - | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| + [通用维表参数信息](docs/plugin/sideParams.md) -## 4.参数 - - |参数名称|含义|是否必填|默认值| - |----|---|---|----| - | type |表明 输出表类型 mongo|是|| - | address | 连接mongo数据库 jdbcUrl |是|| - | userName | mongo连接用户名|否|| - | password | mongo连接密码|否|| - | tableName | mongo表名称|是|| - | database | mongo表名称|是|| - | cache | 维表缓存策略(NONE/LRU)|否|NONE| - | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| - - ---------- - > 缓存策略 - * NONE: 不做内存缓存 - * LRU: - * cacheSize: 缓存的条目数量 - * cacheTTLMs:缓存的过期时间(ms) - + + mongo相关参数配置: + +|参数名称|含义|是否必填|默认值| +|----|---|---|----| +| type |表明 输出表类型 mongo|是|| +| address | 连接mongo数据库 jdbcUrl |是|| +| userName | mongo连接用户名|否|| +| password | mongo连接密码|否|| +| tableName | mongo表名称|是|| +| database | mongo表名称|是|| + +## 4.样例 -## 5.样例 + +### 全量维表结构 + +```aidl +CREATE TABLE source2( + id int, + address VARCHAR, + PERIOD FOR SYSTEM_TIME +)WITH( + type ='mongo', + address ='172.16.8.193:27017', + database ='dtstack', + tableName ='userInfo', + cache ='ALL', + parallelism ='1', + partitionedJoin='false' +); ``` -create table sideTable( - CHANNEL varchar, - XCCOUNT int, - PRIMARY KEY(channel), + +### 异步维表结构 + +```aidl +CREATE TABLE source2( + id int, + address VARCHAR, PERIOD FOR SYSTEM_TIME - )WITH( +)WITH( type ='mongo', - address ='172.21.32.1:27017,172.21.32.1:27017', - database ='test', - tableName ='sidetest', + address ='172.16.8.193:27017', + database ='dtstack', + tableName ='userInfo', cache ='LRU', parallelism ='1', partitionedJoin='false' +); + +``` + +### 异步维表关联样例 + +``` + +CREATE TABLE source1 ( + id int, + name VARCHAR +)WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest03', + timezone='Asia/Shanghai', + topicIsPattern ='false' ); +CREATE TABLE source2( + id int, + address VARCHAR, + PERIOD FOR SYSTEM_TIME +)WITH( + type ='mongo', + address ='172.16.8.193:27017', + database ='dtstack', + tableName ='userInfo', + cache ='ALL', + parallelism ='1', + partitionedJoin='false' +); + + +CREATE TABLE MyResult( + id int, + name VARCHAR, + address VARCHAR, + primary key (id) +)WITH( + type='console' +); + +insert into MyResult +select + s1.id, + s1.name, + s2.address +from + source1 s1 +left join + source2 s2 +on + s1.id = s2.id + + ``` +维表数据:{"id": 1001,"address":"hz""} + +源表数据:{"name":"maqi","id":1001} + + +输出结果: (1001,maqi,hz) + + diff --git a/docs/plugin/mongoSink.md b/docs/plugin/mongoSink.md index 9900dea12..8c32dc95e 100644 --- a/docs/plugin/mongoSink.md +++ b/docs/plugin/mongoSink.md @@ -40,17 +40,70 @@ CREATE TABLE tableName( |parallelism | 并行度设置|否|1| ## 5.样例: + +双流join并插入mongo + ``` + +CREATE TABLE source1 ( + id int, + name VARCHAR +)WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest03', + timezone='Asia/Shanghai', + topicIsPattern ='false' + ); + + + +CREATE TABLE source2( + id int, + address VARCHAR +)WITH( + type ='kafka11', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest04', + timezone='Asia/Shanghai', + topicIsPattern ='false' +); + + CREATE TABLE MyResult( - channel VARCHAR, - pv VARCHAR - )WITH( + id int, + name VARCHAR, + address VARCHAR, + primary key (id) +)WITH( type ='mongo', - address ='172.21.32.1:27017,172.21.32.1:27017', - userName ='dtstack', - password ='abc123', - database ='test', - tableName ='pv', - parallelism ='1' - ) - ``` \ No newline at end of file + address ='172.16.8.193:27017', + database ='dtstack', + tableName ='userInfo' +); + +insert into MyResult +select + s1.id, + s1.name, + s2.address +from + source1 s1 +left join + source2 s2 +on + s1.id = s2.id + + + ``` + + + 数据结果: + + 向Topic mqTest03 发送数据 {"name":"maqi","id":1001} 插入 (1001,"maqi",null) + + 向Topic mqTest04 发送数据 {"address":"hz","id":1001} 插入 (1001,"maqi","hz") \ No newline at end of file From f17dc3fbf526196fb60366c0a996a7361f834ff3 Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 28 Apr 2020 09:41:17 +0800 Subject: [PATCH 073/109] =?UTF-8?q?=E6=B5=8B=E8=AF=95=E7=B1=BB?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/launcher/PluginLoadModeTest.java | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/launcher/src/main/test/java/com/dtstack/flink/sql/launcher/PluginLoadModeTest.java b/launcher/src/main/test/java/com/dtstack/flink/sql/launcher/PluginLoadModeTest.java index 0a153d018..3916d8543 100644 --- a/launcher/src/main/test/java/com/dtstack/flink/sql/launcher/PluginLoadModeTest.java +++ b/launcher/src/main/test/java/com/dtstack/flink/sql/launcher/PluginLoadModeTest.java @@ -48,8 +48,22 @@ public static void testClasspathMode() throws Exception { LauncherMain.main(sql); } + + public static void testRocSql() throws Exception{ + String[] sql = new String[]{"-mode", "local", "-sql", "/Users/roc/Documents/flink_sql/sql/zy_sql/hbase_side.sql", "-name", "roc", + "-localSqlPluginPath", "/Users/roc/workspace/git_code/flinkStreamSQL/plugins", + "-remoteSqlPluginPath", "/Users/roc/workspace/git_code/flinkStreamSQL/plugins", + "-flinkconf", "/Users/roc/Documents/flink_sql/flinkconf", + "-confProp", "{\"sql.checkpoint.cleanup.mode\":\"false\",\"sql.checkpoint.interval\":10000,\"time.characteristic\":\"EventTime\"}", + "-yarnconf", "/Users/roc/Documents/flink_sql/yarnconf", + "-flinkJarPath", "/Users/roc/Documents/flink_sql/flinkJarPath", "-queue", "c", "-pluginLoadMode", "classpath"}; + System.setProperty("HADOOP_USER_NAME", "admin"); + LauncherMain.main(sql); + } + public static void main(String[] args) throws Exception { - testShipfileMode(); + testRocSql(); +// testShipfileMode(); // testClasspathMode(); } } From 9957cfd80fd4ad9d3f5e0668c717ed090327d17a Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 28 Apr 2020 14:06:55 +0800 Subject: [PATCH 074/109] =?UTF-8?q?=E6=94=AF=E6=8C=81decimal=E7=B1=BB?= =?UTF-8?q?=E5=9E=8B?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/side/hbase/utils/HbaseUtils.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseUtils.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseUtils.java index 39df3c913..5a08b278a 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseUtils.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseUtils.java @@ -66,9 +66,12 @@ public static Object convertByte(byte[] hbaseData, String type){ case "double": return Bytes.toDouble(hbaseData); - + case "decimal": + return Bytes.toBigDecimal(hbaseData); + default: + throw new RuntimeException("not support type of " + type); } - - throw new RuntimeException("not support type of " + type); } + + } From dfea94ba8db66547bba0ca734880b21afcaed676 Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 28 Apr 2020 14:42:21 +0800 Subject: [PATCH 075/109] =?UTF-8?q?hbase=20=E7=88=B6=E7=B1=BB=E6=B2=A1?= =?UTF-8?q?=E6=9C=89open=E5=92=8C=E7=BC=93=E5=AD=98=E4=B8=AD=E6=95=B0?= =?UTF-8?q?=E6=8D=AE=E7=B1=BB=E5=9E=8B=E4=B8=8D=E4=B8=80=E8=87=B4?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java | 1 + .../sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) 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 251b88034..0daf2bea5 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 @@ -89,6 +89,7 @@ public HbaseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List Date: Tue, 28 Apr 2020 15:05:46 +0800 Subject: [PATCH 076/109] =?UTF-8?q?=E4=BF=AE=E6=94=B9kafka=20json=20?= =?UTF-8?q?=E5=AF=B9=E7=B1=BB=E5=9E=8B=E6=AF=94=E8=BE=83=E7=9A=84=E9=83=A8?= =?UTF-8?q?=E5=88=86=E3=80=82=E4=BF=AE=E6=94=B9=20'=3D=3D'=20=E4=B8=BA=20e?= =?UTF-8?q?quals;?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../kafka/AbstractKafkaProducerFactory.java | 6 +- .../json/DTJsonRowSerializationSchema.java | 225 +++++++++++++++++ .../kafka/AbstractKafkaConsumerFactory.java | 6 +- .../json/DTJsonRowDeserializationSchema.java | 233 ++++++++++++++++++ 4 files changed, 464 insertions(+), 6 deletions(-) create mode 100644 kafka-base/kafka-base-sink/src/main/java/org/apache/flink/formats/json/DTJsonRowSerializationSchema.java create mode 100644 kafka-base/kafka-base-source/src/main/java/org/apache/flink/formats/json/DTJsonRowDeserializationSchema.java diff --git a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaProducerFactory.java b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaProducerFactory.java index 88c2ca939..9bdc0d64f 100644 --- a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaProducerFactory.java +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaProducerFactory.java @@ -25,7 +25,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.formats.avro.AvroRowSerializationSchema; import org.apache.flink.formats.csv.CsvRowSerializationSchema; -import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.formats.json.DTJsonRowSerializationSchema; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.types.Row; @@ -62,9 +62,9 @@ private SerializationSchema createSerializationSchema(KafkaSinkTableInfo ka if (FormatType.JSON.name().equalsIgnoreCase(kafkaSinkTableInfo.getSinkDataType())) { if (StringUtils.isNotBlank(kafkaSinkTableInfo.getSchemaString())) { - serializationSchema = new JsonRowSerializationSchema(kafkaSinkTableInfo.getSchemaString()); + serializationSchema = new DTJsonRowSerializationSchema(kafkaSinkTableInfo.getSchemaString()); } else if (typeInformation != null && typeInformation.getArity() != 0) { - serializationSchema = new JsonRowSerializationSchema(typeInformation); + serializationSchema = new DTJsonRowSerializationSchema(typeInformation); } else { throw new IllegalArgumentException("sinkDataType:" + FormatType.JSON.name() + " must set schemaString(JSON Schema)or TypeInformation"); } diff --git a/kafka-base/kafka-base-sink/src/main/java/org/apache/flink/formats/json/DTJsonRowSerializationSchema.java b/kafka-base/kafka-base-sink/src/main/java/org/apache/flink/formats/json/DTJsonRowSerializationSchema.java new file mode 100644 index 000000000..f67d2353d --- /dev/null +++ b/kafka-base/kafka-base-sink/src/main/java/org/apache/flink/formats/json/DTJsonRowSerializationSchema.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ContainerNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Time; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; +import java.util.Objects; + +/** + * Serialization schema that serializes an object of Flink types into a JSON bytes. + * + *

Serializes the input Flink object into a JSON string and + * converts it into byte[]. + * + *

Result byte[] messages can be deserialized using {@link DTJsonRowDeserializationSchema}. + */ +@PublicEvolving +public class DTJsonRowSerializationSchema implements SerializationSchema { + + private static final long serialVersionUID = -2885556750743978636L; + + /** Type information describing the input type. */ + private final TypeInformation typeInfo; + + /** Object mapper that is used to create output JSON objects. */ + private final ObjectMapper mapper = new ObjectMapper(); + + /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone, without milliseconds). */ + private SimpleDateFormat timeFormat = new SimpleDateFormat("HH:mm:ss'Z'"); + + /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone). */ + private SimpleDateFormat timeFormatWithMillis = new SimpleDateFormat("HH:mm:ss.SSS'Z'"); + + /** Formatter for RFC 3339-compliant string representation of a timestamp value (with UTC timezone). */ + private SimpleDateFormat timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + + /** Reusable object node. */ + private transient ObjectNode node; + + /** + * Creates a JSON serialization schema for the given type information. + * + * @param typeInfo The field names of {@link Row} are used to map to JSON properties. + */ + public DTJsonRowSerializationSchema(TypeInformation typeInfo) { + Preconditions.checkNotNull(typeInfo, "Type information"); + this.typeInfo = typeInfo; + } + + /** + * Creates a JSON serialization schema for the given JSON schema. + * + * @param jsonSchema JSON schema describing the result type + * + * @see http://json-schema.org/ + */ + public DTJsonRowSerializationSchema(String jsonSchema) { + this(JsonRowSchemaConverter.convert(jsonSchema)); + } + + @Override + public byte[] serialize(Row row) { + if (node == null) { + node = mapper.createObjectNode(); + } + + try { + convertRow(node, (RowTypeInfo) typeInfo, row); + return mapper.writeValueAsBytes(node); + } catch (Throwable t) { + throw new RuntimeException("Could not serialize row '" + row + "'. " + + "Make sure that the schema matches the input.", t); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final DTJsonRowSerializationSchema that = (DTJsonRowSerializationSchema) o; + return Objects.equals(typeInfo, that.typeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(typeInfo); + } + + // -------------------------------------------------------------------------------------------- + + private ObjectNode convertRow(ObjectNode reuse, RowTypeInfo info, Row row) { + if (reuse == null) { + reuse = mapper.createObjectNode(); + } + final String[] fieldNames = info.getFieldNames(); + final TypeInformation[] fieldTypes = info.getFieldTypes(); + + // validate the row + if (row.getArity() != fieldNames.length) { + throw new IllegalStateException(String.format( + "Number of elements in the row '%s' is different from number of field names: %d", row, fieldNames.length)); + } + + for (int i = 0; i < fieldNames.length; i++) { + final String name = fieldNames[i]; + + final JsonNode fieldConverted = convert(reuse, reuse.get(name), fieldTypes[i], row.getField(i)); + reuse.set(name, fieldConverted); + } + + return reuse; + } + + private JsonNode convert(ContainerNode container, JsonNode reuse, TypeInformation info, Object object) { + if (info.equals(Types.VOID) || object == null) { + return container.nullNode(); + } else if (info.equals(Types.BOOLEAN)) { + return container.booleanNode((Boolean) object); + } else if (info.equals(Types.STRING)) { + return container.textNode((String) object); + } else if (info.equals(Types.BIG_DEC)) { + // convert decimal if necessary + if (object instanceof BigDecimal) { + return container.numberNode((BigDecimal) object); + } + return container.numberNode(BigDecimal.valueOf(((Number) object).doubleValue())); + } else if (info.equals(Types.BIG_INT)) { + // convert integer if necessary + if (object instanceof BigInteger) { + return container.numberNode((BigInteger) object); + } + return container.numberNode(BigInteger.valueOf(((Number) object).longValue())); + } else if (info.equals(Types.SQL_DATE)) { + return container.textNode(object.toString()); + } else if (info.equals(Types.SQL_TIME)) { + final Time time = (Time) object; + // strip milliseconds if possible + if (time.getTime() % 1000 > 0) { + return container.textNode(timeFormatWithMillis.format(time)); + } + return container.textNode(timeFormat.format(time)); + } else if (info.equals(Types.SQL_TIMESTAMP)) { + return container.textNode(timestampFormat.format((Timestamp) object)); + } else if (info instanceof RowTypeInfo) { + if (reuse != null && reuse instanceof ObjectNode) { + return convertRow((ObjectNode) reuse, (RowTypeInfo) info, (Row) object); + } else { + return convertRow(null, (RowTypeInfo) info, (Row) object); + } + } else if (info instanceof ObjectArrayTypeInfo) { + if (reuse != null && reuse instanceof ArrayNode) { + return convertObjectArray((ArrayNode) reuse, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } else { + return convertObjectArray(null, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } + } else if (info instanceof BasicArrayTypeInfo) { + if (reuse != null && reuse instanceof ArrayNode) { + return convertObjectArray((ArrayNode) reuse, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } else { + return convertObjectArray(null, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } + } else if (info instanceof PrimitiveArrayTypeInfo && ((PrimitiveArrayTypeInfo) info).getComponentType().equals(Types.BYTE)) { + return container.binaryNode((byte[]) object); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return mapper.valueToTree(object); + } catch (IllegalArgumentException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for object: " + object, e); + } + } + } + + private ArrayNode convertObjectArray(ArrayNode reuse, TypeInformation info, Object[] array) { + if (reuse == null) { + reuse = mapper.createArrayNode(); + } else { + reuse.removeAll(); + } + + for (Object object : array) { + reuse.add(convert(reuse, null, info, object)); + } + return reuse; + } +} diff --git a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/AbstractKafkaConsumerFactory.java b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/AbstractKafkaConsumerFactory.java index 2cae221cf..29cf3193b 100644 --- a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/AbstractKafkaConsumerFactory.java +++ b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/AbstractKafkaConsumerFactory.java @@ -27,7 +27,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.formats.avro.AvroRowDeserializationSchema; import org.apache.flink.formats.csv.CsvRowDeserializationSchema; -import org.apache.flink.formats.json.JsonRowDeserializationSchema; +import org.apache.flink.formats.json.DTJsonRowDeserializationSchema; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase; import org.apache.flink.types.Row; @@ -61,9 +61,9 @@ private DeserializationSchema createDeserializationSchema(KafkaSourceTableI } else if (FormatType.JSON.name().equalsIgnoreCase(kafkaSourceTableInfo.getSourceDataType())) { if (StringUtils.isNotBlank(kafkaSourceTableInfo.getSchemaString())) { - deserializationSchema = new JsonRowDeserializationSchema(kafkaSourceTableInfo.getSchemaString()); + deserializationSchema = new DTJsonRowDeserializationSchema(kafkaSourceTableInfo.getSchemaString()); } else if (typeInformation != null && typeInformation.getArity() != 0) { - deserializationSchema = new JsonRowDeserializationSchema(typeInformation); + deserializationSchema = new DTJsonRowDeserializationSchema(typeInformation); } else { throw new IllegalArgumentException("sourceDataType:" + FormatType.JSON.name() + " must set schemaString(JSON Schema)or TypeInformation"); } diff --git a/kafka-base/kafka-base-source/src/main/java/org/apache/flink/formats/json/DTJsonRowDeserializationSchema.java b/kafka-base/kafka-base-source/src/main/java/org/apache/flink/formats/json/DTJsonRowDeserializationSchema.java new file mode 100644 index 000000000..6204058fe --- /dev/null +++ b/kafka-base/kafka-base-source/src/main/java/org/apache/flink/formats/json/DTJsonRowDeserializationSchema.java @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.json; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.lang.reflect.Array; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.Objects; + +/** + * Deserialization schema from JSON to Flink types. + * + *

Deserializes a byte[] message as a JSON object and reads + * the specified fields. + * + *

Failures during deserialization are forwarded as wrapped IOExceptions. + */ +@PublicEvolving +public class DTJsonRowDeserializationSchema implements DeserializationSchema { + + private static final long serialVersionUID = -228294330688809195L; + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Object mapper for parsing the JSON. */ + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + /** + * Creates a JSON deserialization schema for the given type information. + * + * @param typeInfo Type information describing the result type. The field names of {@link Row} + * are used to parse the JSON properties. + */ + public DTJsonRowDeserializationSchema(TypeInformation typeInfo) { + Preconditions.checkNotNull(typeInfo, "Type information"); + this.typeInfo = typeInfo; + + if (!(typeInfo instanceof RowTypeInfo)) { + throw new IllegalArgumentException("Row type information expected."); + } + } + + /** + * Creates a JSON deserialization schema for the given JSON schema. + * + * @param jsonSchema JSON schema describing the result type + * + * @see http://json-schema.org/ + */ + public DTJsonRowDeserializationSchema(String jsonSchema) { + this(JsonRowSchemaConverter.convert(jsonSchema)); + } + + @Override + public Row deserialize(byte[] message) throws IOException { + try { + final JsonNode root = objectMapper.readTree(message); + return convertRow(root, (RowTypeInfo) typeInfo); + } catch (Throwable t) { + throw new IOException("Failed to deserialize JSON object.", t); + } + } + + @Override + public boolean isEndOfStream(Row nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return typeInfo; + } + + /** + * Configures the failure behaviour if a JSON field is missing. + * + *

By default, a missing field is ignored and the field is set to null. + * + * @param failOnMissingField Flag indicating whether to fail or not on a missing field. + */ + public void setFailOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final DTJsonRowDeserializationSchema that = (DTJsonRowDeserializationSchema) o; + return failOnMissingField == that.failOnMissingField && Objects.equals(typeInfo, that.typeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(typeInfo, failOnMissingField); + } + + // -------------------------------------------------------------------------------------------- + + private Object convert(JsonNode node, TypeInformation info) { + if (info.equals(Types.VOID) || node.isNull()) { + return null; + } else if (info.equals(Types.BOOLEAN)) { + return node.asBoolean(); + } else if (info.equals(Types.STRING)) { + return node.asText(); + } else if (info.equals(Types.BIG_DEC)) { + return node.decimalValue(); + } else if (info.equals(Types.BIG_INT)) { + return node.bigIntegerValue(); + } else if (info.equals(Types.SQL_DATE)) { + return Date.valueOf(node.asText()); + } else if (info.equals(Types.SQL_TIME)) { + // according to RFC 3339 every full-time must have a timezone; + // until we have full timezone support, we only support UTC; + // users can parse their time as string as a workaround + final String time = node.asText(); + if (time.indexOf('Z') < 0 || time.indexOf('.') >= 0) { + throw new IllegalStateException( + "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet. " + + "Format: HH:mm:ss'Z'"); + } + return Time.valueOf(time.substring(0, time.length() - 1)); + } else if (info.equals(Types.SQL_TIMESTAMP)) { + // according to RFC 3339 every date-time must have a timezone; + // until we have full timezone support, we only support UTC; + // users can parse their time as string as a workaround + final String timestamp = node.asText(); + if (timestamp.indexOf('Z') < 0) { + throw new IllegalStateException( + "Invalid timestamp format. Only a timestamp in UTC timezone is supported yet. " + + "Format: yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + } + return Timestamp.valueOf(timestamp.substring(0, timestamp.length() - 1).replace('T', ' ')); + } else if (info instanceof RowTypeInfo) { + return convertRow(node, (RowTypeInfo) info); + } else if (info instanceof ObjectArrayTypeInfo) { + return convertObjectArray(node, ((ObjectArrayTypeInfo) info).getComponentInfo()); + } else if (info instanceof BasicArrayTypeInfo) { + return convertObjectArray(node, ((BasicArrayTypeInfo) info).getComponentInfo()); + } else if (info instanceof PrimitiveArrayTypeInfo && + ((PrimitiveArrayTypeInfo) info).getComponentType() == Types.BYTE) { + return convertByteArray(node); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return objectMapper.treeToValue(node, info.getTypeClass()); + } catch (JsonProcessingException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); + } + } + } + + private Row convertRow(JsonNode node, RowTypeInfo info) { + final String[] names = info.getFieldNames(); + final TypeInformation[] types = info.getFieldTypes(); + + final Row row = new Row(names.length); + for (int i = 0; i < names.length; i++) { + final String name = names[i]; + final JsonNode subNode = node.get(name); + if (subNode == null) { + if (failOnMissingField) { + throw new IllegalStateException( + "Could not find field with name '" + name + "'."); + } else { + row.setField(i, null); + } + } else { + row.setField(i, convert(subNode, types[i])); + } + } + + return row; + } + + private Object convertObjectArray(JsonNode node, TypeInformation elementType) { + final Object[] array = (Object[]) Array.newInstance(elementType.getTypeClass(), node.size()); + for (int i = 0; i < node.size(); i++) { + array[i] = convert(node.get(i), elementType); + } + return array; + } + + private Object convertByteArray(JsonNode node) { + try { + return node.binaryValue(); + } catch (IOException e) { + throw new RuntimeException("Unable to deserialize byte array.", e); + } + } +} From ee766b80005bb4dee464fb67d8d48e6ef39deeaa Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 28 Apr 2020 20:25:29 +0800 Subject: [PATCH 077/109] =?UTF-8?q?fix=20=E7=BC=93=E5=AD=98=E6=97=B6?= =?UTF-8?q?=E9=97=B4=E6=B2=A1=E6=9C=89=E8=A7=A3=E6=9E=90=E7=9A=84=E9=97=AE?= =?UTF-8?q?=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java | 2 +- .../com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) 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 0daf2bea5..47adc88b9 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 @@ -148,7 +148,7 @@ public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exce return; } else if (ECacheContentType.SingleLine == val.getType()) { try { - Row row = fillData(inputCopy.row(), val); + Row row = fillData(inputCopy.row(), val.getContent()); resultFuture.complete(Collections.singleton(new CRow(row, inputCopy.change()))); } catch (Exception e) { dealFillDataError(resultFuture, e, inputCopy); diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java index 7b627da4d..901c000f5 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java @@ -61,6 +61,7 @@ public HbaseSideParser() { public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { HbaseSideTableInfo hbaseTableInfo = new HbaseSideTableInfo(); hbaseTableInfo.setName(tableName); + parseCacheProp(hbaseTableInfo, props); parseFieldsInfo(fieldsInfo, hbaseTableInfo); hbaseTableInfo.setTableName((String) props.get(TABLE_NAME_KEY.toLowerCase())); hbaseTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); From bbb4e9c6b8652f7c8e34412df6b12df002f8ead5 Mon Sep 17 00:00:00 2001 From: tiezhu Date: Wed, 29 Apr 2020 15:26:39 +0800 Subject: [PATCH 078/109] =?UTF-8?q?update=20kudu-side=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/kuduSide.md | 151 +++++++++++++++++++++++++--------------- pom.xml | 4 +- 2 files changed, 98 insertions(+), 57 deletions(-) diff --git a/docs/plugin/kuduSide.md b/docs/plugin/kuduSide.md index 718d43117..a45300deb 100644 --- a/docs/plugin/kuduSide.md +++ b/docs/plugin/kuduSide.md @@ -1,53 +1,34 @@ - -## 1.格式: -All: -``` -create table sideTable( - id int, - tablename1 VARCHAR, - PRIMARY KEY(id), - PERIOD FOR SYSTEM_TIME - )WITH( - type='kudu', - kuduMasters ='ip1,ip2,ip3', - tableName ='impala::default.testSide', - cache ='ALL', - primaryKey='id,xx', - lowerBoundPrimaryKey='10,xx', - upperBoundPrimaryKey='15,xx', - workerCount='1', - defaultOperationTimeoutMs='600000', - defaultSocketReadTimeoutMs='6000000', - batchSizeBytes='100000000', - limitNum='1000', - isFaultTolerant='false', - partitionedJoin='false' - ); -``` -LRU: +## 1.格式 + 通过建表语句中的` PERIOD FOR SYSTEM_TIME`将表标识为维表,其中`PRIMARY KEY(keyInfo)`中的keyInfo,表示用来和源表进行关联的字段, + 维表JOIN的条件必须与`keyInfo`字段一致。 ``` -create table sideTable( - id int, - tablename1 VARCHAR, - PRIMARY KEY(id), + CREATE TABLE tableName( + colName cloType, + ... + PRIMARY KEY(colName1,colName2) , PERIOD FOR SYSTEM_TIME - )WITH( - type='kudu', + )WITH( + type ='kudu', kuduMasters ='ip1,ip2,ip3', tableName ='impala::default.testSide', - cache ='LRU', + primaryKey='id,xx', + lowerBoundPrimaryKey='10,xx', + upperBoundPrimaryKey='15,xx', workerCount='1', defaultOperationTimeoutMs='600000', defaultSocketReadTimeoutMs='6000000', batchSizeBytes='100000000', limitNum='1000', isFaultTolerant='false', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', partitionedJoin='false' - ); - ``` - + ); +``` ## 2.支持版本 -kudu 1.10.0+cdh6.2.0 + kudu 1.10.0+cdh6.2.0 ## 3.表结构定义 @@ -55,12 +36,13 @@ kudu 1.10.0+cdh6.2.0 |----|---| | tableName | 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同)| | colName | 列名称| - | colType | 列类型 [colType支持的类型](docs/colType.md)| + | colType | 列类型 [colType支持的类型](colType.md)| | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| -## 3.参数 +## 4.参数 +参数详细说明请看[参数详细说明](./sideParams.md) |参数名称|含义|是否必填|默认值| |----|---|---|-----| @@ -78,17 +60,25 @@ kudu 1.10.0+cdh6.2.0 | isFaultTolerant |查询是否容错 查询失败是否扫描第二个副本 默认false 容错 | 否|| | cache | 维表缓存策略(NONE/LRU/ALL)|否|NONE| | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| - - -------------- -> 缓存策略 - * NONE: 不做内存缓存 - * LRU: - * cacheSize: 缓存的条目数量 - * cacheTTLMs:缓存的过期时间(ms) -## 4.样例 -All: +## 5.样例 +### LRU维表示例 +``` +create table sideTable( + id int, + tablename1 VARCHAR, + PRIMARY KEY(id), + PERIOD FOR SYSTEM_TIME + )WITH( + type='kudu', + kuduMasters ='ip1,ip2,ip3', + tableName ='impala::default.testSide', + cache ='LRU', + partitionedJoin='false' + ); +``` +### ALL维表示例 ``` create table sideTable( id int, @@ -106,19 +96,70 @@ create table sideTable( partitionedJoin='false' ); ``` -LRU: + +## 6.kudu异步关联完整样例 + ``` -create table sideTable( +CREATE TABLE MyTable( + id bigint, + name varchar, + address varchar +)WITH( + type = 'kafka10', + bootstrapServers = '172.16.101.224:9092', + zookeeperQuorm = '172.16.100.188:2181/kafka', + offsetReset = 'latest', + topic = 'tiezhu_test_in2', + timezone = 'Asia/Shanghai', + topicIsPattern = 'false', + parallelism = '1' +); + +CREATE TABLE sideTable( id int, - tablename1 VARCHAR, + message varchar, PRIMARY KEY(id), PERIOD FOR SYSTEM_TIME - )WITH( +)WITH( type='kudu', kuduMasters ='ip1,ip2,ip3', tableName ='impala::default.testSide', cache ='LRU', partitionedJoin='false' - ); - ``` +); + +CREATE TABLE MyResult( + id bigint, + name varchar, + address varchar, + message varchar +)WITH( + type ='console', + address ='192.168.80.106:9042,192.168.80.107:9042', + userName='cassandra', + password='cassandra', + database ='tiezhu', + tableName ='stu_out', + parallelism ='1' +); + +insert +into + MyResult + select + t1.id AS id, + t1.name AS name, + t1.address AS address, + t2.message AS message + from( + select + id, + name, + address + from + MyTable + ) t1 + join sideTable t2 + on t1.id = t2.id; +``` diff --git a/pom.xml b/pom.xml index b5b674379..2b2d0a986 100644 --- a/pom.xml +++ b/pom.xml @@ -25,10 +25,10 @@ - + kudu - + console From 1a361e1e3c86a0e09b3460ab05b43540f002cd01 Mon Sep 17 00:00:00 2001 From: xuchao Date: Thu, 30 Apr 2020 19:19:43 +0800 Subject: [PATCH 079/109] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E8=A1=A8=E5=90=8D?= =?UTF-8?q?=E7=A7=B0=E5=92=8C=20schema=20=E6=B7=BB=E5=8A=A0=E5=BC=95?= =?UTF-8?q?=E5=8F=B7=E7=9A=84=E9=80=BB=E8=BE=91?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/util/DtStringUtil.java | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) 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 b2486b1ab..f84cc9ae4 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 @@ -261,14 +261,43 @@ public static String firstUpperCase(String str) { } public static String getTableFullPath(String schema, String tableName) { + String[] tableInfoSplit = StringUtils.split(tableName, "."); + //表明表信息带了schema + if(tableInfoSplit.length == 2){ + schema = tableInfoSplit[0]; + tableName = tableInfoSplit[1]; + } + + //清理首个字符" 和最后字符 " + schema = rmStrQuote(schema); + tableName = rmStrQuote(tableName); + if (StringUtils.isEmpty(schema)){ return addQuoteForStr(tableName); } + String schemaAndTabName = addQuoteForStr(schema) + "." + addQuoteForStr(tableName); return schemaAndTabName; } + /** + * 清理首个字符" 和最后字符 " + */ + public static String rmStrQuote(String str){ + if(StringUtils.isEmpty(str)){ + return str; + } + + if(str.startsWith("\"")){ + str = str.substring(1); + } + + if(str.endsWith("\"")){ + str = str.substring(0, str.length()-1); + } + return str; + } public static String addQuoteForStr(String column) { return getStartQuote() + column + getEndQuote(); From cc228def0670a8b6c62add4e4be267c437f3e4ed Mon Sep 17 00:00:00 2001 From: dapeng Date: Wed, 6 May 2020 19:13:48 +0800 Subject: [PATCH 080/109] =?UTF-8?q?=E4=BF=AE=E5=A4=8D=E4=B8=AD=E5=8E=9Fhba?= =?UTF-8?q?se=E7=BB=B4=E8=A1=A8=E5=8F=96=E5=87=BA=E6=95=B0=E6=8D=AE?= =?UTF-8?q?=E6=9C=89=E8=AF=AF=E7=9A=84=E9=97=AE=E9=A2=98=E5=90=97?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 664fe6abd..9b79c5c3d 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 @@ -33,6 +33,7 @@ import com.dtstack.flink.sql.factory.DTThreadFactory; import com.google.common.collect.Maps; import com.stumbleupon.async.Deferred; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.ResultFuture; @@ -83,7 +84,7 @@ public HbaseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List Date: Mon, 11 May 2020 15:06:00 +0800 Subject: [PATCH 081/109] =?UTF-8?q?redis=20=E7=BB=93=E6=9E=9C=E8=A1=A8?= =?UTF-8?q?=E6=8F=90=E4=BA=A4?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../sql/launcher/PluginLoadModeTest.java | 15 +++++++++- pom.xml | 28 +++++++++---------- 2 files changed, 28 insertions(+), 15 deletions(-) diff --git a/launcher/src/main/test/java/com/dtstack/flink/sql/launcher/PluginLoadModeTest.java b/launcher/src/main/test/java/com/dtstack/flink/sql/launcher/PluginLoadModeTest.java index 0a153d018..923c74116 100644 --- a/launcher/src/main/test/java/com/dtstack/flink/sql/launcher/PluginLoadModeTest.java +++ b/launcher/src/main/test/java/com/dtstack/flink/sql/launcher/PluginLoadModeTest.java @@ -48,8 +48,21 @@ public static void testClasspathMode() throws Exception { LauncherMain.main(sql); } + public static void testroc() throws Exception { + String[] sql = new String[]{"-mode", "local", "-sql", "/Users/roc/Documents/flink_sql/sql/result/redis_result.sql", "-name", "roc", + "-localSqlPluginPath", "/Users/roc/workspace/git_code/flinkStreamSQL/plugins", + "-remoteSqlPluginPath", "/Users/roc/workspace/git_code/flinkStreamSQL/plugins", + "-flinkconf", "//Users/roc/Documents/flink_sql/flinkconf", + "-confProp", "{\"sql.checkpoint.cleanup.mode\":\"false\",\"sql.checkpoint.interval\":10000,\"time.characteristic\":\"EventTime\"}", + "-yarnconf", "/Users/roc/Documents/flink_sql/yarnconf", + "-flinkJarPath", "/Users/roc/Documents/flink_sql/flinkJarPath", "-queue", "c", "-pluginLoadMode", "classpath"}; + System.setProperty("HADOOP_USER_NAME", "admin"); + LauncherMain.main(sql); + } + public static void main(String[] args) throws Exception { - testShipfileMode(); + testroc(); +// testShipfileMode(); // testClasspathMode(); } } diff --git a/pom.xml b/pom.xml index 2b2d0a986..2794f9004 100644 --- a/pom.xml +++ b/pom.xml @@ -16,24 +16,24 @@ kafka11 kafka mysql - - - - + hbase + elasticsearch5 + mongo + redis5 launcher rdb - - - + sqlserver + oracle + cassandra kudu - - + postgresql + serversocket console - - - - - + clickhouse + impala + db2 + polardb + elasticsearch6 From 966b17136fb06ad52394908c0f20dd440ebe7911 Mon Sep 17 00:00:00 2001 From: dapeng Date: Mon, 11 May 2020 15:06:17 +0800 Subject: [PATCH 082/109] =?UTF-8?q?=E8=A1=A5=E5=85=85redis=E7=BB=93?= =?UTF-8?q?=E6=9E=9C=E8=A1=A8=E6=8F=90=E4=BA=A4?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/redisSink.md | 73 ++++++++++++++++++++++++++++++++-------- 1 file changed, 59 insertions(+), 14 deletions(-) diff --git a/docs/plugin/redisSink.md b/docs/plugin/redisSink.md index b43cb7377..224c26b2f 100644 --- a/docs/plugin/redisSink.md +++ b/docs/plugin/redisSink.md @@ -13,8 +13,6 @@ CREATE TABLE tableName( tableName ='tableName', parallelism ='parllNum' ); - - ``` ## 2.支持版本 @@ -50,17 +48,64 @@ redis5.0 ## 5.样例: ``` + CREATE TABLE MyTable( + name varchar, + channel varchar + )WITH( + type ='kafka10', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest01', + timezone='Asia/Shanghai', + updateMode ='append', + enableKeyPartitions ='false', + topicIsPattern ='false', + parallelism ='1' + ); + CREATE TABLE MyResult( - channel varchar, - pv varchar, - PRIMARY KEY(channel) - )WITH( - type='redis', - url='172.16.10.79:6379', - password='abc123', - database='0', - redisType='1', - tableName='sinktoredis' - ); + channel VARCHAR, + pv VARCHAR + )WITH( + type ='redis', + redisType ='1', + url ='172.16.8.109:6379', + tableName ='resultTable', + partitionedJoin ='false', + parallelism ='1', + database ='0', + timeout ='10000', + maxTotal ='60000', + maxIdle='8', + minIdle='0' + ); + + insert + into + MyResult + select + channel, + name as pv + from + MyTable a + ``` + +## 6.redis完整样例 +### redis数据说明 +redis使用k-v格式存储,key的构建格式为tableName:privateKey:privateKeyValue:columnName, value=columnValue - ``` \ No newline at end of file +### 源表数据内容 +``` +{"name":"roc","channel":"daishu","age":2} +``` +### redis实际数据内容 +``` +127.0.0.1:6379> keys * +1) "resultTable:name:roc:name" +2) "resultTable:name:roc:channel" +127.0.0.1:6379> get "resultTable:name:roc:name" +"roc" +127.0.0.1:6379> get "resultTable:name:roc:channel" +"daishu" +``` \ No newline at end of file From 85aae4721ccbb27a0319fd8c1d40327a67545ef5 Mon Sep 17 00:00:00 2001 From: xuchao Date: Mon, 11 May 2020 15:34:35 +0800 Subject: [PATCH 083/109] =?UTF-8?q?=E5=9B=9E=E6=BB=9A=205f30a74b=20?= =?UTF-8?q?=EF=BC=8C=E8=A7=A3=E5=86=B3=20'=E5=9B=9E=E6=BB=9A=E5=AF=BC?= =?UTF-8?q?=E8=87=B4=E7=9A=84test=E5=88=86=E6=94=AF=E5=8F=98=E6=9B=B4?= =?UTF-8?q?=E5=86=85=E5=AE=B9'=E6=97=A0=E6=B3=95=E6=8F=90=E4=BA=A4?= =?UTF-8?q?=E7=9A=84=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .gitlab-ci.yml | 10 + .../side/cassandra/CassandraAllReqRow.java | 10 +- .../side/cassandra/CassandraAllSideInfo.java | 10 +- .../side/cassandra/CassandraAsyncReqRow.java | 11 +- .../cassandra/CassandraAsyncSideInfo.java | 18 +- .../cassandra/table/CassandraSideParser.java | 17 +- .../table/CassandraSideTableInfo.java | 4 +- .../sink/cassandra/CassandraOutputFormat.java | 4 +- .../sql/sink/cassandra/CassandraSink.java | 4 +- .../cassandra/table/CassandraSinkParser.java | 10 +- .../cassandra/table/CassandraTableInfo.java | 4 +- ci/sonar_notify.sh | 14 + .../side/clickhouse/ClickhouseAllReqRow.java | 17 +- .../clickhouse/ClickhouseAllSideInfo.java | 4 +- .../clickhouse/ClickhouseAsyncReqRow.java | 6 +- .../clickhouse/ClickhouseAsyncSideInfo.java | 4 +- .../table/ClickhouseSideParser.java | 6 +- .../sql/sink/clickhouse/ClickhouseSink.java | 9 +- .../table/ClickhouseSinkParser.java | 6 +- .../sql/sink/console/ConsoleOutputFormat.java | 6 +- .../flink/sql/sink/console/ConsoleSink.java | 4 +- .../sink/console/table/ConsoleSinkParser.java | 10 +- .../sink/console/table/ConsoleTableInfo.java | 4 +- .../sink/console/table/TablePrintUtil.java | 23 +- core/pom.xml | 10 +- .../main/java/com/dtstack/flink/sql/Main.java | 5 +- .../sql/classloader/ClassLoaderManager.java | 4 +- .../flink/sql/constrant/ConfigConstrant.java | 13 +- .../dtstack/flink/sql/enums/ColumnType.java | 63 ++- .../flink/sql/enums/ECacheContentType.java | 9 + .../dtstack/flink/sql/enums/ECacheType.java | 13 +- .../flink/sql/enums/EDatabaseType.java | 9 + .../flink/sql/enums/EPluginLoadMode.java | 6 + .../flink/sql/enums/EStateBackend.java | 9 + .../environment/StreamEnvConfigManager.java | 59 ++- .../flink/sql/exec/ExecuteProcessHelper.java | 76 ++-- .../dtstack/flink/sql/exec/ParamsInfo.java | 1 - .../format/SerializationMetricWrapper.java | 11 +- .../DtNestRowDeserializationSchema.java | 11 +- .../flink/sql/option/OptionParser.java | 21 +- ...t.java => AbstractDtRichOutputFormat.java} | 2 +- .../flink/sql/parser/CreateFuncParser.java | 8 +- .../dtstack/flink/sql/parser/SqlParser.java | 12 +- .../com/dtstack/flink/sql/parser/SqlTree.java | 8 +- ...leInfo.java => AbstractSideTableInfo.java} | 4 +- .../{AllReqRow.java => BaseAllReqRow.java} | 18 +- ...{AsyncReqRow.java => BaseAsyncReqRow.java} | 18 +- .../side/{SideInfo.java => BaseSideInfo.java} | 22 +- .../flink/sql/side/JoinNodeDealer.java | 10 +- .../flink/sql/side/ParserJoinField.java | 2 + .../flink/sql/side/SidePredicatesParser.java | 8 +- .../dtstack/flink/sql/side/SideSQLParser.java | 7 +- .../dtstack/flink/sql/side/SideSqlExec.java | 41 +- .../flink/sql/side/StreamSideFactory.java | 10 +- ...sSideCache.java => AbstractSideCache.java} | 8 +- .../flink/sql/side/cache/LRUSideCache.java | 6 +- .../sql/side/operator/SideAsyncOperator.java | 16 +- .../operator/SideWithAllCacheOperator.java | 18 +- .../flink/sql/sink/IStreamSinkGener.java | 4 +- .../flink/sql/sink/StreamSinkFactory.java | 12 +- .../flink/sql/source/IStreamSourceGener.java | 4 +- .../flink/sql/source/StreamSourceFactory.java | 12 +- ...rser.java => AbstractSideTableParser.java} | 49 +-- ...eParser.java => AbstractSourceParser.java} | 16 +- ...Info.java => AbstractSourceTableInfo.java} | 2 +- ...{TableInfo.java => AbstractTableInfo.java} | 2 +- ...rser.java => AbstractTableInfoParser.java} | 20 +- ...leParser.java => AbstractTableParser.java} | 20 +- ...Info.java => AbstractTargetTableInfo.java} | 2 +- .../sql/table/ITableFieldDealHandler.java | 2 +- .../com/dtstack/flink/sql/util/ClassUtil.java | 3 +- .../com/dtstack/flink/sql/util/DateUtil.java | 408 ++++++++---------- .../dtstack/flink/sql/util/DtStringUtil.java | 8 +- .../com/dtstack/flink/sql/util/JDBCUtils.java | 4 +- .../com/dtstack/flink/sql/util/MathUtil.java | 2 +- .../dtstack/flink/sql/util/PluginUtil.java | 10 +- ....java => AbstractCustomerWaterMarker.java} | 4 +- .../CustomerWaterMarkerForLong.java | 2 +- .../CustomerWaterMarkerForTimeStamp.java | 3 +- .../sql/watermarker/WaterMarkerAssigner.java | 8 +- .../sql/side/SidePredicatesParserTest.java | 4 +- .../flink/sql/side/db2/Db2AllReqRow.java | 15 +- .../flink/sql/side/db2/Db2AllSideInfo.java | 4 +- .../flink/sql/side/db2/Db2AsyncReqRow.java | 6 +- .../flink/sql/side/db2/Db2AsyncSideInfo.java | 4 +- .../sql/side/db2/table/Db2SideParser.java | 6 +- .../com/dtstack/flink/sql/sink/db/DbSink.java | 9 +- .../flink/sql/sink/db/table/DbSinkParser.java | 6 +- docs/kafkaSink.md | 223 ++++++++++ docs/plugin/kafkaSource.md | 226 ++-------- docs/plugin/mongoSide.md | 5 +- docs/plugin/mongoSink.md | 7 +- .../sink/elasticsearch/ElasticsearchSink.java | 4 +- .../table/ElasticsearchSinkParser.java | 8 +- .../table/ElasticsearchTableInfo.java | 4 +- .../Elasticsearch6AllReqRow.java | 21 +- .../Elasticsearch6AllSideInfo.java | 11 +- .../Elasticsearch6AsyncReqRow.java | 11 +- .../Elasticsearch6AsyncSideInfo.java | 13 +- .../table/Elasticsearch6SideParser.java | 8 +- .../table/Elasticsearch6SideTableInfo.java | 4 +- .../sql/side/elasticsearch6/util/Es6Util.java | 10 +- .../sink/elasticsearch/ElasticsearchSink.java | 4 +- .../table/ElasticsearchSinkParser.java | 8 +- .../table/ElasticsearchTableInfo.java | 4 +- .../flink/sql/side/hbase/HbaseAllReqRow.java | 22 +- .../sql/side/hbase/HbaseAllSideInfo.java | 10 +- .../sql/side/hbase/HbaseAsyncReqRow.java | 14 +- .../sql/side/hbase/HbaseAsyncSideInfo.java | 10 +- ...ler.java => AbstractRowKeyModeDealer.java} | 12 +- .../PreRowKeyModeDealerDealer.java | 12 +- .../rowkeydealer/RowKeyEqualModeDealer.java | 9 +- ...ator.java => AbstractReplaceOperator.java} | 4 +- .../sql/side/hbase/Md5ReplaceOperator.java | 2 +- .../sql/side/hbase/enums/EReplaceOpType.java | 9 +- .../sql/side/hbase/enums/EReplaceType.java | 9 + .../sql/side/hbase/table/HbaseSideParser.java | 12 +- .../side/hbase/table/HbaseSideTableInfo.java | 4 +- .../sql/side/hbase/utils/HbaseUtils.java | 2 + .../sql/sink/hbase/HbaseOutputFormat.java | 11 +- .../flink/sql/sink/hbase/HbaseSink.java | 5 +- .../sql/sink/hbase/table/HbaseSinkParser.java | 10 +- .../sql/sink/hbase/table/HbaseTableInfo.java | 4 +- .../sql/side/impala/ImpalaAllReqRow.java | 13 +- .../sql/side/impala/ImpalaAllSideInfo.java | 7 +- .../sql/side/impala/ImpalaAsyncReqRow.java | 12 +- .../sql/side/impala/ImpalaAsyncSideInfo.java | 6 +- .../side/impala/table/ImpalaSideParser.java | 6 +- .../flink/sql/sink/impala/ImpalaSink.java | 14 +- .../sink/impala/table/ImpalaSinkParser.java | 7 +- .../kafka/AbstractKafkaProducerFactory.java | 37 +- .../sql/sink/kafka/AbstractKafkaSink.java | 148 +++++++ .../sink/kafka/CustomerFlinkPartition.java | 8 +- .../CustomerKeyedSerializationSchema.java | 35 +- .../AvroCRowSerializationSchema.java | 366 ++++++++++++++++ .../CsvCRowSerializationSchema.java | 374 ++++++++++++++++ .../JsonCRowSerializationSchema.java | 234 ++++++++++ .../sql/sink/kafka/table/KafkaSinkParser.java | 13 +- .../sink/kafka/table/KafkaSinkTableInfo.java | 44 +- .../sql/source/kafka/AbstractKafkaSource.java | 117 +++++ .../sql/source/kafka/enums/EKafkaOffset.java | 43 +- .../source/kafka/table/KafkaSourceParser.java | 42 +- .../kafka/table/KafkaSourceTableInfo.java | 181 ++++---- .../flink/sql/sink/kafka/KafkaProducer.java | 6 +- .../sql/sink/kafka/KafkaProducerFactory.java | 4 +- .../flink/sql/sink/kafka/KafkaSink.java | 118 +---- .../flink/sql/source/kafka/KafkaSource.java | 78 +--- .../flink/sql/sink/kafka/KafkaProducer09.java | 5 +- .../sink/kafka/KafkaProducer09Factory.java | 3 +- .../flink/sql/sink/kafka/KafkaSink.java | 124 +----- .../flink/sql/source/kafka/KafkaSource.java | 76 +--- .../sql/sink/kafka/KafkaProducer010.java | 6 +- .../sink/kafka/KafkaProducer010Factory.java | 3 +- .../flink/sql/sink/kafka/KafkaSink.java | 122 +----- .../flink/sql/source/kafka/KafkaSource.java | 89 +--- .../sql/sink/kafka/KafkaProducer011.java | 6 +- .../sink/kafka/KafkaProducer011Factory.java | 3 +- .../flink/sql/sink/kafka/KafkaSink.java | 121 +----- .../source/kafka/KafkaConsumer011Factory.java | 2 +- .../flink/sql/source/kafka/KafkaSource.java | 92 +--- .../flink/sql/side/kudu/KuduAllReqRow.java | 24 +- .../flink/sql/side/kudu/KuduAllSideInfo.java | 10 +- .../flink/sql/side/kudu/KuduAsyncReqRow.java | 26 +- .../sql/side/kudu/KuduAsyncSideInfo.java | 10 +- .../sql/side/kudu/table/KuduSideParser.java | 11 +- .../side/kudu/table/KuduSideTableInfo.java | 4 +- .../flink/sql/side/kudu/utils/KuduUtil.java | 1 + .../flink/sql/sink/kudu/KuduOutputFormat.java | 8 +- .../dtstack/flink/sql/sink/kudu/KuduSink.java | 4 +- .../sql/sink/kudu/table/KuduSinkParser.java | 11 +- .../sql/sink/kudu/table/KuduTableInfo.java | 4 +- .../sql/launcher/ClusterClientFactory.java | 8 +- .../perjob/PerJobClusterClientBuilder.java | 3 +- .../sql/launcher/perjob/PerJobSubmitter.java | 1 - .../flink/sql/side/mongo/MongoAllReqRow.java | 62 +-- .../sql/side/mongo/MongoAllSideInfo.java | 10 +- .../sql/side/mongo/MongoAsyncReqRow.java | 83 ++-- .../sql/side/mongo/MongoAsyncSideInfo.java | 13 +- .../sql/side/mongo/table/MongoSideParser.java | 12 +- .../side/mongo/table/MongoSideTableInfo.java | 4 +- .../flink/sql/side/mongo/utils/MongoUtil.java | 1 + .../sql/sink/mongo/MongoOutputFormat.java | 44 +- .../flink/sql/sink/mongo/MongoSink.java | 4 +- .../sql/sink/mongo/table/MongoSinkParser.java | 10 +- .../sql/sink/mongo/table/MongoTableInfo.java | 4 +- .../test/java/com/dtstack/flinkx/AppTest.java | 58 --- .../flink/sql/side/mysql/MysqlAllReqRow.java | 12 +- .../sql/side/mysql/MysqlAllSideInfo.java | 4 +- .../sql/side/mysql/MysqlAsyncReqRow.java | 11 +- .../sql/side/mysql/MysqlAsyncSideInfo.java | 4 +- .../sql/side/mysql/table/MysqlSideParser.java | 6 +- .../flink/sql/sink/mysql/MysqlSink.java | 6 +- .../sql/sink/mysql/table/MysqlSinkParser.java | 6 +- .../test/java/com/dtstack/flinkx/AppTest.java | 58 --- .../sql/side/oracle/OracleAllReqRow.java | 12 +- .../sql/side/oracle/OracleAllSideInfo.java | 5 +- .../sql/side/oracle/OracleAsyncReqRow.java | 11 +- .../sql/side/oracle/OracleAsyncSideInfo.java | 9 +- .../side/oracle/table/OracleSideParser.java | 6 +- .../flink/sql/sink/oracle/OracleDialect.java | 8 +- .../flink/sql/sink/oracle/OracleSink.java | 6 +- .../sink/oracle/table/OracleSinkParser.java | 6 +- .../sql/side/polardb/PolardbAllReqRow.java | 34 +- .../sql/side/polardb/PolardbAllSideInfo.java | 26 +- .../sql/side/polardb/PolardbAsyncReqRow.java | 28 +- .../side/polardb/PolardbAsyncSideInfo.java | 26 +- .../side/polardb/table/PolardbSideParser.java | 29 +- .../flink/sql/sink/polardb/PolardbSink.java | 30 +- .../sink/polardb/table/PolardbSinkParser.java | 28 +- pom.xml | 16 +- .../side/postgresql/PostgresqlAllReqRow.java | 12 +- .../postgresql/PostgresqlAllSideInfo.java | 4 +- .../postgresql/PostgresqlAsyncReqRow.java | 6 +- .../postgresql/PostgresqlAsyncSideInfo.java | 4 +- .../table/PostgresqlSideParser.java | 6 +- .../sql/sink/postgresql/PostgresqlSink.java | 6 +- .../table/PostgresqlSinkParser.java | 6 +- ...lReqRow.java => AbstractRdbAllReqRow.java} | 12 +- .../sql/side/rdb/all/RdbAllSideInfo.java | 15 +- .../sql/side/rdb/async/RdbAsyncReqRow.java | 28 +- .../sql/side/rdb/async/RdbAsyncSideInfo.java | 16 +- .../provider/DTC3P0DataSourceProvider.java | 10 +- .../sql/side/rdb/table/RdbSideParser.java | 8 +- .../sql/side/rdb/table/RdbSideTableInfo.java | 4 +- .../flink/sql/side/rdb/util/SwitchUtil.java | 1 + .../{RdbSink.java => AbstractRdbSink.java} | 19 +- .../flink/sql/sink/rdb/JDBCOptions.java | 22 +- .../rdb/format/AbstractJDBCOutputFormat.java | 14 +- .../rdb/format/JDBCUpsertOutputFormat.java | 14 +- .../rdb/format/RetractJDBCOutputFormat.java | 0 .../sql/sink/rdb/table/RdbSinkParser.java | 9 +- .../sql/sink/rdb/table/RdbTableInfo.java | 6 +- ...tWriter.java => AbstractUpsertWriter.java} | 86 ++-- .../sql/sink/rdb/writer/AppendOnlyWriter.java | 13 +- .../flink/sql/side/redis/RedisAllReqRow.java | 32 +- .../sql/side/redis/RedisAllSideInfo.java | 14 +- .../sql/side/redis/RedisAsyncReqRow.java | 26 +- .../sql/side/redis/RedisAsyncSideInfo.java | 14 +- .../sql/side/redis/table/RedisSideParser.java | 11 +- .../sql/side/redis/table/RedisSideReqRow.java | 8 +- .../side/redis/table/RedisSideTableInfo.java | 8 +- .../sql/sink/redis/RedisOutputFormat.java | 29 +- .../flink/sql/sink/redis/RedisSink.java | 8 +- .../sql/sink/redis/table/RedisSinkParser.java | 12 +- .../sql/sink/redis/table/RedisTableInfo.java | 8 +- .../CustomerSocketTextStreamFunction.java | 6 +- .../serversocket/ServersocketSource.java | 4 +- .../table/ServersocketSourceParser.java | 8 +- .../table/ServersocketSourceTableInfo.java | 6 +- .../side/sqlserver/SqlserverAllReqRow.java | 20 +- .../side/sqlserver/SqlserverAllSideInfo.java | 10 +- .../side/sqlserver/SqlserverAsyncReqRow.java | 10 +- .../sqlserver/SqlserverAsyncSideInfo.java | 11 +- .../sqlserver/table/SqlserverSideParser.java | 12 +- .../sql/sink/sqlserver/SqlserverSink.java | 6 +- .../sqlserver/table/SqlserverSinkParser.java | 6 +- 256 files changed, 3487 insertions(+), 2621 deletions(-) create mode 100644 .gitlab-ci.yml create mode 100644 ci/sonar_notify.sh rename core/src/main/java/com/dtstack/flink/sql/outputformat/{DtRichOutputFormat.java => AbstractDtRichOutputFormat.java} (95%) rename core/src/main/java/com/dtstack/flink/sql/side/{SideTableInfo.java => AbstractSideTableInfo.java} (97%) rename core/src/main/java/com/dtstack/flink/sql/side/{AllReqRow.java => BaseAllReqRow.java} (79%) rename core/src/main/java/com/dtstack/flink/sql/side/{AsyncReqRow.java => BaseAsyncReqRow.java} (89%) rename core/src/main/java/com/dtstack/flink/sql/side/{SideInfo.java => BaseSideInfo.java} (92%) rename core/src/main/java/com/dtstack/flink/sql/side/cache/{AbsSideCache.java => AbstractSideCache.java} (84%) rename core/src/main/java/com/dtstack/flink/sql/table/{AbsSideTableParser.java => AbstractSideTableParser.java} (68%) rename core/src/main/java/com/dtstack/flink/sql/table/{AbsSourceParser.java => AbstractSourceParser.java} (81%) rename core/src/main/java/com/dtstack/flink/sql/table/{SourceTableInfo.java => AbstractSourceTableInfo.java} (97%) rename core/src/main/java/com/dtstack/flink/sql/table/{TableInfo.java => AbstractTableInfo.java} (98%) rename core/src/main/java/com/dtstack/flink/sql/table/{TableInfoParser.java => AbstractTableInfoParser.java} (83%) rename core/src/main/java/com/dtstack/flink/sql/table/{AbsTableParser.java => AbstractTableParser.java} (88%) rename core/src/main/java/com/dtstack/flink/sql/table/{TargetTableInfo.java => AbstractTargetTableInfo.java} (94%) rename core/src/main/java/com/dtstack/flink/sql/watermarker/{AbsCustomerWaterMarker.java => AbstractCustomerWaterMarker.java} (94%) create mode 100644 docs/kafkaSink.md rename hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/{AbsRowKeyModeDealer.java => AbstractRowKeyModeDealer.java} (87%) rename hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/{ReplaceOperator.java => AbstractReplaceOperator.java} (93%) create mode 100644 kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaSink.java create mode 100644 kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/AvroCRowSerializationSchema.java create mode 100644 kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/CsvCRowSerializationSchema.java create mode 100644 kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/JsonCRowSerializationSchema.java create mode 100644 kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/AbstractKafkaSource.java rename cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.java => kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/enums/EKafkaOffset.java (56%) delete mode 100644 mongo/mongo-sink/src/test/java/com/dtstack/flinkx/AppTest.java delete mode 100644 mysql/mysql-sink/src/test/java/com/dtstack/flinkx/AppTest.java rename rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/{RdbAllReqRow.java => AbstractRdbAllReqRow.java} (96%) rename rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/{RdbSink.java => AbstractRdbSink.java} (90%) create mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java rename rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/{UpsertWriter.java => AbstractUpsertWriter.java} (84%) diff --git a/.gitlab-ci.yml b/.gitlab-ci.yml new file mode 100644 index 000000000..2952551ff --- /dev/null +++ b/.gitlab-ci.yml @@ -0,0 +1,10 @@ +build: + stage: test + script: + - mvn clean org.jacoco:jacoco-maven-plugin:0.7.8:prepare-agent package -Dmaven.test.failure.ignore=true -q + - mvn sonar:sonar -Dsonar.projectKey="dt-insight-engine/flinkStreamSQL" -Dsonar.branch.name="v1.8.0_dev" -Dsonar.login=11974c5e9a29625efa09fdc3c3fdc031efb1aab1 -Dsonar.host.url=http://172.16.100.198:9000 -Dsonar.jdbc.url=jdbc:postgresql://172.16.100.198:5432/sonar -Dsonar.java.binaries=target/sonar + - sh ci/sonar_notify.sh + only: + - v1.8.0_dev + tags: + - dt-insight-engine \ No newline at end of file diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java index bc0a0f8cd..3ad8b934e 100644 --- a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java @@ -34,10 +34,10 @@ import com.datastax.driver.core.SocketOptions; import com.datastax.driver.core.policies.DowngradingConsistencyRetryPolicy; import com.datastax.driver.core.policies.RetryPolicy; -import com.dtstack.flink.sql.side.AllReqRow; +import com.dtstack.flink.sql.side.BaseAllReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -62,14 +62,12 @@ * * @author xuqianjin */ -public class CassandraAllReqRow extends AllReqRow { +public class CassandraAllReqRow extends BaseAllReqRow { private static final long serialVersionUID = 54015343561288219L; private static final Logger LOG = LoggerFactory.getLogger(CassandraAllReqRow.class); - private static final String cassandra_DRIVER = "com.cassandra.jdbc.Driver"; - private static final int CONN_RETRY_NUM = 3; private static final int FETCH_SIZE = 1000; @@ -79,7 +77,7 @@ public class CassandraAllReqRow extends AllReqRow { private AtomicReference>>> cacheRef = new AtomicReference<>(); - public CassandraAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public CassandraAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new com.dtstack.flink.sql.side.cassandra.CassandraAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java index fa665f9a0..5d95dc9bf 100644 --- a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java @@ -20,8 +20,8 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.BaseSideInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlNode; @@ -37,16 +37,16 @@ * * @author xuqianjin */ -public class CassandraAllSideInfo extends SideInfo { +public class CassandraAllSideInfo extends BaseSideInfo { private static final long serialVersionUID = -8690814317653033557L; - public CassandraAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public CassandraAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + public void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo) { CassandraSideTableInfo cassandraSideTableInfo = (CassandraSideTableInfo) sideTableInfo; sqlCondition = "select ${selectField} from ${tableName} "; diff --git a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java index 0173f2d6f..7eee3cb5f 100644 --- a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java +++ b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java @@ -37,11 +37,11 @@ import com.datastax.driver.core.policies.DowngradingConsistencyRetryPolicy; import com.datastax.driver.core.policies.RetryPolicy; import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.AsyncReqRow; +import com.dtstack.flink.sql.side.BaseAsyncReqRow; import com.dtstack.flink.sql.side.CacheMissVal; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo; import com.google.common.base.Function; @@ -67,7 +67,7 @@ * * @author xuqianjin */ -public class CassandraAsyncReqRow extends AsyncReqRow { +public class CassandraAsyncReqRow extends BaseAsyncReqRow { private static final long serialVersionUID = 6631584128079864735L; @@ -83,7 +83,7 @@ public class CassandraAsyncReqRow extends AsyncReqRow { private transient ListenableFuture session; private transient CassandraSideTableInfo cassandraSideTableInfo; - public CassandraAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public CassandraAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new com.dtstack.flink.sql.side.cassandra.CassandraAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @@ -216,7 +216,7 @@ public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exce connCassandraDB(cassandraSideTableInfo); String sqlCondition = sideInfo.getSqlCondition() + " " + sqlWhere + " ALLOW FILTERING "; - System.out.println("sqlCondition:" + sqlCondition); + LOG.info("sqlCondition:{}" + sqlCondition); ListenableFuture resultSet = Futures.transformAsync(session, new AsyncFunction() { @@ -265,7 +265,6 @@ public void onSuccess(List rows) { public void onFailure(Throwable t) { LOG.error("Failed to retrieve the data: %s%n", t.getMessage()); - System.out.println("Failed to retrieve the data: " + t.getMessage()); cluster.closeAsync(); resultFuture.completeExceptionally(t); } diff --git a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java index 3557f0f73..4bee5648b 100644 --- a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java +++ b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java @@ -20,8 +20,8 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.BaseSideInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlBasicCall; @@ -30,6 +30,8 @@ import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; import com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.List; @@ -39,16 +41,18 @@ * * @author xuqianjin */ -public class CassandraAsyncSideInfo extends SideInfo { +public class CassandraAsyncSideInfo extends BaseSideInfo { private static final long serialVersionUID = -4403313049809013362L; + private static final Logger LOG = LoggerFactory.getLogger(CassandraAsyncSideInfo.class.getSimpleName()); - public CassandraAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + + public CassandraAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + public void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo) { CassandraSideTableInfo cassandraSideTableInfo = (CassandraSideTableInfo) sideTableInfo; String sideTableName = joinInfo.getSideTableName(); @@ -63,9 +67,9 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { } sqlCondition = "select ${selectField} from ${tableName}"; - sqlCondition = sqlCondition.replace("${tableName}", cassandraSideTableInfo.getDatabase()+"."+cassandraSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); - System.out.println("---------side_exe_sql-----\n" + sqlCondition); + + LOG.info("---------side_exe_sql-----\n{}" + sqlCondition); } diff --git a/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java b/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java index 62dd753b9..e893e56b0 100644 --- a/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java +++ b/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java @@ -19,18 +19,16 @@ package com.dtstack.flink.sql.side.cassandra.table; -import com.dtstack.flink.sql.table.AbsSideTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractSideTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; -import java.math.BigDecimal; -import java.sql.Date; import java.sql.Timestamp; import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; -import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; +import static com.dtstack.flink.sql.table.AbstractTableInfo.PARALLELISM_KEY; /** * Reason: @@ -38,7 +36,7 @@ * * @author xuqianjin */ -public class CassandraSideParser extends AbsSideTableParser { +public class CassandraSideParser extends AbstractSideTableParser { private final static String SIDE_SIGN_KEY = "sideSignKey"; @@ -73,7 +71,7 @@ public CassandraSideParser() { } @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo cassandraSideTableInfo = new com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo(); cassandraSideTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, cassandraSideTableInfo); @@ -96,9 +94,10 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map { +public class CassandraOutputFormat extends AbstractDtRichOutputFormat { private static final long serialVersionUID = -7994311331389155692L; private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormat.class); 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 eb7b23b53..26152a7d3 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 @@ -22,7 +22,7 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.cassandra.table.CassandraTableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -63,7 +63,7 @@ public CassandraSink() { } @Override - public CassandraSink genStreamSink(TargetTableInfo targetTableInfo) { + public CassandraSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { CassandraTableInfo cassandraTableInfo = (CassandraTableInfo) targetTableInfo; this.address = cassandraTableInfo.getAddress(); this.tableName = cassandraTableInfo.getTableName(); diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.java index 4c68e71ae..9ef8639ba 100644 --- a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.java +++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.java @@ -19,13 +19,13 @@ package com.dtstack.flink.sql.sink.cassandra.table; -import com.dtstack.flink.sql.table.AbsTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; -import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; +import static com.dtstack.flink.sql.table.AbstractTableInfo.PARALLELISM_KEY; /** * Reason: @@ -33,7 +33,7 @@ * * @author xuqianjin */ -public class CassandraSinkParser extends AbsTableParser { +public class CassandraSinkParser extends AbstractTableParser { public static final String ADDRESS_KEY = "address"; @@ -60,7 +60,7 @@ public class CassandraSinkParser extends AbsTableParser { public static final String POOL_TIMEOUT_MILLIS_KEY = "poolTimeoutMillis"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { CassandraTableInfo cassandraTableInfo = new CassandraTableInfo(); cassandraTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, cassandraTableInfo); diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java index c6626c42a..ffb5fa876 100644 --- a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java +++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.sink.cassandra.table; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import com.google.common.base.Preconditions; /** @@ -28,7 +28,7 @@ * * @author xuqianjin */ -public class CassandraTableInfo extends TargetTableInfo { +public class CassandraTableInfo extends AbstractTargetTableInfo { private static final String CURR_TYPE = "cassandra"; diff --git a/ci/sonar_notify.sh b/ci/sonar_notify.sh new file mode 100644 index 000000000..41f8a3c0e --- /dev/null +++ b/ci/sonar_notify.sh @@ -0,0 +1,14 @@ +#!/bin/bash +#参考钉钉文档 https://open-doc.dingtalk.com/microapp/serverapi2/qf2nxq + sonarreport=$(curl -s http://172.16.100.198:8082/?projectname=dt-insight-engine/flinkStreamSQL) + curl -s "https://oapi.dingtalk.com/robot/send?access_token=71555061297a53d3ac922a6f4d94285d8e23bccdca0c00b4dc6df0a2d49da724" \ + -H "Content-Type: application/json" \ + -d "{ + \"msgtype\": \"markdown\", + \"markdown\": { + \"title\":\"sonar代码质量\", + \"text\": \"## sonar代码质量报告: \n +> [sonar地址](http://172.16.100.198:9000/dashboard?id=dt-insight-engine/flinkStreamSQL) \n +> ${sonarreport} \n\" + } + }" \ No newline at end of file diff --git a/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java b/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java index 68c0c7984..c9a0c447b 100644 --- a/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java +++ b/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java @@ -20,40 +20,37 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; -import com.dtstack.flink.sql.util.DtStringUtil; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.AbstractRdbAllReqRow; import com.dtstack.flink.sql.util.JDBCUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.sql.Connection; import java.sql.DriverManager; import java.util.List; -import java.util.Map; -public class ClickhouseAllReqRow extends RdbAllReqRow { +public class ClickhouseAllReqRow extends AbstractRdbAllReqRow { private static final Logger LOG = LoggerFactory.getLogger(ClickhouseAllReqRow.class); private static final String CLICKHOUSE_DRIVER = "ru.yandex.clickhouse.ClickHouseDriver"; - public ClickhouseAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public ClickhouseAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new ClickhouseAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @Override - public Connection getConn(String dbURL, String userName, String passWord) { + public Connection getConn(String dbUrl, String userName, String passWord) { try { Connection connection ; JDBCUtils.forName(CLICKHOUSE_DRIVER, getClass().getClassLoader()); // ClickHouseProperties contains all properties if (userName == null) { - connection = DriverManager.getConnection(dbURL); + connection = DriverManager.getConnection(dbUrl); } else { - connection = DriverManager.getConnection(dbURL, userName, passWord); + connection = DriverManager.getConnection(dbUrl, userName, passWord); } return connection; } catch (Exception e) { diff --git a/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllSideInfo.java b/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllSideInfo.java index 973c069b9..43fbeaa56 100644 --- a/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllSideInfo.java +++ b/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllSideInfo.java @@ -20,7 +20,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -28,7 +28,7 @@ public class ClickhouseAllSideInfo extends RdbAllSideInfo { - public ClickhouseAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public ClickhouseAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } } diff --git a/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java b/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java index db49e4337..25478c0dd 100644 --- a/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java +++ b/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java @@ -21,7 +21,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncReqRow; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import io.vertx.core.Vertx; @@ -37,7 +37,7 @@ public class ClickhouseAsyncReqRow extends RdbAsyncReqRow { private static final String CLICKHOUSE_DRIVER = "ru.yandex.clickhouse.ClickHouseDriver"; - public ClickhouseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public ClickhouseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new ClickhouseAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @@ -62,7 +62,7 @@ public void open(Configuration parameters) throws Exception { vo.setWorkerPoolSize(rdbSideTableInfo.getAsyncPoolSize()); vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); - setRdbSQLClient(JDBCClient.createNonShared(vertx, clickhouseClientConfig)); + setRdbSqlClient(JDBCClient.createNonShared(vertx, clickhouseClientConfig)); } } diff --git a/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncSideInfo.java b/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncSideInfo.java index 254561de0..eec5fbe74 100644 --- a/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncSideInfo.java +++ b/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncSideInfo.java @@ -20,7 +20,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -29,7 +29,7 @@ public class ClickhouseAsyncSideInfo extends RdbAsyncSideInfo { - public ClickhouseAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public ClickhouseAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } } diff --git a/clickhouse/clickhouse-side/clickhouse-side-core/src/main/java/com/dtstack/flink/sql/side/clickhouse/table/ClickhouseSideParser.java b/clickhouse/clickhouse-side/clickhouse-side-core/src/main/java/com/dtstack/flink/sql/side/clickhouse/table/ClickhouseSideParser.java index 7be387fd8..ab285c37f 100644 --- a/clickhouse/clickhouse-side/clickhouse-side-core/src/main/java/com/dtstack/flink/sql/side/clickhouse/table/ClickhouseSideParser.java +++ b/clickhouse/clickhouse-side/clickhouse-side-core/src/main/java/com/dtstack/flink/sql/side/clickhouse/table/ClickhouseSideParser.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.side.clickhouse.table; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import ru.yandex.clickhouse.domain.ClickHouseDataType; import java.util.Map; @@ -38,8 +38,8 @@ public class ClickhouseSideParser extends RdbSideParser { private static final String CURR_TYPE = "clickhouse"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo clickhouseTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + AbstractTableInfo clickhouseTableInfo = super.getTableInfo(tableName, fieldsInfo, props); clickhouseTableInfo.setType(CURR_TYPE); return clickhouseTableInfo; } diff --git a/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/ClickhouseSink.java b/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/ClickhouseSink.java index bbb6a6fe4..5f7e41f7b 100644 --- a/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/ClickhouseSink.java +++ b/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/ClickhouseSink.java @@ -22,14 +22,11 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.JDBCOptions; -import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.AbstractRdbSink; import com.dtstack.flink.sql.sink.rdb.format.JDBCUpsertOutputFormat; -import java.util.List; -import java.util.Map; - -public class ClickhouseSink extends RdbSink implements IStreamSinkGener { +public class ClickhouseSink extends AbstractRdbSink implements IStreamSinkGener { public ClickhouseSink() { super(new ClickhouseDialect()); } @@ -37,7 +34,7 @@ public ClickhouseSink() { @Override public JDBCUpsertOutputFormat getOutputFormat() { JDBCOptions jdbcOptions = JDBCOptions.builder() - .setDBUrl(dbURL) + .setDbUrl(dbUrl) .setDialect(jdbcDialect) .setUsername(userName) .setPassword(password) diff --git a/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/table/ClickhouseSinkParser.java b/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/table/ClickhouseSinkParser.java index 8c3df93d7..5b0f2598f 100644 --- a/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/table/ClickhouseSinkParser.java +++ b/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/table/ClickhouseSinkParser.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.sink.clickhouse.table; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import ru.yandex.clickhouse.domain.ClickHouseDataType; import java.util.Map; @@ -30,8 +30,8 @@ public class ClickhouseSinkParser extends RdbSinkParser { private static final String CURR_TYPE = "clickhouse"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo clickhouseTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + AbstractTableInfo clickhouseTableInfo = super.getTableInfo(tableName, fieldsInfo, props); clickhouseTableInfo.setType(CURR_TYPE); return clickhouseTableInfo; } diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java index a397036ef..fc5a768c4 100644 --- a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.sink.console; -import com.dtstack.flink.sql.outputformat.DtRichOutputFormat; +import com.dtstack.flink.sql.outputformat.AbstractDtRichOutputFormat; import com.dtstack.flink.sql.sink.console.table.TablePrintUtil; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; @@ -37,7 +37,7 @@ * * @author xuqianjin */ -public class ConsoleOutputFormat extends DtRichOutputFormat { +public class ConsoleOutputFormat extends AbstractDtRichOutputFormat { private static final Logger LOG = LoggerFactory.getLogger(ConsoleOutputFormat.class); @@ -56,7 +56,7 @@ public void open(int taskNumber, int numTasks) throws IOException { @Override public void writeRecord(Tuple2 tuple2) throws IOException { - System.out.println("received oriainal data:" + tuple2); + LOG.info("received oriainal data:{}" + tuple2); Tuple2 tupleTrans = tuple2; Boolean retract = tupleTrans.getField(0); if (!retract) { diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleSink.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleSink.java index 77a3efea2..6cfdd2d3d 100644 --- a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleSink.java +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleSink.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.sink.console; import com.dtstack.flink.sql.sink.IStreamSinkGener; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -80,7 +80,7 @@ public void emitDataStream(DataStream> dataStream) { } @Override - public ConsoleSink genStreamSink(TargetTableInfo targetTableInfo) { + public ConsoleSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { return this; } } diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleSinkParser.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleSinkParser.java index e77444bfd..93ed02420 100644 --- a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleSinkParser.java +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleSinkParser.java @@ -18,13 +18,13 @@ package com.dtstack.flink.sql.sink.console.table; -import com.dtstack.flink.sql.table.AbsTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; -import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; +import static com.dtstack.flink.sql.table.AbstractTableInfo.PARALLELISM_KEY; /** * Reason: @@ -32,9 +32,9 @@ * * @author xuqianjin */ -public class ConsoleSinkParser extends AbsTableParser { +public class ConsoleSinkParser extends AbstractTableParser { @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { ConsoleTableInfo consoleTableInfo = new ConsoleTableInfo(); consoleTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, consoleTableInfo); diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleTableInfo.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleTableInfo.java index 4b286c667..6fd7063c6 100644 --- a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleTableInfo.java +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleTableInfo.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.sink.console.table; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; /** * Reason: @@ -26,7 +26,7 @@ * * @author xuqianjin */ -public class ConsoleTableInfo extends TargetTableInfo { +public class ConsoleTableInfo extends AbstractTargetTableInfo { private static final String CURR_TYPE = "console"; diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java index 8813da619..12bb2a5d4 100644 --- a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java @@ -22,6 +22,7 @@ public class TablePrintUtil { public static final int ALIGN_LEFT = 1;//左对齐 public static final int ALIGN_RIGHT = 2;//右对齐 public static final int ALIGN_CENTER = 3;//居中对齐 + private static final Pattern PATTERN = Pattern.compile("[\u4e00-\u9fa5]"); private int align = ALIGN_CENTER;//默认居中对齐 private boolean equilong = false;//默认不等宽 @@ -58,7 +59,9 @@ public static TablePrintUtil build(String[][] data) { public static TablePrintUtil build(List data) { TablePrintUtil self = new TablePrintUtil(); self.data = new ArrayList<>(); - if (data.size() <= 0) throw new RuntimeException("数据源至少得有一行吧"); + if (data.size() <= 0) { + throw new RuntimeException("数据源至少得有一行吧"); + } Object obj = data.get(0); @@ -70,7 +73,9 @@ public static TablePrintUtil build(List data) { int length = ((List) obj).size(); for (Object item : data) { List col = (List) item; - if (col.size() != length) throw new RuntimeException("数据源每列长度必须一致"); + if (col.size() != length) { + throw new RuntimeException("数据源每列长度必须一致"); + } self.data.add(col.toArray(new String[length])); } } else { @@ -90,7 +95,7 @@ public static TablePrintUtil build(List data) { try { value = obj.getClass().getMethod(colList.get(j).getMethodName).invoke(data.get(i)).toString(); } catch (IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { - e.printStackTrace(); + LOG.error("", e); } item[j] = value == null ? "null" : value; } @@ -115,7 +120,7 @@ private static List getColList(Object obj) { Method[] methods = obj.getClass().getMethods(); for (Method m : methods) { StringBuilder getMethodName = new StringBuilder(m.getName()); - if (getMethodName.substring(0, 3).equals("get") && !m.getName().equals("getClass")) { + if ("get".equals(getMethodName.substring(0, 3)) && !"getClass".equals(m.getName())) { Col col = new Col(); col.getMethodName = getMethodName.toString(); char first = Character.toLowerCase(getMethodName.delete(0, 3).charAt(0)); @@ -134,8 +139,8 @@ private static List getColList(Object obj) { * @return */ private int getStringCharLength(String str) { - Pattern p = Pattern.compile("[\u4e00-\u9fa5]");//利用正则找到中文 - Matcher m = p.matcher(str); + //利用正则找到中文 + Matcher m = PATTERN.matcher(str); int count = 0; while (m.find()) { count++; @@ -163,7 +168,9 @@ private int[] getColLengths() { if (equilong) {//如果等宽表格 int max = 0; for (int len : result) { - if (len > max) max = len; + if (len > max) { + max = len; + } } for (int i = 0; i < result.length; i++) { result[i] = max; @@ -213,6 +220,8 @@ public String getTableString() { sb.append(cell); for (int i = 0; i < right + padding; i++) {sb.append(s);} break; + default: + break; } sb.append(v); } diff --git a/core/pom.xml b/core/pom.xml index b17209612..eb470a3c0 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -20,7 +20,7 @@ 1.16.0 2.7.9 19.0 - 1.1.7 + 1.0.0-SNAPSHOT @@ -122,13 +122,11 @@ junit 4.12 - - ch.qos.logback - logback-classic - ${logback.version} + com.aiweiergou + tools-logger + ${logger.tool.version} - diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index b54c3feb2..fe925f9a6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -21,14 +21,12 @@ -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; - import com.dtstack.flink.sql.exec.ExecuteProcessHelper; import com.dtstack.flink.sql.exec.ParamsInfo; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - /** * Date: 2018/6/26 * Company: www.dtstack.com @@ -40,6 +38,7 @@ public class Main { public static void main(String[] args) throws Exception { ParamsInfo paramsInfo = ExecuteProcessHelper.parseParams(args); + ExecuteProcessHelper.setLogLevel(paramsInfo); StreamExecutionEnvironment env = ExecuteProcessHelper.getStreamExecution(paramsInfo); env.execute(paramsInfo.getName()); LOG.info("program {} execution success", paramsInfo.getName()); 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 c03f60617..2e62e11ab 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 @@ -98,9 +98,9 @@ public static List getClassPath() { - public static URLClassLoader loadExtraJar(List jarURLList, URLClassLoader classLoader) + public static URLClassLoader loadExtraJar(List jarUrlList, URLClassLoader classLoader) throws IllegalAccessException, InvocationTargetException { - for(URL url : jarURLList){ + for(URL url : jarUrlList){ if(url.toString().endsWith(".jar")){ urlClassLoaderAddUrl(classLoader, url); } diff --git a/core/src/main/java/com/dtstack/flink/sql/constrant/ConfigConstrant.java b/core/src/main/java/com/dtstack/flink/sql/constrant/ConfigConstrant.java index 00d1ea0a5..13e1de8c8 100644 --- a/core/src/main/java/com/dtstack/flink/sql/constrant/ConfigConstrant.java +++ b/core/src/main/java/com/dtstack/flink/sql/constrant/ConfigConstrant.java @@ -63,14 +63,21 @@ public class ConfigConstrant { public static final String CHECKPOINTS_DIRECTORY_KEY = "state.checkpoints.dir"; public static final String STATE_BACKEND_INCREMENTAL_KEY = "state.backend.incremental"; + public static final String RESTOREENABLE = "restore.enable"; + public static final String LOG_LEVEL_KEY = "logLevel"; // restart plocy - public static final int failureRate = 3; + public static final int FAILUEE_RATE = 3; + + public static final int FAILUEE_INTERVAL = 6; //min + + public static final int DELAY_INTERVAL = 10; //sec + + public static final String FAILUREINTERVAL = "failure.interval"; //min - public static final int failureInterval = 6; //min + public static final String DELAYINTERVAL= "delay.interval"; //sec - public static final int delayInterval = 10; //sec } 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 749bbc907..7f3f0019c 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 @@ -46,11 +46,66 @@ * @author huyifan.zju@163.com */ public enum ColumnType { - STRING, VARCHAR, CHAR, - INT, MEDIUMINT, TINYINT, DATETIME, SMALLINT, BIGINT, - DOUBLE, FLOAT, + /* + * string + */ + STRING, + /** + * varchar + */ + VARCHAR, + /** + * char + */ + CHAR, + /** + * int + */ + INT, + /** + * mediumint + */ + MEDIUMINT, + /** + * tinyint + */ + TINYINT, + /** + * datetime + */ + DATETIME, + /** + * smallint + */ + SMALLINT, + /** + * bigint + */ + BIGINT, + /** + * double + */ + DOUBLE, + /** + * float + */ + FLOAT, + /** + * boolean + */ BOOLEAN, - DATE, TIMESTAMP, DECIMAL; + /** + * date + */ + DATE, + /** + * timestamp + */ + TIMESTAMP, + /** + * decimal + */ + DECIMAL; public static ColumnType fromString(String type) { if(type == null) { diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/ECacheContentType.java b/core/src/main/java/com/dtstack/flink/sql/enums/ECacheContentType.java index 66160d820..49e352757 100644 --- a/core/src/main/java/com/dtstack/flink/sql/enums/ECacheContentType.java +++ b/core/src/main/java/com/dtstack/flink/sql/enums/ECacheContentType.java @@ -30,8 +30,17 @@ public enum ECacheContentType { + /** + * 无 + */ MissVal(0), + /** + * 1行 + */ SingleLine(1), + /** + * 多行 + */ MultiLine(2); int type; diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/ECacheType.java b/core/src/main/java/com/dtstack/flink/sql/enums/ECacheType.java index 582148c2c..9d5bb5d11 100644 --- a/core/src/main/java/com/dtstack/flink/sql/enums/ECacheType.java +++ b/core/src/main/java/com/dtstack/flink/sql/enums/ECacheType.java @@ -27,7 +27,18 @@ * @author xuchao */ public enum ECacheType { - NONE, LRU, ALL; + /** + * none + */ + NONE, + /** + * lru + */ + LRU, + /** + * all + */ + ALL; public static boolean isValid(String type){ for(ECacheType tmpType : ECacheType.values()){ diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/EDatabaseType.java b/core/src/main/java/com/dtstack/flink/sql/enums/EDatabaseType.java index 9b01bf052..7d2235f2e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/enums/EDatabaseType.java +++ b/core/src/main/java/com/dtstack/flink/sql/enums/EDatabaseType.java @@ -26,8 +26,17 @@ */ public enum EDatabaseType { + /** + * mysql + */ MYSQL, + /** + * sqlserver + */ SQLSERVER, + /** + * oracle + */ ORACLE, } diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/EPluginLoadMode.java b/core/src/main/java/com/dtstack/flink/sql/enums/EPluginLoadMode.java index 6cb027ac3..439966dd2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/enums/EPluginLoadMode.java +++ b/core/src/main/java/com/dtstack/flink/sql/enums/EPluginLoadMode.java @@ -26,7 +26,13 @@ */ public enum EPluginLoadMode { + /** + * 0:classpath + */ CLASSPATH(0), + /** + * 1:shipfile + */ SHIPFILE(1); private int type; diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/EStateBackend.java b/core/src/main/java/com/dtstack/flink/sql/enums/EStateBackend.java index a8f926175..098cb57fe 100644 --- a/core/src/main/java/com/dtstack/flink/sql/enums/EStateBackend.java +++ b/core/src/main/java/com/dtstack/flink/sql/enums/EStateBackend.java @@ -25,8 +25,17 @@ * @author maqi */ public enum EStateBackend { + /** + * memory + */ MEMORY, + /** + * rockdb + */ ROCKSDB, + /** + * filesystem + */ FILESYSTEM; public static EStateBackend convertFromString(String type) { diff --git a/core/src/main/java/com/dtstack/flink/sql/environment/StreamEnvConfigManager.java b/core/src/main/java/com/dtstack/flink/sql/environment/StreamEnvConfigManager.java index ce763ea99..d7b772c6e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/environment/StreamEnvConfigManager.java +++ b/core/src/main/java/com/dtstack/flink/sql/environment/StreamEnvConfigManager.java @@ -18,6 +18,12 @@ package com.dtstack.flink.sql.environment; +import com.dtstack.flink.sql.constrant.ConfigConstrant; +import com.dtstack.flink.sql.enums.EStateBackend; +import com.dtstack.flink.sql.util.MathUtil; +import com.dtstack.flink.sql.util.PropertiesUtils; +import org.apache.commons.lang3.BooleanUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; @@ -34,13 +40,6 @@ import org.apache.flink.table.api.StreamQueryConfig; import org.apache.flink.table.api.java.StreamTableEnvironment; -import com.dtstack.flink.sql.constrant.ConfigConstrant; -import com.dtstack.flink.sql.enums.EStateBackend; -import com.dtstack.flink.sql.util.MathUtil; -import com.dtstack.flink.sql.util.PropertiesUtils; -import org.apache.commons.lang3.BooleanUtils; -import org.apache.commons.lang3.StringUtils; - import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; @@ -102,11 +101,15 @@ public static void streamExecutionEnvironmentConfig(StreamExecutionEnvironment s } }); - streamEnv.setRestartStrategy(RestartStrategies.failureRateRestart( - ConfigConstrant.failureRate, - Time.of(ConfigConstrant.failureInterval, TimeUnit.MINUTES), - Time.of(ConfigConstrant.delayInterval, TimeUnit.SECONDS) - )); + if(isRestore(confProperties).get()){ + streamEnv.setRestartStrategy(RestartStrategies.failureRateRestart( + ConfigConstrant.FAILUEE_RATE, + Time.of(getFailureInterval(confProperties).get(), TimeUnit.MINUTES), + Time.of(getDelayInterval(confProperties).get(), TimeUnit.SECONDS) + )); + } else { + streamEnv.setRestartStrategy(RestartStrategies.noRestart()); + } // checkpoint config Optional checkpointingEnabled = isCheckpointingEnabled(confProperties); @@ -164,6 +167,20 @@ public static Optional getAutoWatermarkInterval(Properties properties) { return StringUtils.isNotBlank(autoWatermarkInterval) ? Optional.of(Long.valueOf(autoWatermarkInterval)) : Optional.empty(); } + public static Optional isRestore(Properties properties){ + String restoreEnable = properties.getProperty(ConfigConstrant.RESTOREENABLE, "true"); + return Optional.of(Boolean.valueOf(restoreEnable)); + } + + public static Optional getDelayInterval(Properties properties){ + String delayInterval = properties.getProperty(ConfigConstrant.DELAYINTERVAL, "10"); + return Optional.of(Integer.valueOf(delayInterval)); + } + public static Optional getFailureInterval(Properties properties){ + String failureInterval = properties.getProperty(ConfigConstrant.FAILUREINTERVAL, "6"); + return Optional.of(Integer.valueOf(failureInterval)); + } + /** * #ProcessingTime(默认), IngestionTime, EventTime * @param properties @@ -259,6 +276,8 @@ private static Optional createStateBackend(String backendType, Str checkpointDataUriEmptyCheck(checkpointDataUri, backendType); stateBackend = new RocksDBStateBackend(checkpointDataUri, BooleanUtils.toBoolean(backendIncremental)); break; + default: + break; } return stateBackend == null ? Optional.empty() : Optional.of(stateBackend); } @@ -318,14 +337,14 @@ private static void verityTtl(String ttlMintimeStr, String ttlMaxtimeStr) { * @return */ private static Long getTtlTime(Integer timeNumber, String timeUnit) { - if (timeUnit.equalsIgnoreCase("d")) { - return timeNumber * 1000l * 60 * 60 * 24; - } else if (timeUnit.equalsIgnoreCase("h")) { - return timeNumber * 1000l * 60 * 60; - } else if (timeUnit.equalsIgnoreCase("m")) { - return timeNumber * 1000l * 60; - } else if (timeUnit.equalsIgnoreCase("s")) { - return timeNumber * 1000l; + if ("d".equalsIgnoreCase(timeUnit)) { + return timeNumber * 1000L * 60 * 60 * 24; + } else if ("h".equalsIgnoreCase(timeUnit)) { + return timeNumber * 1000L * 60 * 60; + } else if ("m".equalsIgnoreCase(timeUnit)) { + return timeNumber * 1000L * 60; + } else if ("s".equalsIgnoreCase(timeUnit)) { + return timeNumber * 1000L; } else { throw new RuntimeException("not support " + timeNumber + timeUnit); } 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 3c9f37811..3858dc5c5 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 @@ -18,7 +18,13 @@ package com.dtstack.flink.sql.exec; -import com.dtstack.flink.sql.parser.*; +import com.aiweiergou.tool.logger.api.ChangeLogLevelProcess; +import com.dtstack.flink.sql.parser.CreateFuncParser; +import com.dtstack.flink.sql.parser.CreateTmpTableParser; +import com.dtstack.flink.sql.parser.FlinkPlanner; +import com.dtstack.flink.sql.parser.InsertSqlParser; +import com.dtstack.flink.sql.parser.SqlParser; +import com.dtstack.flink.sql.parser.SqlTree; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -33,6 +39,7 @@ import org.apache.flink.types.Row; import com.dtstack.flink.sql.classloader.ClassLoaderManager; +import com.dtstack.flink.sql.constrant.ConfigConstrant; import com.dtstack.flink.sql.enums.ClusterMode; import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.enums.EPluginLoadMode; @@ -42,12 +49,12 @@ import com.dtstack.flink.sql.option.OptionParser; import com.dtstack.flink.sql.option.Options; import com.dtstack.flink.sql.side.SideSqlExec; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.sink.StreamSinkFactory; import com.dtstack.flink.sql.source.StreamSourceFactory; -import com.dtstack.flink.sql.table.SourceTableInfo; -import com.dtstack.flink.sql.table.TableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractSourceTableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; import com.dtstack.flink.sql.watermarker.WaterMarkerAssigner; @@ -90,11 +97,8 @@ public class ExecuteProcessHelper { public static ParamsInfo parseParams(String[] args) throws Exception { LOG.info("------------program params-------------------------"); - System.out.println("------------program params-------------------------"); Arrays.stream(args).forEach(arg -> LOG.info("{}", arg)); - Arrays.stream(args).forEach(System.out::println); LOG.info("-------------------------------------------"); - System.out.println("----------------------------------------"); OptionParser optionParser = new OptionParser(args); Options options = optionParser.getOptions(); @@ -105,13 +109,14 @@ public static ParamsInfo parseParams(String[] args) throws Exception { String remoteSqlPluginPath = options.getRemoteSqlPluginPath(); String pluginLoadMode = options.getPluginLoadMode(); String deployMode = options.getMode(); + String logLevel = options.getLogLevel(); Preconditions.checkArgument(checkRemoteSqlPluginPath(remoteSqlPluginPath, deployMode, pluginLoadMode), "Non-local mode or shipfile deployment mode, remoteSqlPluginPath is required"); String confProp = URLDecoder.decode(options.getConfProp(), Charsets.UTF_8.toString()); Properties confProperties = PluginUtil.jsonStrToObject(confProp, Properties.class); - List jarURList = getExternalJarUrls(options.getAddjar()); + List jarUrlList = getExternalJarUrls(options.getAddjar()); return ParamsInfo.builder() .setSql(sql) @@ -121,7 +126,7 @@ public static ParamsInfo parseParams(String[] args) throws Exception { .setPluginLoadMode(pluginLoadMode) .setDeployMode(deployMode) .setConfProp(confProperties) - .setJarUrlList(jarURList) + .setJarUrlList(jarUrlList) .build(); } @@ -152,7 +157,7 @@ public static StreamExecutionEnvironment getStreamExecution(ParamsInfo paramsInf SqlParser.setLocalSqlPluginRoot(paramsInfo.getLocalSqlPluginPath()); SqlTree sqlTree = SqlParser.parseSql(paramsInfo.getSql()); - Map sideTableMap = Maps.newHashMap(); + Map sideTableMap = Maps.newHashMap(); Map registerTableCache = Maps.newHashMap(); //register udf @@ -185,14 +190,14 @@ public static List getExternalJarUrls(String addJarListStr) throws java.io. } return jarUrlList; } - + private static void sqlTranslation(String localSqlPluginPath, - StreamTableEnvironment tableEnv, - SqlTree sqlTree,Map sideTableMap, - Map registerTableCache, - StreamQueryConfig queryConfig) throws Exception { + StreamTableEnvironment tableEnv, + SqlTree sqlTree,Map sideTableMap, + Map registerTableCache, + StreamQueryConfig queryConfig) throws Exception { - SideSqlExec sideSqlExec = new SideSqlExec(); + SideSqlExec sideSqlExec = new SideSqlExec(); sideSqlExec.setLocalSqlPluginPath(localSqlPluginPath); for (CreateTmpTableParser.SqlParserResult result : sqlTree.getTmpSqlList()) { sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache, queryConfig, result); @@ -224,9 +229,8 @@ private static void sqlTranslation(String localSqlPluginPath, //sql-dimensional table contains the dimension table of execution sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache, queryConfig, null); } else { - System.out.println("----------exec sql without dimension join-----------"); - System.out.println("----------real sql exec is--------------------------"); - System.out.println(result.getExecSql()); + LOG.info("----------exec sql without dimension join-----------"); + LOG.info("----------real sql exec is--------------------------\n{}", result.getExecSql()); FlinkSQLExec.sqlUpdate(tableEnv, result.getExecSql(), queryConfig); if (LOG.isInfoEnabled()) { LOG.info("exec sql: " + result.getExecSql()); @@ -266,14 +270,14 @@ public static void registerUserDefinedFunction(SqlTree sqlTree, List jarUrl * @throws Exception */ public static Set registerTable(SqlTree sqlTree, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv, String localSqlPluginPath, - String remoteSqlPluginPath, String pluginLoadMode, Map sideTableMap, Map registerTableCache) throws Exception { + String remoteSqlPluginPath, String pluginLoadMode, Map sideTableMap, Map registerTableCache) throws Exception { Set pluginClassPatshSets = Sets.newHashSet(); WaterMarkerAssigner waterMarkerAssigner = new WaterMarkerAssigner(); - for (TableInfo tableInfo : sqlTree.getTableInfoMap().values()) { + for (AbstractTableInfo tableInfo : sqlTree.getTableInfoMap().values()) { - if (tableInfo instanceof SourceTableInfo) { + if (tableInfo instanceof AbstractSourceTableInfo) { - SourceTableInfo sourceTableInfo = (SourceTableInfo) tableInfo; + AbstractSourceTableInfo sourceTableInfo = (AbstractSourceTableInfo) tableInfo; Table table = StreamSourceFactory.getStreamSource(sourceTableInfo, env, tableEnv, localSqlPluginPath); tableEnv.registerTable(sourceTableInfo.getAdaptName(), table); //Note --- parameter conversion function can not be used inside a function of the type of polymerization @@ -304,21 +308,21 @@ public static Set registerTable(SqlTree sqlTree, StreamExecutionEnvironment } registerTableCache.put(tableInfo.getName(), regTable); - URL sourceTablePathUrl = PluginUtil.buildSourceAndSinkPathByLoadMode(tableInfo.getType(), SourceTableInfo.SOURCE_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode); + URL sourceTablePathUrl = PluginUtil.buildSourceAndSinkPathByLoadMode(tableInfo.getType(), AbstractSourceTableInfo.SOURCE_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode); pluginClassPatshSets.add(sourceTablePathUrl); - } else if (tableInfo instanceof TargetTableInfo) { + } else if (tableInfo instanceof AbstractTargetTableInfo) { - TableSink tableSink = StreamSinkFactory.getTableSink((TargetTableInfo) tableInfo, localSqlPluginPath); + TableSink tableSink = StreamSinkFactory.getTableSink((AbstractTargetTableInfo) tableInfo, localSqlPluginPath); TypeInformation[] flinkTypes = FunctionManager.transformTypes(tableInfo.getFieldClasses()); tableEnv.registerTableSink(tableInfo.getName(), tableInfo.getFields(), flinkTypes, tableSink); - URL sinkTablePathUrl = PluginUtil.buildSourceAndSinkPathByLoadMode(tableInfo.getType(), TargetTableInfo.TARGET_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode); + URL sinkTablePathUrl = PluginUtil.buildSourceAndSinkPathByLoadMode(tableInfo.getType(), AbstractTargetTableInfo.TARGET_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode); pluginClassPatshSets.add(sinkTablePathUrl); - } else if (tableInfo instanceof SideTableInfo) { - String sideOperator = ECacheType.ALL.name().equals(((SideTableInfo) tableInfo).getCacheType()) ? "all" : "async"; - sideTableMap.put(tableInfo.getName(), (SideTableInfo) tableInfo); + } else if (tableInfo instanceof AbstractSideTableInfo) { + String sideOperator = ECacheType.ALL.name().equals(((AbstractSideTableInfo) tableInfo).getCacheType()) ? "all" : "async"; + sideTableMap.put(tableInfo.getName(), (AbstractSideTableInfo) tableInfo); - URL sideTablePathUrl = PluginUtil.buildSidePathByLoadMode(tableInfo.getType(), sideOperator, SideTableInfo.TARGET_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode); + URL sideTablePathUrl = PluginUtil.buildSidePathByLoadMode(tableInfo.getType(), sideOperator, AbstractSideTableInfo.TARGET_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode); pluginClassPatshSets.add(sideTablePathUrl); } else { throw new RuntimeException("not support table type:" + tableInfo.getType()); @@ -351,4 +355,12 @@ public static StreamExecutionEnvironment getStreamExeEnv(Properties confProperti } + public static void setLogLevel(ParamsInfo paramsInfo){ + String logLevel = paramsInfo.getConfProp().getProperty(ConfigConstrant.LOG_LEVEL_KEY); + if(org.apache.commons.lang3.StringUtils.isBlank(logLevel)){ + return; + } + ChangeLogLevelProcess logLevelProcess = new ChangeLogLevelProcess(); + logLevelProcess.process(logLevel); + } } \ 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 b36dc31eb..27cc7702d 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 @@ -84,7 +84,6 @@ public Properties getConfProp() { return confProp; } - @Override public String toString() { return "ParamsInfo{" + diff --git a/core/src/main/java/com/dtstack/flink/sql/format/SerializationMetricWrapper.java b/core/src/main/java/com/dtstack/flink/sql/format/SerializationMetricWrapper.java index 3a5af18b1..8802198a0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/format/SerializationMetricWrapper.java +++ b/core/src/main/java/com/dtstack/flink/sql/format/SerializationMetricWrapper.java @@ -24,6 +24,7 @@ import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.MeterView; +import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.types.Row; @@ -34,11 +35,11 @@ * author: toutian * create: 2019/12/24 */ -public class SerializationMetricWrapper implements SerializationSchema { +public class SerializationMetricWrapper implements SerializationSchema { private static final long serialVersionUID = 1L; - private SerializationSchema serializationSchema; + private SerializationSchema serializationSchema; private transient RuntimeContext runtimeContext; @@ -47,7 +48,7 @@ public class SerializationMetricWrapper implements SerializationSchema { protected transient Meter dtNumRecordsOutRate; - public SerializationMetricWrapper(SerializationSchema serializationSchema) { + public SerializationMetricWrapper(SerializationSchema serializationSchema) { this.serializationSchema = serializationSchema; } @@ -57,7 +58,7 @@ public void initMetric() { } @Override - public byte[] serialize(Row element) { + public byte[] serialize(CRow element) { beforeSerialize(); byte[] row = serializationSchema.serialize(element); afterSerialize(); @@ -79,7 +80,7 @@ public void setRuntimeContext(RuntimeContext runtimeContext) { this.runtimeContext = runtimeContext; } - public SerializationSchema getSerializationSchema() { + public SerializationSchema getSerializationSchema() { return serializationSchema; } 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 78b579305..5f1b1c6f3 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 @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.format.dtnest; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.google.common.base.Strings; import com.google.common.collect.Maps; import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; @@ -28,8 +28,9 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.*; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.NullNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.flink.types.Row; import java.io.IOException; @@ -57,9 +58,9 @@ public class DtNestRowDeserializationSchema extends AbstractDeserializationSchem private final String[] fieldNames; private final TypeInformation[] fieldTypes; - private List fieldExtraInfos; + private List fieldExtraInfos; - public DtNestRowDeserializationSchema(TypeInformation typeInfo, Map rowAndFieldMapping, List fieldExtraInfos) { + public DtNestRowDeserializationSchema(TypeInformation typeInfo, Map rowAndFieldMapping, List fieldExtraInfos) { this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); this.rowAndFieldMapping = rowAndFieldMapping; @@ -75,7 +76,7 @@ public Row deserialize(byte[] message) throws IOException { try { for (int i = 0; i < fieldNames.length; i++) { JsonNode node = getIgnoreCase(fieldNames[i]); - TableInfo.FieldExtraInfo fieldExtraInfo = fieldExtraInfos.get(i); + AbstractTableInfo.FieldExtraInfo fieldExtraInfo = fieldExtraInfos.get(i); if (node == null) { if (fieldExtraInfo != null && fieldExtraInfo.getNotNull()) { diff --git a/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java b/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java index 43f599d14..0fd057c5a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java @@ -31,7 +31,10 @@ import java.io.File; import java.io.FileInputStream; import java.net.URLEncoder; +import java.util.stream.Stream; + import org.apache.commons.codec.Charsets; +import org.apache.flink.util.FileUtils; /** @@ -92,19 +95,16 @@ public Options getOptions(){ } public List getProgramExeArgList() throws Exception { - Map mapConf = PluginUtil.ObjectToMap(properties); + Map mapConf = PluginUtil.objectToMap(properties); List args = Lists.newArrayList(); - for(Map.Entry one : mapConf.entrySet()){ + for (Map.Entry one : mapConf.entrySet()) { String key = one.getKey(); Object value = one.getValue(); - if(value == null){ + if (value == null) { continue; - }else if(OPTION_SQL.equalsIgnoreCase(key)){ + } else if (OPTION_SQL.equalsIgnoreCase(key)) { File file = new File(value.toString()); - FileInputStream in = new FileInputStream(file); - byte[] filecontent = new byte[(int) file.length()]; - in.read(filecontent); - String content = new String(filecontent, Charsets.UTF_8.name()); + String content = FileUtils.readFile(file, "UTF-8"); value = URLEncoder.encode(content, Charsets.UTF_8.name()); } args.add("-" + key); @@ -112,9 +112,4 @@ public List getProgramExeArgList() throws Exception { } return args; } - - public static void main(String[] args) throws Exception { - OptionParser OptionParser = new OptionParser(args); - System.out.println(OptionParser.getOptions()); - } } diff --git a/core/src/main/java/com/dtstack/flink/sql/outputformat/DtRichOutputFormat.java b/core/src/main/java/com/dtstack/flink/sql/outputformat/AbstractDtRichOutputFormat.java similarity index 95% rename from core/src/main/java/com/dtstack/flink/sql/outputformat/DtRichOutputFormat.java rename to core/src/main/java/com/dtstack/flink/sql/outputformat/AbstractDtRichOutputFormat.java index 1fc40c13b..fbcc86bbd 100644 --- a/core/src/main/java/com/dtstack/flink/sql/outputformat/DtRichOutputFormat.java +++ b/core/src/main/java/com/dtstack/flink/sql/outputformat/AbstractDtRichOutputFormat.java @@ -27,7 +27,7 @@ * extend RichOutputFormat with metric 'dtNumRecordsOut', 'dtNumDirtyRecordsOut', 'dtNumRecordsOutRate' * Created by sishu.yss on 2018/11/28. */ -public abstract class DtRichOutputFormat extends RichOutputFormat{ +public abstract class AbstractDtRichOutputFormat extends RichOutputFormat{ public transient Counter outRecords; public transient Counter outDirtyRecords; diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java index 670d98a7e..fc6a7e16c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java @@ -32,18 +32,18 @@ public class CreateFuncParser implements IParser { - private static final String funcPatternStr = "(?i)\\s*create\\s+(scala|table|aggregate)\\s+function\\s+(\\S+)\\s+WITH\\s+(\\S+)"; + private static final String FUNC_PATTERN_STR = "(?i)\\s*create\\s+(scala|table|aggregate)\\s+function\\s+(\\S+)\\s+WITH\\s+(\\S+)"; - private static final Pattern funcPattern = Pattern.compile(funcPatternStr); + private static final Pattern FUNC_PATTERN = Pattern.compile(FUNC_PATTERN_STR); @Override public boolean verify(String sql) { - return funcPattern.matcher(sql).find(); + return FUNC_PATTERN.matcher(sql).find(); } @Override public void parseSql(String sql, SqlTree sqlTree) { - Matcher matcher = funcPattern.matcher(sql); + Matcher matcher = FUNC_PATTERN.matcher(sql); if(matcher.find()){ String type = matcher.group(1); String funcName = matcher.group(2); 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 a76c1b31a..2afc76c48 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 @@ -21,8 +21,8 @@ package com.dtstack.flink.sql.parser; import com.dtstack.flink.sql.enums.ETableType; -import com.dtstack.flink.sql.table.TableInfo; -import com.dtstack.flink.sql.table.TableInfoParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfoParser; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.commons.lang3.StringUtils; import com.google.common.collect.Lists; @@ -75,7 +75,7 @@ public static SqlTree parseSql(String sql) throws Exception { List sqlArr = DtStringUtil.splitIgnoreQuota(sql, SQL_DELIMITER); SqlTree sqlTree = new SqlTree(); - TableInfoParser tableInfoParser = new TableInfoParser(); + AbstractTableInfoParser tableInfoParser = new AbstractTableInfoParser(); for(String childSql : sqlArr){ if(Strings.isNullOrEmpty(childSql)){ continue; @@ -113,7 +113,7 @@ public static SqlTree parseSql(String sql) throws Exception { throw new RuntimeException("can't find table " + tableName); } - TableInfo tableInfo = tableInfoParser.parseWithTableType(ETableType.SOURCE.getType(), + AbstractTableInfo tableInfo = tableInfoParser.parseWithTableType(ETableType.SOURCE.getType(), createTableResult, LOCAL_SQL_PLUGIN_ROOT); sqlTree.addTableInfo(tableName, tableInfo); } @@ -126,7 +126,7 @@ public static SqlTree parseSql(String sql) throws Exception { throw new RuntimeException("can't find table " + tableName); } - TableInfo tableInfo = tableInfoParser.parseWithTableType(ETableType.SINK.getType(), + AbstractTableInfo tableInfo = tableInfoParser.parseWithTableType(ETableType.SINK.getType(), createTableResult, LOCAL_SQL_PLUGIN_ROOT); sqlTree.addTableInfo(tableName, tableInfo); } @@ -144,7 +144,7 @@ public static SqlTree parseSql(String sql) throws Exception { throw new RuntimeException("can't find table " + tableName); } } else { - TableInfo tableInfo = tableInfoParser.parseWithTableType(ETableType.SOURCE.getType(), + AbstractTableInfo tableInfo = tableInfoParser.parseWithTableType(ETableType.SOURCE.getType(), createTableResult, LOCAL_SQL_PLUGIN_ROOT); sqlTree.addTableInfo(tableName, tableInfo); } diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java index 1b64b7c68..5252ee022 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java @@ -21,7 +21,7 @@ package com.dtstack.flink.sql.parser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.google.common.collect.Maps; import com.google.common.collect.Lists; @@ -41,7 +41,7 @@ public class SqlTree { private Map preDealTableMap = Maps.newHashMap(); - private Map tableInfoMap = Maps.newLinkedHashMap(); + private Map tableInfoMap = Maps.newLinkedHashMap(); private List execSqlList = Lists.newArrayList(); @@ -89,11 +89,11 @@ public List getTmpSqlList(){ return tmpSqlList; } - public Map getTableInfoMap() { + public Map getTableInfoMap() { return tableInfoMap; } - public void addTableInfo(String tableName, TableInfo tableInfo){ + public void addTableInfo(String tableName, AbstractTableInfo tableInfo){ tableInfoMap.put(tableName, tableInfo); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java similarity index 97% rename from core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java rename to core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java index 32f065123..19dd67f9a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.side; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.google.common.collect.Lists; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -35,7 +35,7 @@ * @author xuchao */ -public abstract class SideTableInfo extends TableInfo implements Serializable { +public abstract class AbstractSideTableInfo extends AbstractTableInfo implements Serializable { public static final String TARGET_SUFFIX = "Side"; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/BaseAllReqRow.java similarity index 79% rename from core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java rename to core/src/main/java/com/dtstack/flink/sql/side/BaseAllReqRow.java index 64fd844b1..3e3d0a8fb 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/BaseAllReqRow.java @@ -25,13 +25,15 @@ import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.types.Row; import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.dtstack.flink.sql.factory.DTThreadFactory; import org.apache.calcite.sql.JoinType; import java.sql.SQLException; -import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; /** @@ -41,13 +43,15 @@ * @author xuchao */ -public abstract class AllReqRow extends RichFlatMapFunction implements ISideReqRow { +public abstract class BaseAllReqRow extends RichFlatMapFunction implements ISideReqRow { - protected SideInfo sideInfo; + private static final Logger LOG = LoggerFactory.getLogger(BaseAllReqRow.class); + + protected BaseSideInfo sideInfo; private ScheduledExecutorService es; - public AllReqRow(SideInfo sideInfo){ + public BaseAllReqRow(BaseSideInfo sideInfo){ this.sideInfo = sideInfo; } @@ -60,11 +64,11 @@ public AllReqRow(SideInfo sideInfo){ public void open(Configuration parameters) throws Exception { super.open(parameters); initCache(); - System.out.println("----- all cacheRef init end-----"); + LOG.info("----- all cacheRef init end-----"); //start reload cache thread - SideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); - es = Executors.newSingleThreadScheduledExecutor(new DTThreadFactory("cache-all-reload")); + AbstractSideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); + es = new ScheduledThreadPoolExecutor(1,new DTThreadFactory("cache-all-reload")); es.scheduleAtFixedRate(() -> reloadCache(), sideTableInfo.getCacheTimeout(), sideTableInfo.getCacheTimeout(), TimeUnit.MILLISECONDS); } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/BaseAsyncReqRow.java similarity index 89% rename from core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java rename to core/src/main/java/com/dtstack/flink/sql/side/BaseAsyncReqRow.java index 6f0dce58a..7ac4813a2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/BaseAsyncReqRow.java @@ -22,7 +22,7 @@ import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.metric.MetricConstant; -import com.dtstack.flink.sql.side.cache.AbsSideCache; +import com.dtstack.flink.sql.side.cache.AbstractSideCache; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.cache.LRUSideCache; import org.apache.calcite.sql.JoinType; @@ -30,14 +30,12 @@ import org.apache.flink.metrics.Counter; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; -import org.apache.flink.streaming.api.operators.async.queue.StreamRecordQueueEntry; import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Collections; -import java.util.concurrent.TimeoutException; /** * All interfaces inherit naming rules: type + "AsyncReqRow" such as == "MysqlAsyncReqRow @@ -47,17 +45,17 @@ * @author xuchao */ -public abstract class AsyncReqRow extends RichAsyncFunction implements ISideReqRow { - private static final Logger LOG = LoggerFactory.getLogger(AsyncReqRow.class); +public abstract class BaseAsyncReqRow extends RichAsyncFunction implements ISideReqRow { + private static final Logger LOG = LoggerFactory.getLogger(BaseAsyncReqRow.class); private static final long serialVersionUID = 2098635244857937717L; private static int TIMEOUT_LOG_FLUSH_NUM = 10; private int timeOutNum = 0; - protected SideInfo sideInfo; + protected BaseSideInfo sideInfo; protected transient Counter parseErrorRecords; - public AsyncReqRow(SideInfo sideInfo){ + public BaseAsyncReqRow(BaseSideInfo sideInfo){ this.sideInfo = sideInfo; } @@ -70,12 +68,12 @@ public void open(Configuration parameters) throws Exception { } private void initCache(){ - SideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); + AbstractSideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); if(sideTableInfo.getCacheType() == null || ECacheType.NONE.name().equalsIgnoreCase(sideTableInfo.getCacheType())){ return; } - AbsSideCache sideCache; + AbstractSideCache sideCache; if(ECacheType.LRU.name().equalsIgnoreCase(sideTableInfo.getCacheType())){ sideCache = new LRUSideCache(sideTableInfo); sideInfo.setSideCache(sideCache); @@ -124,10 +122,10 @@ protected void dealCacheData(String key, CacheObj missKeyObj) { @Override public void timeout(CRow input, ResultFuture resultFuture) throws Exception { - //TODO 需要添加数据指标 if(timeOutNum % TIMEOUT_LOG_FLUSH_NUM == 0){ LOG.info("Async function call has timed out. input:{}, timeOutNum:{}",input.toString(), timeOutNum); } + timeOutNum ++; if(timeOutNum > sideInfo.getSideTableInfo().getAsyncTimeoutNumLimit()){ resultFuture.completeExceptionally(new Exception("Async function call timedoutNum beyond limit.")); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/BaseSideInfo.java similarity index 92% rename from core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java rename to core/src/main/java/com/dtstack/flink/sql/side/BaseSideInfo.java index d8a22e022..732691d0e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/BaseSideInfo.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.side; -import com.dtstack.flink.sql.side.cache.AbsSideCache; +import com.dtstack.flink.sql.side.cache.AbstractSideCache; import org.apache.calcite.sql.JoinType; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlIdentifier; @@ -41,7 +41,7 @@ * @author xuchao */ -public abstract class SideInfo implements Serializable{ +public abstract class BaseSideInfo implements Serializable{ protected RowTypeInfo rowTypeInfo; @@ -68,12 +68,12 @@ public abstract class SideInfo implements Serializable{ //key:Returns the value of the position, value: the ref field name​in the side table protected Map sideFieldNameIndex = Maps.newHashMap(); - protected SideTableInfo sideTableInfo; + protected AbstractSideTableInfo sideTableInfo; - protected AbsSideCache sideCache; + protected AbstractSideCache sideCache; - public SideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, - SideTableInfo sideTableInfo){ + public BaseSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, + AbstractSideTableInfo sideTableInfo){ this.rowTypeInfo = rowTypeInfo; this.outFieldInfoList = outFieldInfoList; this.joinType = joinInfo.getJoinType(); @@ -173,7 +173,7 @@ public void dealOneEqualCon(SqlNode sqlNode, String sideTableName){ } } - public abstract void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo); + public abstract void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo); public RowTypeInfo getRowTypeInfo() { return rowTypeInfo; @@ -247,19 +247,19 @@ public void setSideFieldIndex(Map sideFieldIndex) { this.sideFieldIndex = sideFieldIndex; } - public SideTableInfo getSideTableInfo() { + public AbstractSideTableInfo getSideTableInfo() { return sideTableInfo; } - public void setSideTableInfo(SideTableInfo sideTableInfo) { + public void setSideTableInfo(AbstractSideTableInfo sideTableInfo) { this.sideTableInfo = sideTableInfo; } - public AbsSideCache getSideCache() { + public AbstractSideCache getSideCache() { return sideCache; } - public void setSideCache(AbsSideCache sideCache) { + public void setSideCache(AbstractSideCache sideCache) { this.sideCache = sideCache; } 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 544e8ebb0..d2bb2cb0d 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 @@ -22,6 +22,7 @@ import com.dtstack.flink.sql.parser.FlinkPlanner; import com.dtstack.flink.sql.util.ParseUtils; import com.dtstack.flink.sql.util.TableUtils; +import com.esotericsoftware.minlog.Log; import com.google.common.base.Preconditions; import com.google.common.collect.*; import org.apache.calcite.sql.JoinType; @@ -438,18 +439,18 @@ private void extractTemporaryQuery(SqlNode node, String tableAlias, tableRef.put(tbTmp, tableAlias); } - System.out.println("-------build temporary query-----------"); - System.out.println(tmpSelectSql); - System.out.println("---------------------------------------"); + Log.info("-------build temporary query-----------\n{}", tmpSelectSql); + Log.info("---------------------------------------"); }catch (Exception e){ - e.printStackTrace(); + Log.error("", e); throw new RuntimeException(e); } } /** * 抽取上层需用使用到的字段 + * 由于where字段已经抽取到上一层了所以不用查询出来 * @param parentSelectList * @param fromTableNameSet * @return @@ -592,7 +593,6 @@ private void extractSelectField(SqlNode selectNode, } }else if(selectNode.getKind() == CASE){ - System.out.println("selectNode"); SqlCase sqlCase = (SqlCase) selectNode; SqlNodeList whenOperands = sqlCase.getWhenOperands(); SqlNodeList thenOperands = sqlCase.getThenOperands(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java index 738e0f84e..d9d79f34e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java @@ -80,6 +80,7 @@ public static List getRowTypeInfo(SqlNode sqlNode, JoinScope scope, b switch(identifierSize) { case 1: fieldInfoList.addAll(getAllField(scope)); + break; default: SqlIdentifier tableIdentify = identifier.skipLast(1); JoinScope.ScopeChild scopeChild = scope.getScope(tableIdentify.getSimple()); @@ -99,6 +100,7 @@ public static List getRowTypeInfo(SqlNode sqlNode, JoinScope scope, b fieldInfo.setTypeInformation(type); fieldInfoList.add(fieldInfo); } + break; } } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java index 4d6112aa8..fa973508f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SidePredicatesParser.java @@ -46,7 +46,7 @@ * @author maqi */ public class SidePredicatesParser { - public void fillPredicatesForSideTable(String exeSql, Map sideTableMap) throws SqlParseException { + public void fillPredicatesForSideTable(String exeSql, Map sideTableMap) throws SqlParseException { FlinkPlannerImpl flinkPlanner = FlinkPlanner.getFlinkPlanner(); SqlNode sqlNode = flinkPlanner.parse(exeSql); parseSql(sqlNode, sideTableMap, Maps.newHashMap()); @@ -58,7 +58,7 @@ public void fillPredicatesForSideTable(String exeSql, Map * @param sideTableMap * @param tabMapping 谓词属性中别名对应的真实维表名称 */ - private void parseSql(SqlNode sqlNode, Map sideTableMap, Map tabMapping) { + private void parseSql(SqlNode sqlNode, Map sideTableMap, Map tabMapping) { SqlKind sqlKind = sqlNode.getKind(); switch (sqlKind) { case INSERT: @@ -101,10 +101,12 @@ private void parseSql(SqlNode sqlNode, Map sideTableMap, parseSql(unionLeft, sideTableMap, tabMapping); parseSql(unionRight, sideTableMap, tabMapping); break; + default: + break; } } - private void fillToSideTableInfo(Map sideTableMap, Map tabMapping, List predicateInfoList) { + private void fillToSideTableInfo(Map sideTableMap, Map tabMapping, List predicateInfoList) { predicateInfoList.stream().filter(info -> sideTableMap.containsKey(tabMapping.getOrDefault(info.getOwnerTable(), info.getOwnerTable()))) .map(info -> sideTableMap.get(tabMapping.getOrDefault(info.getOwnerTable(), info.getOwnerTable())).getPredicateInfoes().add(info)) .count(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index f37c3f78c..2296f64c5 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -47,7 +47,8 @@ import java.util.Queue; import java.util.Set; -import static org.apache.calcite.sql.SqlKind.*; +import static org.apache.calcite.sql.SqlKind.IDENTIFIER; +import static org.apache.calcite.sql.SqlKind.LITERAL; /** * Parsing sql, obtain execution information dimension table @@ -62,8 +63,6 @@ public class SideSQLParser { private Map localTableCache = Maps.newHashMap(); public Queue getExeQueue(String exeSql, Set sideTableSet) throws SqlParseException { - System.out.println("----------exec original Sql----------"); - System.out.println(exeSql); LOG.info("----------exec original Sql----------"); LOG.info(exeSql); @@ -168,6 +167,8 @@ public Object parseSql(SqlNode sqlNode, case LITERAL: return LITERAL.toString(); + default: + break; } return ""; } 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 f90138b2a..c41e2264a 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 @@ -43,24 +43,17 @@ import com.dtstack.flink.sql.util.ParseUtils; import com.dtstack.flink.sql.util.TableUtils; import com.google.common.base.Preconditions; -import com.google.common.collect.*; -import org.apache.calcite.sql.SqlAsOperator; +import com.google.common.collect.HashBasedTable; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlDataTypeSpec; import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlInsert; -import org.apache.calcite.sql.SqlJoin; import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlLiteral; import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlNodeList; -import org.apache.calcite.sql.SqlOperator; -import org.apache.calcite.sql.SqlOrderBy; import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.SqlWithItem; -import org.apache.calcite.sql.fun.SqlCase; import org.apache.calcite.sql.parser.SqlParseException; -import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -68,7 +61,6 @@ import java.sql.Timestamp; import java.util.Arrays; -import java.util.Collection; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -97,7 +89,7 @@ public class SideSqlExec { private Map localTableCache = Maps.newHashMap(); public void exec(String sql, - Map sideTableMap, + Map sideTableMap, StreamTableEnvironment tableEnv, Map tableCache, StreamQueryConfig queryConfig, @@ -131,11 +123,9 @@ public void exec(String sql, if(pollSqlNode.getKind() == INSERT){ - System.out.println("----------real exec sql-----------" ); - System.out.println(pollSqlNode.toString()); FlinkSQLExec.sqlUpdate(tableEnv, pollSqlNode.toString(), queryConfig); if(LOG.isInfoEnabled()){ - LOG.info("exec sql: " + pollSqlNode.toString()); + LOG.info("----------real exec sql-----------\n{}", pollSqlNode.toString()); } }else if(pollSqlNode.getKind() == AS){ @@ -166,8 +156,7 @@ public void exec(String sql, } }else if (pollObj instanceof JoinInfo){ - System.out.println("----------exec join info----------"); - System.out.println(pollObj.toString()); + LOG.info("----------exec join info----------\n{}", pollObj.toString()); joinFun(pollObj, localTableCache, sideTableMap, tableEnv); } } @@ -294,7 +283,7 @@ private Table getTableFromCache(Map localTableCache, String table * * @return */ - private boolean checkJoinCondition(SqlNode conditionNode, String sideTableAlias, SideTableInfo sideTableInfo) { + private boolean checkJoinCondition(SqlNode conditionNode, String sideTableAlias, AbstractSideTableInfo sideTableInfo) { List conditionFields = getConditionFields(conditionNode, sideTableAlias, sideTableInfo); if(CollectionUtils.isEqualCollection(conditionFields, convertPrimaryAlias(sideTableInfo))){ return true; @@ -302,7 +291,7 @@ private boolean checkJoinCondition(SqlNode conditionNode, String sideTableAlias, return false; } - private List convertPrimaryAlias(SideTableInfo sideTableInfo) { + private List convertPrimaryAlias(AbstractSideTableInfo sideTableInfo) { List res = Lists.newArrayList(); sideTableInfo.getPrimaryKeys().forEach(field -> { res.add(sideTableInfo.getPhysicalFields().getOrDefault(field, field)); @@ -310,7 +299,7 @@ private List convertPrimaryAlias(SideTableInfo sideTableInfo) { return res; } - public List getConditionFields(SqlNode conditionNode, String specifyTableName, SideTableInfo sideTableInfo){ + public List getConditionFields(SqlNode conditionNode, String specifyTableName, AbstractSideTableInfo sideTableInfo){ List sqlNodeList = Lists.newArrayList(); ParseUtils.parseAnd(conditionNode, sqlNodeList); List conditionFields = Lists.newArrayList(); @@ -369,7 +358,7 @@ protected void dealAsSourceTable(StreamTableEnvironment tableEnv, private void joinFun(Object pollObj, Map localTableCache, - Map sideTableMap, + Map sideTableMap, StreamTableEnvironment tableEnv) throws Exception{ JoinInfo joinInfo = (JoinInfo) pollObj; @@ -385,7 +374,7 @@ private void joinFun(Object pollObj, JoinScope.ScopeChild rightScopeChild = new JoinScope.ScopeChild(); rightScopeChild.setAlias(joinInfo.getRightTableAlias()); rightScopeChild.setTableName(joinInfo.getRightTableName()); - SideTableInfo sideTableInfo = sideTableMap.get(joinInfo.getRightTableName()); + AbstractSideTableInfo sideTableInfo = sideTableMap.get(joinInfo.getRightTableName()); if(sideTableInfo == null){ sideTableInfo = sideTableMap.get(joinInfo.getRightTableAlias()); } @@ -394,9 +383,9 @@ private void joinFun(Object pollObj, throw new RuntimeException("can't not find side table:" + joinInfo.getRightTableName()); } -// if(!checkJoinCondition(joinInfo.getCondition(), joinInfo.getRightTableAlias(), sideTableInfo)){ -// throw new RuntimeException("ON condition must contain all equal fields!!!"); -// } + if(!checkJoinCondition(joinInfo.getCondition(), joinInfo.getRightTableAlias(), sideTableInfo)){ + throw new RuntimeException("ON condition must contain all equal fields!!!"); + } rightScopeChild.setRowTypeInfo(sideTableInfo.getRowTypeInfo()); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java b/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java index 8417c4519..583e4597b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java @@ -21,8 +21,8 @@ import com.dtstack.flink.sql.classloader.ClassLoaderManager; import com.dtstack.flink.sql.enums.ECacheType; -import com.dtstack.flink.sql.table.AbsSideTableParser; -import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.AbstractSideTableParser; +import com.dtstack.flink.sql.table.AbstractTableParser; import com.dtstack.flink.sql.util.PluginUtil; /** @@ -37,7 +37,7 @@ public class StreamSideFactory { private static final String CURR_TYPE = "side"; - public static AbsTableParser getSqlParser(String pluginType, String sqlRootDir, String cacheType) throws Exception { + public static AbstractTableParser getSqlParser(String pluginType, String sqlRootDir, String cacheType) throws Exception { String sideOperator = ECacheType.ALL.name().equalsIgnoreCase(cacheType) ? "all" : "async"; String pluginJarPath = PluginUtil.getSideJarFileDirPath(pluginType, sideOperator, "side", sqlRootDir); @@ -45,10 +45,10 @@ public static AbsTableParser getSqlParser(String pluginType, String sqlRootDir, return ClassLoaderManager.newInstance(pluginJarPath, (cl) -> { Class sideParser = cl.loadClass(className); - if (!AbsSideTableParser.class.isAssignableFrom(sideParser)) { + if (!AbstractSideTableParser.class.isAssignableFrom(sideParser)) { throw new RuntimeException("class " + sideParser.getName() + " not subClass of AbsSideTableParser"); } - return sideParser.asSubclass(AbsTableParser.class).newInstance(); + return sideParser.asSubclass(AbstractTableParser.class).newInstance(); }); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/cache/AbsSideCache.java b/core/src/main/java/com/dtstack/flink/sql/side/cache/AbstractSideCache.java similarity index 84% rename from core/src/main/java/com/dtstack/flink/sql/side/cache/AbsSideCache.java rename to core/src/main/java/com/dtstack/flink/sql/side/cache/AbstractSideCache.java index 757f91600..58832c583 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/cache/AbsSideCache.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/cache/AbstractSideCache.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.side.cache; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; /** * Reason: @@ -30,11 +30,11 @@ * @author xuchao */ -public abstract class AbsSideCache { +public abstract class AbstractSideCache { - protected SideTableInfo sideTableInfo; + protected AbstractSideTableInfo sideTableInfo; - public AbsSideCache(SideTableInfo sideTableInfo){ + public AbstractSideCache(AbstractSideTableInfo sideTableInfo){ this.sideTableInfo = sideTableInfo; } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/cache/LRUSideCache.java b/core/src/main/java/com/dtstack/flink/sql/side/cache/LRUSideCache.java index 700e13bb2..2664efd34 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/cache/LRUSideCache.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/cache/LRUSideCache.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.side.cache; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; @@ -33,11 +33,11 @@ * @author xuchao */ -public class LRUSideCache extends AbsSideCache{ +public class LRUSideCache extends AbstractSideCache { protected transient Cache cache; - public LRUSideCache(SideTableInfo sideTableInfo) { + public LRUSideCache(AbstractSideTableInfo sideTableInfo) { super(sideTableInfo); } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java index 290804200..3c2010907 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java @@ -20,10 +20,10 @@ package com.dtstack.flink.sql.side.operator; import com.dtstack.flink.sql.classloader.ClassLoaderManager; -import com.dtstack.flink.sql.side.AsyncReqRow; +import com.dtstack.flink.sql.side.BaseAsyncReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.AsyncDataStream; @@ -49,20 +49,20 @@ public class SideAsyncOperator { private static final String ORDERED = "ordered"; - private static AsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, - JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { + private static BaseAsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, + JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) throws Exception { String pathOfType = String.format(PATH_FORMAT, sideType); String pluginJarPath = PluginUtil.getJarFileDirPath(pathOfType, sqlRootDir); String className = PluginUtil.getSqlSideClassName(sideType, "side", OPERATOR_TYPE); return ClassLoaderManager.newInstance(pluginJarPath, (cl) -> - cl.loadClass(className).asSubclass(AsyncReqRow.class) - .getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class) + cl.loadClass(className).asSubclass(BaseAsyncReqRow.class) + .getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, AbstractSideTableInfo.class) .newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } public static DataStream getSideJoinDataStream(DataStream inputStream, String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, - List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { - AsyncReqRow asyncDbReq = loadAsyncReq(sideType, sqlRootDir, rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + List outFieldInfoList, AbstractSideTableInfo sideTableInfo) throws Exception { + BaseAsyncReqRow asyncDbReq = loadAsyncReq(sideType, sqlRootDir, rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); //TODO How much should be set for the degree of parallelism? Timeout? capacity settings? if (ORDERED.equals(sideTableInfo.getCacheMode())){ 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 5aa810b0f..6b6f9fe1b 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 @@ -20,10 +20,10 @@ package com.dtstack.flink.sql.side.operator; import com.dtstack.flink.sql.classloader.ClassLoaderManager; -import com.dtstack.flink.sql.side.AllReqRow; +import com.dtstack.flink.sql.side.BaseAllReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; @@ -44,22 +44,22 @@ public class SideWithAllCacheOperator { private static final String OPERATOR_TYPE = "All"; - private static AllReqRow loadFlatMap(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, - JoinInfo joinInfo, List outFieldInfoList, - SideTableInfo sideTableInfo) throws Exception { + private static BaseAllReqRow loadFlatMap(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, + JoinInfo joinInfo, List outFieldInfoList, + AbstractSideTableInfo sideTableInfo) throws Exception { String pathOfType = String.format(PATH_FORMAT, sideType); String pluginJarPath = PluginUtil.getJarFileDirPath(pathOfType, sqlRootDir); String className = PluginUtil.getSqlSideClassName(sideType, "side", OPERATOR_TYPE); - return ClassLoaderManager.newInstance(pluginJarPath, (cl) -> cl.loadClass(className).asSubclass(AllReqRow.class) - .getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class) + return ClassLoaderManager.newInstance(pluginJarPath, (cl) -> cl.loadClass(className).asSubclass(BaseAllReqRow.class) + .getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, AbstractSideTableInfo.class) .newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } public static DataStream getSideJoinDataStream(DataStream inputStream, String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, - List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { - AllReqRow allReqRow = loadFlatMap(sideType, sqlRootDir, rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + List outFieldInfoList, AbstractSideTableInfo sideTableInfo) throws Exception { + BaseAllReqRow allReqRow = loadFlatMap(sideType, sqlRootDir, rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); return inputStream.flatMap(allReqRow); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/sink/IStreamSinkGener.java b/core/src/main/java/com/dtstack/flink/sql/sink/IStreamSinkGener.java index 3cfd48f1b..e29421369 100644 --- a/core/src/main/java/com/dtstack/flink/sql/sink/IStreamSinkGener.java +++ b/core/src/main/java/com/dtstack/flink/sql/sink/IStreamSinkGener.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.sink; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; /** * Reason: @@ -30,5 +30,5 @@ */ public interface IStreamSinkGener { - T genStreamSink(TargetTableInfo targetTableInfo); + T genStreamSink(AbstractTargetTableInfo targetTableInfo); } diff --git a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java index 53460081d..89061db3a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java @@ -21,8 +21,8 @@ package com.dtstack.flink.sql.sink; import com.dtstack.flink.sql.classloader.ClassLoaderManager; -import com.dtstack.flink.sql.table.AbsTableParser; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTableParser; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.flink.table.sinks.TableSink; @@ -40,21 +40,21 @@ public class StreamSinkFactory { private static final String DIR_NAME_FORMAT = "%ssink"; - public static AbsTableParser getSqlParser(String pluginType, String sqlRootDir) throws Exception { + public static AbstractTableParser getSqlParser(String pluginType, String sqlRootDir) throws Exception { String pluginJarPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, pluginType), sqlRootDir); String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(pluginType); String className = PluginUtil.getSqlParserClassName(typeNoVersion, CURR_TYPE); return ClassLoaderManager.newInstance(pluginJarPath, (cl) -> { Class targetParser = cl.loadClass(className); - if(!AbsTableParser.class.isAssignableFrom(targetParser)){ + if(!AbstractTableParser.class.isAssignableFrom(targetParser)){ throw new RuntimeException("class " + targetParser.getName() + " not subClass of AbsTableParser"); } - return targetParser.asSubclass(AbsTableParser.class).newInstance(); + return targetParser.asSubclass(AbstractTableParser.class).newInstance(); }); } - public static TableSink getTableSink(TargetTableInfo targetTableInfo, String localSqlRootDir) throws Exception { + public static TableSink getTableSink(AbstractTargetTableInfo targetTableInfo, String localSqlRootDir) throws Exception { String pluginType = targetTableInfo.getType(); String pluginJarDirPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, pluginType), localSqlRootDir); String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(pluginType); diff --git a/core/src/main/java/com/dtstack/flink/sql/source/IStreamSourceGener.java b/core/src/main/java/com/dtstack/flink/sql/source/IStreamSourceGener.java index b8b8f6edc..fde5a235d 100644 --- a/core/src/main/java/com/dtstack/flink/sql/source/IStreamSourceGener.java +++ b/core/src/main/java/com/dtstack/flink/sql/source/IStreamSourceGener.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.source; -import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.table.AbstractSourceTableInfo; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.java.StreamTableEnvironment; @@ -38,6 +38,6 @@ public interface IStreamSourceGener { * @param tableEnv * @return */ - T genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv); + T genStreamSource(AbstractSourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv); } diff --git a/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java b/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java index 1057fb0ed..e0cec1415 100644 --- a/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java @@ -22,8 +22,8 @@ import com.dtstack.flink.sql.classloader.ClassLoaderManager; -import com.dtstack.flink.sql.table.AbsSourceParser; -import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.table.AbstractSourceParser; +import com.dtstack.flink.sql.table.AbstractSourceTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -43,17 +43,17 @@ public class StreamSourceFactory { private static final String DIR_NAME_FORMAT = "%ssource"; - public static AbsSourceParser getSqlParser(String pluginType, String sqlRootDir) throws Exception { + public static AbstractSourceParser getSqlParser(String pluginType, String sqlRootDir) throws Exception { String pluginJarPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, pluginType), sqlRootDir); String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(pluginType); String className = PluginUtil.getSqlParserClassName(typeNoVersion, CURR_TYPE); return ClassLoaderManager.newInstance(pluginJarPath, (cl) -> { Class sourceParser = cl.loadClass(className); - if(!AbsSourceParser.class.isAssignableFrom(sourceParser)){ + if(!AbstractSourceParser.class.isAssignableFrom(sourceParser)){ throw new RuntimeException("class " + sourceParser.getName() + " not subClass of AbsSourceParser"); } - return sourceParser.asSubclass(AbsSourceParser.class).newInstance(); + return sourceParser.asSubclass(AbstractSourceParser.class).newInstance(); }); } @@ -62,7 +62,7 @@ public static AbsSourceParser getSqlParser(String pluginType, String sqlRootDir) * @param sourceTableInfo * @return */ - public static Table getStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, + public static Table getStreamSource(AbstractSourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv, String sqlRootDir) throws Exception { String sourceTypeStr = sourceTableInfo.getType(); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java similarity index 68% rename from core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java rename to core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java index 403360400..4973a891e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java @@ -21,7 +21,7 @@ package com.dtstack.flink.sql.table; import com.dtstack.flink.sql.enums.ECacheType; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.util.MathUtil; import org.apache.flink.util.Preconditions; @@ -36,24 +36,24 @@ * @author xuchao */ -public abstract class AbsSideTableParser extends AbsTableParser { +public abstract class AbstractSideTableParser extends AbstractTableParser { private final static String SIDE_SIGN_KEY = "sideSignKey"; private final static Pattern SIDE_TABLE_SIGN = Pattern.compile("(?i)^PERIOD\\s+FOR\\s+SYSTEM_TIME$"); - public AbsSideTableParser() { + public AbstractSideTableParser() { addParserHandler(SIDE_SIGN_KEY, SIDE_TABLE_SIGN, this::dealSideSign); } - private void dealSideSign(Matcher matcher, TableInfo tableInfo){ + private void dealSideSign(Matcher matcher, AbstractTableInfo tableInfo){ //FIXME SIDE_TABLE_SIGN current just used as a sign for side table; and do nothing } //Analytical create table attributes ==> Get information cache - protected void parseCacheProp(SideTableInfo sideTableInfo, Map props){ - if(props.containsKey(SideTableInfo.CACHE_KEY.toLowerCase())){ - String cacheType = MathUtil.getString(props.get(SideTableInfo.CACHE_KEY.toLowerCase())); + protected void parseCacheProp(AbstractSideTableInfo sideTableInfo, Map props){ + if(props.containsKey(AbstractSideTableInfo.CACHE_KEY.toLowerCase())){ + String cacheType = MathUtil.getString(props.get(AbstractSideTableInfo.CACHE_KEY.toLowerCase())); if(cacheType == null){ return; } @@ -63,65 +63,66 @@ protected void parseCacheProp(SideTableInfo sideTableInfo, Map p } sideTableInfo.setCacheType(cacheType); - if(props.containsKey(SideTableInfo.CACHE_SIZE_KEY.toLowerCase())){ - Integer cacheSize = MathUtil.getIntegerVal(props.get(SideTableInfo.CACHE_SIZE_KEY.toLowerCase())); + if(props.containsKey(AbstractSideTableInfo.CACHE_SIZE_KEY.toLowerCase())){ + Integer cacheSize = MathUtil.getIntegerVal(props.get(AbstractSideTableInfo.CACHE_SIZE_KEY.toLowerCase())); if(cacheSize < 0){ throw new RuntimeException("cache size need > 0."); } sideTableInfo.setCacheSize(cacheSize); } - if(props.containsKey(SideTableInfo.CACHE_TTLMS_KEY.toLowerCase())){ - Long cacheTTLMS = MathUtil.getLongVal(props.get(SideTableInfo.CACHE_TTLMS_KEY.toLowerCase())); + if(props.containsKey(AbstractSideTableInfo.CACHE_TTLMS_KEY.toLowerCase())){ + Long cacheTTLMS = MathUtil.getLongVal(props.get(AbstractSideTableInfo.CACHE_TTLMS_KEY.toLowerCase())); if(cacheTTLMS < 1000){ throw new RuntimeException("cache time out need > 1000 ms."); } sideTableInfo.setCacheTimeout(cacheTTLMS); } - if(props.containsKey(SideTableInfo.PARTITIONED_JOIN_KEY.toLowerCase())){ - Boolean partitionedJoinKey = MathUtil.getBoolean(props.get(SideTableInfo.PARTITIONED_JOIN_KEY.toLowerCase())); + if(props.containsKey(AbstractSideTableInfo.PARTITIONED_JOIN_KEY.toLowerCase())){ + Boolean partitionedJoinKey = MathUtil.getBoolean(props.get(AbstractSideTableInfo.PARTITIONED_JOIN_KEY.toLowerCase())); if(partitionedJoinKey){ sideTableInfo.setPartitionedJoin(true); } } - if(props.containsKey(SideTableInfo.CACHE_MODE_KEY.toLowerCase())){ - String cachemode = MathUtil.getString(props.get(SideTableInfo.CACHE_MODE_KEY.toLowerCase())); + if(props.containsKey(AbstractSideTableInfo.CACHE_MODE_KEY.toLowerCase())){ + String cachemode = MathUtil.getString(props.get(AbstractSideTableInfo.CACHE_MODE_KEY.toLowerCase())); - if(!cachemode.equalsIgnoreCase("ordered") && !cachemode.equalsIgnoreCase("unordered")){ + if(!"ordered".equalsIgnoreCase(cachemode) && !"unordered".equalsIgnoreCase(cachemode)){ throw new RuntimeException("cachemode must ordered or unordered!"); } sideTableInfo.setCacheMode(cachemode.toLowerCase()); } - if(props.containsKey(SideTableInfo.ASYNC_CAP_KEY.toLowerCase())){ - Integer asyncCap = MathUtil.getIntegerVal(props.get(SideTableInfo.ASYNC_CAP_KEY.toLowerCase())); + if(props.containsKey(AbstractSideTableInfo.ASYNC_CAP_KEY.toLowerCase())){ + Integer asyncCap = MathUtil.getIntegerVal(props.get(AbstractSideTableInfo.ASYNC_CAP_KEY.toLowerCase())); if(asyncCap < 0){ throw new RuntimeException("asyncCapacity size need > 0."); } sideTableInfo.setAsyncCapacity(asyncCap); } - if(props.containsKey(SideTableInfo.ASYNC_TIMEOUT_KEY.toLowerCase())){ - Integer asyncTimeout = MathUtil.getIntegerVal(props.get(SideTableInfo.ASYNC_TIMEOUT_KEY.toLowerCase())); + if(props.containsKey(AbstractSideTableInfo.ASYNC_TIMEOUT_KEY.toLowerCase())){ + Integer asyncTimeout = MathUtil.getIntegerVal(props.get(AbstractSideTableInfo.ASYNC_TIMEOUT_KEY.toLowerCase())); if (asyncTimeout<0){ throw new RuntimeException("asyncTimeout size need > 0."); } sideTableInfo.setAsyncTimeout(asyncTimeout); } - if(props.containsKey(SideTableInfo.ASYNC_TIMEOUT_NUM_KEY.toLowerCase())){ - Integer asyncTimeoutNum = MathUtil.getIntegerVal(props.get(SideTableInfo.ASYNC_TIMEOUT_NUM_KEY.toLowerCase())); + if(props.containsKey(AbstractSideTableInfo.ASYNC_TIMEOUT_NUM_KEY.toLowerCase())){ + Integer asyncTimeoutNum = MathUtil.getIntegerVal(props.get(AbstractSideTableInfo.ASYNC_TIMEOUT_NUM_KEY.toLowerCase())); if (asyncTimeoutNum > 0){ sideTableInfo.setAsyncTimeoutNumLimit(asyncTimeoutNum); } } - if (props.containsKey(SideTableInfo.ASYNC_REQ_POOL_KEY.toLowerCase())) { - Integer asyncPoolSize = MathUtil.getIntegerVal(props.get(SideTableInfo.ASYNC_REQ_POOL_KEY.toLowerCase())); + if (props.containsKey(AbstractSideTableInfo.ASYNC_REQ_POOL_KEY.toLowerCase())) { + Integer asyncPoolSize = MathUtil.getIntegerVal(props.get(AbstractSideTableInfo.ASYNC_REQ_POOL_KEY.toLowerCase())); Preconditions.checkArgument(asyncPoolSize > 0 && asyncPoolSize <= 20, "asyncPoolSize size limit (0,20]"); sideTableInfo.setAsyncPoolSize(asyncPoolSize); } + } } } diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSourceParser.java similarity index 81% rename from core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java rename to core/src/main/java/com/dtstack/flink/sql/table/AbstractSourceParser.java index 745357162..308f5859f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSourceParser.java @@ -33,7 +33,7 @@ * @author xuchao */ -public abstract class AbsSourceParser extends AbsTableParser { +public abstract class AbstractSourceParser extends AbstractTableParser { private static final String VIRTUAL_KEY = "virtualFieldKey"; private static final String WATERMARK_KEY = "waterMarkKey"; @@ -43,21 +43,21 @@ public abstract class AbsSourceParser extends AbsTableParser { private static Pattern waterMarkKeyPattern = Pattern.compile("(?i)^\\s*WATERMARK\\s+FOR\\s+(\\S+)\\s+AS\\s+withOffset\\(\\s*(\\S+)\\s*,\\s*(\\d+)\\s*\\)$"); private static Pattern notNullKeyPattern = Pattern.compile("(?i)^(\\w+)\\s+(\\w+)\\s+NOT\\s+NULL?$"); - public AbsSourceParser() { + public AbstractSourceParser() { addParserHandler(VIRTUAL_KEY, virtualFieldKeyPattern, this::dealVirtualField); addParserHandler(WATERMARK_KEY, waterMarkKeyPattern, this::dealWaterMark); addParserHandler(NOTNULL_KEY, notNullKeyPattern, this::dealNotNull); } - protected void dealVirtualField(Matcher matcher, TableInfo tableInfo){ - SourceTableInfo sourceTableInfo = (SourceTableInfo) tableInfo; + protected void dealVirtualField(Matcher matcher, AbstractTableInfo tableInfo){ + AbstractSourceTableInfo sourceTableInfo = (AbstractSourceTableInfo) tableInfo; String fieldName = matcher.group(2); String expression = matcher.group(1); sourceTableInfo.addVirtualField(fieldName, expression); } - protected void dealWaterMark(Matcher matcher, TableInfo tableInfo){ - SourceTableInfo sourceTableInfo = (SourceTableInfo) tableInfo; + protected void dealWaterMark(Matcher matcher, AbstractTableInfo tableInfo){ + AbstractSourceTableInfo sourceTableInfo = (AbstractSourceTableInfo) tableInfo; String eventTimeField = matcher.group(1); //FIXME Temporarily resolve the second parameter row_time_field Integer offset = MathUtil.getIntegerVal(matcher.group(3)); @@ -65,11 +65,11 @@ protected void dealWaterMark(Matcher matcher, TableInfo tableInfo){ sourceTableInfo.setMaxOutOrderness(offset); } - protected void dealNotNull(Matcher matcher, TableInfo tableInfo) { + protected void dealNotNull(Matcher matcher, AbstractTableInfo tableInfo) { String fieldName = matcher.group(1); String fieldType = matcher.group(2); Class fieldClass= dbTypeConvertToJavaType(fieldType); - TableInfo.FieldExtraInfo fieldExtraInfo = new TableInfo.FieldExtraInfo(); + AbstractTableInfo.FieldExtraInfo fieldExtraInfo = new AbstractTableInfo.FieldExtraInfo(); fieldExtraInfo.setNotNull(true); tableInfo.addPhysicalMappings(fieldName, fieldName); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSourceTableInfo.java similarity index 97% rename from core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java rename to core/src/main/java/com/dtstack/flink/sql/table/AbstractSourceTableInfo.java index 9a41fa0a1..0fdc0e911 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSourceTableInfo.java @@ -36,7 +36,7 @@ * @author xuchao */ -public abstract class SourceTableInfo extends TableInfo { +public abstract class AbstractSourceTableInfo extends AbstractTableInfo { public static final String SOURCE_SUFFIX = "Source"; diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableInfo.java similarity index 98% rename from core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java rename to core/src/main/java/com/dtstack/flink/sql/table/AbstractTableInfo.java index 2fdc297a2..3b514dd48 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableInfo.java @@ -34,7 +34,7 @@ * @author xuchao */ -public abstract class TableInfo implements Serializable { +public abstract class AbstractTableInfo implements Serializable { public static final String PARALLELISM_KEY = "parallelism"; diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableInfoParser.java similarity index 83% rename from core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java rename to core/src/main/java/com/dtstack/flink/sql/table/AbstractTableInfoParser.java index 3c1be25c6..d50e1f23c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableInfoParser.java @@ -22,7 +22,7 @@ import com.dtstack.flink.sql.enums.ETableType; import com.dtstack.flink.sql.parser.CreateTableParser; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.StreamSideFactory; import com.dtstack.flink.sql.sink.StreamSinkFactory; import com.dtstack.flink.sql.source.StreamSourceFactory; @@ -42,7 +42,7 @@ * @author xuchao */ -public class TableInfoParser { +public class AbstractTableInfoParser { private final static String TYPE_KEY = "type"; @@ -50,16 +50,16 @@ public class TableInfoParser { private final static Pattern SIDE_PATTERN = Pattern.compile(SIDE_TABLE_SIGN); - private Map sourceTableInfoMap = Maps.newConcurrentMap(); + private Map sourceTableInfoMap = Maps.newConcurrentMap(); - private Map targetTableInfoMap = Maps.newConcurrentMap(); + private Map targetTableInfoMap = Maps.newConcurrentMap(); - private Map sideTableInfoMap = Maps.newConcurrentMap(); + private Map sideTableInfoMap = Maps.newConcurrentMap(); //Parsing loaded plugin - public TableInfo parseWithTableType(int tableType, CreateTableParser.SqlParserResult parserResult, - String localPluginRoot) throws Exception { - AbsTableParser absTableParser = null; + public AbstractTableInfo parseWithTableType(int tableType, CreateTableParser.SqlParserResult parserResult, + String localPluginRoot) throws Exception { + AbstractTableParser absTableParser = null; Map props = parserResult.getPropMap(); String type = MathUtil.getString(props.get(TYPE_KEY)); @@ -77,9 +77,9 @@ public TableInfo parseWithTableType(int tableType, CreateTableParser.SqlParserRe sourceTableInfoMap.put(type, absTableParser); } }else{ - String cacheType = MathUtil.getString(props.get(SideTableInfo.CACHE_KEY)); - absTableParser = sideTableInfoMap.get(type + cacheType); + absTableParser = sideTableInfoMap.get(type); if(absTableParser == null){ + String cacheType = MathUtil.getString(props.get(AbstractSideTableInfo.CACHE_KEY)); absTableParser = StreamSideFactory.getSqlParser(type, localPluginRoot, cacheType); sideTableInfoMap.put(type + cacheType, absTableParser); } diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableParser.java similarity index 88% rename from core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java rename to core/src/main/java/com/dtstack/flink/sql/table/AbstractTableParser.java index 3e4027b2c..1f210cbf2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTableParser.java @@ -39,7 +39,7 @@ * @author xuchao */ -public abstract class AbsTableParser { +public abstract class AbstractTableParser { private static final String PRIMARY_KEY = "primaryKey"; private static final String NEST_JSON_FIELD_KEY = "nestFieldKey"; @@ -54,7 +54,7 @@ public abstract class AbsTableParser { private Map handlerMap = Maps.newHashMap(); - public AbsTableParser() { + public AbstractTableParser() { addParserHandler(PRIMARY_KEY, primaryKeyPattern, this::dealPrimaryKey); addParserHandler(NEST_JSON_FIELD_KEY, nestJsonFieldKeyPattern, this::dealNestField); } @@ -63,9 +63,9 @@ protected boolean fieldNameNeedsUpperCase() { return true; } - public abstract TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) throws Exception; + public abstract AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) throws Exception; - public boolean dealKeyPattern(String fieldRow, TableInfo tableInfo){ + public boolean dealKeyPattern(String fieldRow, AbstractTableInfo tableInfo){ for(Map.Entry keyPattern : patternMap.entrySet()){ Pattern pattern = keyPattern.getValue(); String key = keyPattern.getKey(); @@ -84,7 +84,7 @@ public boolean dealKeyPattern(String fieldRow, TableInfo tableInfo){ return false; } - public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ + public void parseFieldsInfo(String fieldsInfo, AbstractTableInfo tableInfo){ List fieldRows = DtStringUtil.splitIgnoreQuota(fieldsInfo, ','); for(String fieldRow : fieldRows){ @@ -112,12 +112,12 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ Class fieldClass = null; - TableInfo.FieldExtraInfo fieldExtraInfo = null; + AbstractTableInfo.FieldExtraInfo fieldExtraInfo = null; Matcher matcher = charTypePattern.matcher(fieldType); if (matcher.find()) { fieldClass = dbTypeConvertToJavaType(CHAR_TYPE_NO_LENGTH); - fieldExtraInfo = new TableInfo.FieldExtraInfo(); + fieldExtraInfo = new AbstractTableInfo.FieldExtraInfo(); fieldExtraInfo.setLength(Integer.valueOf(matcher.group(1))); } else { fieldClass = dbTypeConvertToJavaType(fieldType); @@ -133,7 +133,7 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ tableInfo.finish(); } - public void dealPrimaryKey(Matcher matcher, TableInfo tableInfo){ + public void dealPrimaryKey(Matcher matcher, AbstractTableInfo tableInfo){ String primaryFields = matcher.group(1).trim(); String[] splitArry = primaryFields.split(","); List primaryKes = Lists.newArrayList(splitArry); @@ -145,7 +145,7 @@ public void dealPrimaryKey(Matcher matcher, TableInfo tableInfo){ * @param matcher * @param tableInfo */ - protected void dealNestField(Matcher matcher, TableInfo tableInfo) { + protected void dealNestField(Matcher matcher, AbstractTableInfo tableInfo) { String physicalField = matcher.group(1); Preconditions.checkArgument(!physicalFieldFunPattern.matcher(physicalField).find(), "No need to add data types when using functions, The correct way is : strLen(name) as nameSize, "); @@ -154,7 +154,7 @@ protected void dealNestField(Matcher matcher, TableInfo tableInfo) { String mappingField = matcher.group(4); Class fieldClass = dbTypeConvertToJavaType(fieldType); boolean notNull = matcher.group(5) != null; - TableInfo.FieldExtraInfo fieldExtraInfo = new TableInfo.FieldExtraInfo(); + AbstractTableInfo.FieldExtraInfo fieldExtraInfo = new AbstractTableInfo.FieldExtraInfo(); fieldExtraInfo.setNotNull(notNull); tableInfo.addPhysicalMappings(mappingField, physicalField); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTargetTableInfo.java similarity index 94% rename from core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.java rename to core/src/main/java/com/dtstack/flink/sql/table/AbstractTargetTableInfo.java index 0dc9ca77b..dbd99f993 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractTargetTableInfo.java @@ -29,7 +29,7 @@ * @author xuchao */ -public abstract class TargetTableInfo extends TableInfo { +public abstract class AbstractTargetTableInfo extends AbstractTableInfo { public static final String TARGET_SUFFIX = "Sink"; diff --git a/core/src/main/java/com/dtstack/flink/sql/table/ITableFieldDealHandler.java b/core/src/main/java/com/dtstack/flink/sql/table/ITableFieldDealHandler.java index db804ea34..fab634cdc 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/ITableFieldDealHandler.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/ITableFieldDealHandler.java @@ -30,5 +30,5 @@ */ public interface ITableFieldDealHandler { - void dealPrimaryKey(Matcher matcher, TableInfo tableInfo); + void dealPrimaryKey(Matcher matcher, AbstractTableInfo tableInfo); } 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 feebbe3f1..4bb4ff0d8 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 @@ -86,7 +86,8 @@ public static Class stringConvertClass(String str) { case "decimal": case "decimalunsigned": return BigDecimal.class; - + default: + break; } throw new RuntimeException("不支持 " + str + " 类型"); 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 a0f5291e7..409602baa 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 @@ -16,22 +16,17 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.util; import java.sql.Timestamp; import java.text.ParseException; import java.text.SimpleDateFormat; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalTime; -import java.time.ZoneOffset; -import java.util.Calendar; -import java.util.Date; -import java.util.Locale; -import java.util.SimpleTimeZone; -import java.util.TimeZone; +import java.time.*; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeParseException; +import java.util.*; import java.util.regex.Pattern; import static java.time.format.DateTimeFormatter.ISO_INSTANT; @@ -39,7 +34,7 @@ /** * - * Reason: TODO ADD REASON(可选) + * 日期工具 * Date: 2017年03月10日 下午1:16:37 * Company: www.dtstack.com * @author sishu.yss @@ -47,20 +42,16 @@ */ public class DateUtil { - static final String timeZone = "GMT+8"; - static final String datetimeFormat = "yyyy-MM-dd HH:mm:ss"; - static final String dateFormat = "yyyy-MM-dd"; - static final String timeFormat = "HH:mm:ss"; - static final SimpleDateFormat datetimeFormatter = new SimpleDateFormat(datetimeFormat); - static final SimpleDateFormat dateFormatter = new SimpleDateFormat(dateFormat); - static final SimpleDateFormat timeFormatter = new SimpleDateFormat(timeFormat); + static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); + static final DateTimeFormatter DATE_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd"); + static final DateTimeFormatter TIME_FORMATTER = DateTimeFormatter.ofPattern("HH:mm:ss"); 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 int MILLIS_PER_SECOND = 1000; + public static java.sql.Date columnToDate(Object column) { if(column instanceof String) { return new java.sql.Date(stringToDate((String)column).getTime()); @@ -84,23 +75,32 @@ public static Date stringToDate(String strDate) { return null; } try { - return datetimeFormatter.parse(strDate); - } catch (ParseException ignored) { + ; + return localDateTimetoDate(LocalDateTime.parse(strDate, DATE_TIME_FORMATTER)); + } catch (DateTimeParseException ignored) { } try { - return dateFormatter.parse(strDate); - } catch (ParseException ignored) { + return localDateTimetoDate(LocalDate.parse(strDate, DATE_FORMATTER).atStartOfDay()); + } catch (DateTimeParseException ignored) { } try { - return timeFormatter.parse(strDate); - } catch (ParseException ignored) { + return localDateTimetoDate(LocalDateTime.of(LocalDate.now(), LocalTime.parse(strDate, TIME_FORMATTER))); + } catch (DateTimeParseException ignored) { } throw new RuntimeException("can't parse date"); } + public static Date localDateTimetoDate(LocalDateTime localDateTime){ + return Date.from(localDateTime.atZone(ZoneId.systemDefault()).toInstant()); + } + + public static LocalDateTime dateToLocalDateTime(Date date){ + return date.toInstant().atZone(ZoneId.systemDefault()).toLocalDateTime(); + } + /** * * @@ -130,13 +130,13 @@ public static long getTodayStart(long day) { * @return */ public static long getTodayStart(long day,String scope) { - if(scope.equals("MS")){ - return getTodayStart(day)*1000; - }else if(scope.equals("S")){ - return getTodayStart(day); - }else{ - return getTodayStart(day); - } + if("MS".equals(scope)){ + return getTodayStart(day)*1000; + }else if("S".equals(scope)){ + return getTodayStart(day); + }else{ + return getTodayStart(day); + } } /** @@ -168,15 +168,15 @@ public static long getNextDayStart(long day) { * @return */ public static long getNextDayStart(long day,String scope) { - if(scope.equals("MS")){ - return getNextDayStart(day)*1000; - }else if(scope.equals("S")){ - return getNextDayStart(day); - }else{ - return getNextDayStart(day); - } + if("MS".equals(scope)){ + return getNextDayStart(day)*1000; + }else if("S".equals(scope)){ + return getNextDayStart(day); + }else{ + return getNextDayStart(day); + } } - + /** * @@ -237,7 +237,7 @@ public static long getWeekFirst(long day) { /** * 根据某个日期时间戳秒值,获取所在周在一年中是第几周. - * + * * @param day * @return */ @@ -257,17 +257,17 @@ public static int getWeekOfYear(long day) { */ public static String getYesterdayByString(String day, String inFormat, String outFormat){ try { - SimpleDateFormat sdf = new SimpleDateFormat(inFormat); - Date date = sdf.parse(day); - Calendar calendar = Calendar.getInstance(); - calendar.setTime(date); - int calendarDay = calendar.get(Calendar.DATE); - calendar.set(Calendar.DATE, calendarDay - 1); - String dayBefore = new SimpleDateFormat(outFormat).format(calendar.getTime()); - return dayBefore; - } catch (ParseException e) { - return null; - } + SimpleDateFormat sdf = new SimpleDateFormat(inFormat); + Date date = sdf.parse(day); + Calendar calendar = Calendar.getInstance(); + calendar.setTime(date); + int calendarDay = calendar.get(Calendar.DATE); + calendar.set(Calendar.DATE, calendarDay - 1); + String dayBefore = new SimpleDateFormat(outFormat).format(calendar.getTime()); + return dayBefore; + } catch (ParseException e) { + return null; + } } /** @@ -288,7 +288,7 @@ public static String getTomorrowByString(String day, String inFormat, String out String dayBefore = new SimpleDateFormat(outFormat).format(calendar.getTime()); return dayBefore; } - + /** * * @param date @@ -320,7 +320,7 @@ public static String get30DaysBeforeByString(String day, String inFormat, String calendar.set(Calendar.DATE, calendarDay - 30); return new SimpleDateFormat(outFormat).format(calendar.getTime()); } - + /** * * @param day @@ -330,14 +330,14 @@ public static String get30DaysBeforeByString(String day, String inFormat, String * @throws ParseException */ public static String get30DaysLaterByString(String day, String inFormat, String outFormat) throws ParseException { - SimpleDateFormat sdf = new SimpleDateFormat(inFormat); - Date date = sdf.parse(day); - Calendar calendar = Calendar.getInstance(); - calendar.setTime(date); - int calendarDay = calendar.get(Calendar.DATE); - calendar.set(Calendar.DATE, calendarDay + 30); - String dayBefore = new SimpleDateFormat(outFormat).format(calendar.getTime()); - return dayBefore; + SimpleDateFormat sdf = new SimpleDateFormat(inFormat); + Date date = sdf.parse(day); + Calendar calendar = Calendar.getInstance(); + calendar.setTime(date); + int calendarDay = calendar.get(Calendar.DATE); + calendar.set(Calendar.DATE, calendarDay + 30); + String dayBefore = new SimpleDateFormat(outFormat).format(calendar.getTime()); + return dayBefore; } @@ -349,7 +349,7 @@ public static String get30DaysLaterByString(String day, String inFormat, String * @return String * @throws ParseException */ - public static String getDateStrTOFormat(String day, String inFormat, String outFormat) throws ParseException { + public static String getDateStrToFormat(String day, String inFormat, String outFormat) throws ParseException { SimpleDateFormat sdf = new SimpleDateFormat(inFormat); Date date = sdf.parse(day); Calendar calendar = Calendar.getInstance(); @@ -357,8 +357,8 @@ public static String getDateStrTOFormat(String day, String inFormat, String outF String dayBefore = new SimpleDateFormat(outFormat).format(calendar.getTime()); return dayBefore; } - - public static long getDateMillTOFormat(String day, String inFormat) throws ParseException { + + public static long getDateMillToFormat(String day, String inFormat) throws ParseException { SimpleDateFormat sdf = new SimpleDateFormat(inFormat); Date date = sdf.parse(day); Calendar calendar = Calendar.getInstance(); @@ -474,79 +474,79 @@ public static long getMillByLastWeekDay() { cal.set(Calendar.MILLISECOND, 0); return cal.getTimeInMillis() / 1000; } - - /** - * @return long - */ + + /** + * @return long + */ public static long getMillByDay(int severalDays,String condition) { - int dateT=0; + int dateT=0; Calendar cal = Calendar.getInstance(); - if(condition==null){ - return getMillToDay(cal,dateT); - } - if(condition.equals("-")){ - dateT = (cal.get(Calendar.DATE) - severalDays); - return getMillToDay(cal,dateT); - } - if(condition.equals("+")){ - dateT = (cal.get(Calendar.DATE) + severalDays); - return getMillToDay(cal,dateT); - } - return getMillToDay(cal,dateT); - } - + if(condition==null){ + return getMillToDay(cal,dateT); + } + if("-".equals(condition)){ + dateT = (cal.get(Calendar.DATE) - severalDays); + return getMillToDay(cal,dateT); + } + if("+".equals(condition)){ + dateT = (cal.get(Calendar.DATE) + severalDays); + return getMillToDay(cal,dateT); + } + return getMillToDay(cal,dateT); + } + /** * @return long */ public static long getStampByDay(int severalDays,String condition) { - int dateT=0; - Calendar cal = Calendar.getInstance(); - if(condition==null){ - return getStampToDay(cal,dateT); - } - if(condition.equals("-")){ - dateT = (cal.get(Calendar.DATE) - severalDays); - return getStampToDay(cal,dateT); - } - if(condition.equals("+")){ - dateT = (cal.get(Calendar.DATE) + severalDays); - return getStampToDay(cal,dateT); - } - return getStampToDay(cal,dateT); + int dateT=0; + Calendar cal = Calendar.getInstance(); + if(condition==null){ + return getStampToDay(cal,dateT); + } + if("-".equals(condition)){ + dateT = (cal.get(Calendar.DATE) - severalDays); + return getStampToDay(cal,dateT); + } + if("+".equals(condition)){ + dateT = (cal.get(Calendar.DATE) + severalDays); + return getStampToDay(cal,dateT); + } + return getStampToDay(cal,dateT); } /** * @return long */ public static long getMillByDay(){ - return getMillByDay(0,null); + return getMillByDay(0,null); } - + /** * @param cal Calendar - * @param dateT Integer + * @param dateT Integer * @return long */ public static long getMillToDay(Calendar cal,int dateT){ - if(dateT!=0){ - cal.set(Calendar.DATE, dateT); - } - cal.set(Calendar.HOUR_OF_DAY, 0); - cal.set(Calendar.MINUTE, 0); - cal.set(Calendar.SECOND, 0); - cal.set(Calendar.MILLISECOND, 0); - return cal.getTimeInMillis()/1000; - } - + if(dateT!=0){ + cal.set(Calendar.DATE, dateT); + } + cal.set(Calendar.HOUR_OF_DAY, 0); + cal.set(Calendar.MINUTE, 0); + cal.set(Calendar.SECOND, 0); + cal.set(Calendar.MILLISECOND, 0); + return cal.getTimeInMillis()/1000; + } + /** * @param cal Calendar - * @param dateT Integer + * @param dateT Integer * @return long */ public static long getStampToDay(Calendar cal,int dateT){ - if(dateT!=0){ - cal.set(Calendar.DATE, dateT); - } - return cal.getTimeInMillis(); + if(dateT!=0){ + cal.set(Calendar.DATE, dateT); + } + return cal.getTimeInMillis(); } public static String getToday() { @@ -568,7 +568,7 @@ public static String getDate(long day, String format) { SimpleDateFormat sf = new SimpleDateFormat(format); return sf.format(cal.getTime()); } - + /** * * @param date @@ -578,21 +578,21 @@ public static String getDate(Date date, String format) { SimpleDateFormat sf = new SimpleDateFormat(format); return sf.format(date); } - - + + /** * * @param day * @param format * @return long - * @throws ParseException + * @throws ParseException */ public static long stringToLong(String day, String format) throws ParseException { - SimpleDateFormat dateFormat = new SimpleDateFormat(format); - long Date = dateFormat.parse(day).getTime(); - return Date; + SimpleDateFormat dateFormat = new SimpleDateFormat(format); + long date = dateFormat.parse(day).getTime(); + return date; } - + /** * @param day * @param format @@ -600,30 +600,30 @@ public static long stringToLong(String day, String format) throws ParseException * @throws ParseException */ public static Date stringToDate(String day, String format) { - try { - SimpleDateFormat dateFormat = new SimpleDateFormat(format); - Date Date = dateFormat.parse(day); - return Date; - } catch (ParseException e) { - return new Date(); - } + try { + SimpleDateFormat dateFormat = new SimpleDateFormat(format); + Date date = dateFormat.parse(day); + return date; + } catch (ParseException e) { + return new Date(); + } } - - + + /** * long型时间戳转为String型 - * + * * @param day 秒 * @return 格式化后的日期 - * @throws ParseException + * @throws ParseException */ public static String longToString(long day, String format) throws ParseException { - if (("" + day).length() <= 10){ + if (("" + day).length() <= 10){ day=day*1000; } - SimpleDateFormat dateFormat = new SimpleDateFormat(format); - String Date = dateFormat.format(day); - return Date; + SimpleDateFormat dateFormat = new SimpleDateFormat(format); + String date = dateFormat.format(day); + return date; } /** @@ -653,39 +653,39 @@ public static long getMillByNow() { return cal.getTimeInMillis(); } - public static int getWeeksBetweenTwoDates(long startDay, long endDay) { - int week = getWeekOfYear(endDay) - getWeekOfYear(startDay) + 1; - if(week<1){ - week = getWeekOfYear(endDay) + getMaxWeekOfYear(startDay) - getWeekOfYear(startDay) + 1; - } - return week; - } + public static int getWeeksBetweenTwoDates(long startDay, long endDay) { + int week = getWeekOfYear(endDay) - getWeekOfYear(startDay) + 1; + if(week<1){ + week = getWeekOfYear(endDay) + getMaxWeekOfYear(startDay) - getWeekOfYear(startDay) + 1; + } + return week; + } - public static int getMaxWeekOfYear(long startDay) { - Calendar cal = Calendar.getInstance(); + public static int getMaxWeekOfYear(long startDay) { + Calendar cal = Calendar.getInstance(); cal.setTime(new Date(startDay * 1000)); return cal.getMaximum(Calendar.WEEK_OF_YEAR); - } - - public static int getMonthsBetweenTwoDates(long startDay, long endDay) { - int month = DateUtil.getMonth(endDay) - DateUtil.getMonth(startDay) + 1; - if(month<1){ - month = getMonth(endDay) + 12 - getMonth(startDay) +1; - } - return month; - } - - public static Date parseDate(String dateStr, String pattern){ - SimpleDateFormat sdf = new SimpleDateFormat(); - sdf.applyPattern(pattern); - try { - return sdf.parse(dateStr); - } catch (ParseException e) { - return null; - } - } - - /** + } + + public static int getMonthsBetweenTwoDates(long startDay, long endDay) { + int month = DateUtil.getMonth(endDay) - DateUtil.getMonth(startDay) + 1; + if(month<1){ + month = getMonth(endDay) + 12 - getMonth(startDay) +1; + } + return month; + } + + public static Date parseDate(String dateStr, String pattern){ + SimpleDateFormat sdf = new SimpleDateFormat(); + sdf.applyPattern(pattern); + try { + return sdf.parse(dateStr); + } catch (ParseException e) { + return null; + } + } + + /** * * @param time Long 时间 * @return long @@ -703,7 +703,7 @@ public static long getMinuteStart(long time) { firstDay = cal.getTimeInMillis() / 1000; return firstDay; } - + /** * @param time Long * @return long @@ -732,24 +732,24 @@ public static Date getDateByLong(long time){ date.setTime(time); return date; } - + public static Date parseDate(String dateStr, String pattern, Locale locale){ - SimpleDateFormat df = new SimpleDateFormat( - pattern, locale); - - df.setTimeZone(new SimpleTimeZone(0, "GMT")); - try { - return df.parse(dateStr); - } catch (ParseException e) { - return null; - } - } - + SimpleDateFormat df = new SimpleDateFormat( + pattern, locale); + + df.setTimeZone(new SimpleTimeZone(0, "GMT")); + try { + return df.parse(dateStr); + } catch (ParseException e) { + return null; + } + } + public static String getDate(Date date, String format, Locale locale) { - SimpleDateFormat df = new SimpleDateFormat( - format, locale); - df.setTimeZone(new SimpleTimeZone(0, "GMT")); + SimpleDateFormat df = new SimpleDateFormat( + format, locale); + df.setTimeZone(new SimpleTimeZone(0, "GMT")); return df.format(date); } @@ -777,59 +777,33 @@ public static java.sql.Timestamp columnToTimestamp(Object column) { } public static String dateToString(Date date) { - return dateFormatter.format(date); + LocalDateTime localDateTime = dateToLocalDateTime(date); + return localDateTime.format(DATE_FORMATTER); } public static String timestampToString(Date date) { - return datetimeFormatter.format(date); + LocalDateTime localDateTime = dateToLocalDateTime(date); + return localDateTime.format(DATE_TIME_FORMATTER); } - public static Timestamp getTimestampFromStr(String timeStr) { if (DATETIME.matcher(timeStr).matches()) { Instant instant = Instant.from(ISO_INSTANT.parse(timeStr)); return new Timestamp(instant.getEpochSecond() * MILLIS_PER_SECOND); - } else { - java.sql.Date date = null; - try { - date = new java.sql.Date(datetimeFormatter.parse(timeStr).getTime()); - } catch (ParseException e) { - throw new RuntimeException("getTimestampFromStr error data is " + timeStr); - } - return new Timestamp(date.getTime()); } + return new Timestamp(stringToDate(timeStr).getTime()); } public static java.sql.Date getDateFromStr(String dateStr) { - // 2020-01-01 format if (DATE.matcher(dateStr).matches()) { - // convert from local date to instant Instant instant = LocalDate.parse(dateStr).atTime(LocalTime.of(0, 0, 0, 0)).toInstant(ZoneOffset.UTC); - // calculate the timezone offset in millis int offset = TimeZone.getDefault().getOffset(instant.toEpochMilli()); - // need to remove the offset since time has no TZ component return new java.sql.Date(instant.toEpochMilli() - offset); } else if (DATETIME.matcher(dateStr).matches()) { - // 2020-01-01T12:12:12Z format Instant instant = Instant.from(ISO_INSTANT.parse(dateStr)); return new java.sql.Date(instant.toEpochMilli()); - } else { - try { - // 2020-01-01 12:12:12.0 format - return new java.sql.Date(datetimeFormatter.parse(dateStr).getTime()); - } catch (ParseException e) { - throw new RuntimeException("String convert to Date fail."); - } } - } - - - public static String getStringFromTimestamp(Timestamp timestamp) { - return datetimeFormatter.format(timestamp); - } - - public static String getStringFromDate(java.sql.Date date) { - return dateFormatter.format(date); + return new java.sql.Date(stringToDate(dateStr).getTime()); } } 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 b2486b1ab..10a6b4f63 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 @@ -219,14 +219,14 @@ public static String addJdbcParam(String dbUrl, Map addParams, b return preStr + "?" + sb.toString(); } - public static boolean isJosn(String str){ + public static boolean isJson(String str) { boolean flag = false; - if(StringUtils.isNotBlank(str)){ + if (StringUtils.isNotBlank(str)) { try { - objectMapper.readValue(str,Map.class); + objectMapper.readValue(str, Map.class); flag = true; } catch (Throwable e) { - flag=false; + flag = false; } } return flag; 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 fde2f166e..3d66d8a6d 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 @@ -28,10 +28,10 @@ public class JDBCUtils { private static final Logger LOG = LoggerFactory.getLogger(ClassUtil.class); - public final static String lock_str = "jdbc_lock_str"; + public final static String LOCK_STR = "jdbc_lock_str"; public static void forName(String clazz, ClassLoader classLoader) { - synchronized (lock_str){ + synchronized (LOCK_STR){ try { Class.forName(clazz, true, classLoader); DriverManager.setLoginTimeout(10); 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 23513a912..5aaa40a15 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 @@ -227,7 +227,7 @@ public static BigDecimal getBigDecimal(Object obj) { } else if (obj instanceof BigInteger) { return new BigDecimal((BigInteger) obj); } else if (obj instanceof Number) { - return new BigDecimal(((Number) obj).doubleValue()); + return BigDecimal.valueOf(((Number) obj).doubleValue()); } throw new RuntimeException("not support type of " + obj.getClass() + " convert to BigDecimal."); } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index a63e5a663..8eb743cf0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -109,7 +109,7 @@ public static String getSqlSideClassName(String pluginTypeName, String type, Str return CLASS_PRE_STR + "." + type.toLowerCase() + "." + pluginTypeName + "." + pluginClassName; } - public static Map ObjectToMap(Object obj) throws Exception{ + public static Map objectToMap(Object obj) throws Exception{ return objectMapper.readValue(objectMapper.writeValueAsBytes(obj), Map.class); } @@ -173,8 +173,8 @@ public static void addPluginJar(String pluginDir, DtClassLoader classLoader) thr } for(File file : files){ - URL pluginJarURL = file.toURI().toURL(); - classLoader.addURL(pluginJarURL); + URL pluginJarUrl = file.toURI().toURL(); + classLoader.addURL(pluginJarUrl); } } @@ -191,8 +191,8 @@ public static URL[] getPluginJarUrls(String pluginDir) throws MalformedURLExcept } for(File file : files){ - URL pluginJarURL = file.toURI().toURL(); - urlList.add(pluginJarURL); + URL pluginJarUrl = file.toURI().toURL(); + urlList.add(pluginJarUrl); } return urlList.toArray(new URL[urlList.size()]); } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/AbstractCustomerWaterMarker.java similarity index 94% rename from core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java rename to core/src/main/java/com/dtstack/flink/sql/watermarker/AbstractCustomerWaterMarker.java index ed8f64288..d75d26a61 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/AbstractCustomerWaterMarker.java @@ -38,7 +38,7 @@ * @author xuchao */ -public abstract class AbsCustomerWaterMarker extends BoundedOutOfOrdernessTimestampExtractor implements RichFunction { +public abstract class AbstractCustomerWaterMarker extends BoundedOutOfOrdernessTimestampExtractor implements RichFunction { private static final long serialVersionUID = 1L; @@ -55,7 +55,7 @@ public abstract class AbsCustomerWaterMarker extends BoundedOutOfOrdernessTim protected TimeZone timezone; - public AbsCustomerWaterMarker(Time maxOutOfOrderness) { + public AbstractCustomerWaterMarker(Time maxOutOfOrderness) { super(maxOutOfOrderness); } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java index 99415ba61..979b4c3d6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java @@ -35,7 +35,7 @@ * @author xuchao */ -public class CustomerWaterMarkerForLong extends AbsCustomerWaterMarker { +public class CustomerWaterMarkerForLong extends AbstractCustomerWaterMarker { private static final Logger logger = LoggerFactory.getLogger(CustomerWaterMarkerForLong.class); diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java index b7961e410..05ee3e46e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java @@ -20,7 +20,6 @@ package com.dtstack.flink.sql.watermarker; -import com.dtstack.flink.sql.util.MathUtil; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.types.Row; import org.slf4j.Logger; @@ -36,7 +35,7 @@ * @author xuchao */ -public class CustomerWaterMarkerForTimeStamp extends AbsCustomerWaterMarker { +public class CustomerWaterMarkerForTimeStamp extends AbstractCustomerWaterMarker { private static final Logger logger = LoggerFactory.getLogger(CustomerWaterMarkerForTimeStamp.class); diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java index f3208db03..b0ca0335f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.watermarker; -import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.table.AbstractSourceTableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; import com.google.common.base.Strings; @@ -40,7 +40,7 @@ public class WaterMarkerAssigner { - public boolean checkNeedAssignWaterMarker(SourceTableInfo tableInfo){ + public boolean checkNeedAssignWaterMarker(AbstractSourceTableInfo tableInfo){ if(Strings.isNullOrEmpty(tableInfo.getEventTimeField())){ return false; } @@ -48,7 +48,7 @@ public boolean checkNeedAssignWaterMarker(SourceTableInfo tableInfo){ return true; } - public DataStream assignWaterMarker(DataStream dataStream, RowTypeInfo typeInfo, SourceTableInfo sourceTableInfo){ + public DataStream assignWaterMarker(DataStream dataStream, RowTypeInfo typeInfo, AbstractSourceTableInfo sourceTableInfo){ String eventTimeFieldName = sourceTableInfo.getEventTimeField(); @@ -75,7 +75,7 @@ public DataStream assignWaterMarker(DataStream dataStream, RowTypeInfo type TypeInformation fieldType = fieldTypes[pos]; - AbsCustomerWaterMarker waterMarker = null; + AbstractCustomerWaterMarker waterMarker = null; if(fieldType.getTypeClass().isAssignableFrom(Timestamp.class)){ waterMarker = new CustomerWaterMarkerForTimeStamp(Time.milliseconds(maxOutOrderness), pos,timeZone); }else if(fieldType.getTypeClass().isAssignableFrom(Long.class)){ diff --git a/core/src/test/java/com/dtstack/flink/sql/side/SidePredicatesParserTest.java b/core/src/test/java/com/dtstack/flink/sql/side/SidePredicatesParserTest.java index 546fcbbac..feec8e547 100644 --- a/core/src/test/java/com/dtstack/flink/sql/side/SidePredicatesParserTest.java +++ b/core/src/test/java/com/dtstack/flink/sql/side/SidePredicatesParserTest.java @@ -54,7 +54,7 @@ public void testfillPredicatesForSideTable() throws SqlParseException { " MyTable.a='1' and s.d='1' and s.d <> '3' and s.c LIKE '%xx%' and s.c in ('1','2') and s.c between '10' and '23' and s.d is not null\n"; - SideTableInfo sideTableInfo = new SideTableInfo(){ + AbstractSideTableInfo sideTableInfo = new AbstractSideTableInfo(){ @Override public boolean check() { return false; @@ -63,7 +63,7 @@ public boolean check() { sideTableInfo.setName("sideTable"); - Map sideTableMap = new HashMap<>(); + Map sideTableMap = new HashMap<>(); sideTableMap.put("sideTable", sideTableInfo); SidePredicatesParser sidePredicatesParser = new SidePredicatesParser(); diff --git a/db2/db2-side/db2-all-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AllReqRow.java b/db2/db2-side/db2-all-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AllReqRow.java index a77201b80..b354a6269 100644 --- a/db2/db2-side/db2-all-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AllReqRow.java +++ b/db2/db2-side/db2-all-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AllReqRow.java @@ -20,10 +20,8 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; -import com.dtstack.flink.sql.util.DtStringUtil; -import com.google.common.collect.Maps; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.AbstractRdbAllReqRow; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,7 +29,6 @@ import java.sql.Connection; import java.sql.DriverManager; import java.util.List; -import java.util.Map; /** * Date: 2019/11/20 @@ -40,21 +37,21 @@ * @author xiuzhu */ -public class Db2AllReqRow extends RdbAllReqRow { +public class Db2AllReqRow extends AbstractRdbAllReqRow { private static final Logger LOG = LoggerFactory.getLogger(Db2AllReqRow.class); private static final String DB2_DRIVER = "com.ibm.db2.jcc.DB2Driver"; - public Db2AllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public Db2AllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new Db2AllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @Override - public Connection getConn(String dbURL, String userName, String password) { + public Connection getConn(String dbUrl, String userName, String password) { try { Class.forName(DB2_DRIVER); - Connection conn = DriverManager.getConnection(dbURL, userName, password); + Connection conn = DriverManager.getConnection(dbUrl, userName, password); return conn; } catch (Exception e) { LOG.error("", e); diff --git a/db2/db2-side/db2-all-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AllSideInfo.java b/db2/db2-side/db2-all-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AllSideInfo.java index b40cfd772..4ce01bcc8 100644 --- a/db2/db2-side/db2-all-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AllSideInfo.java +++ b/db2/db2-side/db2-all-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AllSideInfo.java @@ -20,7 +20,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; @@ -37,7 +37,7 @@ */ public class Db2AllSideInfo extends RdbAllSideInfo { - public Db2AllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public Db2AllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java b/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java index ed9fb025b..b190973e3 100644 --- a/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java +++ b/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java @@ -20,7 +20,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncReqRow; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import io.vertx.core.Vertx; @@ -45,7 +45,7 @@ public class Db2AsyncReqRow extends RdbAsyncReqRow { private final static String DB2_DRIVER = "com.ibm.db2.jcc.DB2Driver"; - public Db2AsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public Db2AsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new Db2AsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @@ -71,7 +71,7 @@ public void open(Configuration parameters) throws Exception { vo.setWorkerPoolSize(rdbSideTableInfo.getAsyncPoolSize()); vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); - setRdbSQLClient(JDBCClient.createNonShared(vertx, db2lientConfig)); + setRdbSqlClient(JDBCClient.createNonShared(vertx, db2lientConfig)); } } diff --git a/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncSideInfo.java b/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncSideInfo.java index dbb8230e7..91418a1d0 100644 --- a/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncSideInfo.java +++ b/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncSideInfo.java @@ -20,7 +20,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; @@ -37,7 +37,7 @@ */ public class Db2AsyncSideInfo extends RdbAsyncSideInfo { - public Db2AsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public Db2AsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/db2/db2-side/db2-side-core/src/main/java/com/dtstack/flink/sql/side/db2/table/Db2SideParser.java b/db2/db2-side/db2-side-core/src/main/java/com/dtstack/flink/sql/side/db2/table/Db2SideParser.java index db8a07dc6..96be4ec15 100644 --- a/db2/db2-side/db2-side-core/src/main/java/com/dtstack/flink/sql/side/db2/table/Db2SideParser.java +++ b/db2/db2-side/db2-side-core/src/main/java/com/dtstack/flink/sql/side/db2/table/Db2SideParser.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.side.db2.table; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import java.util.Map; @@ -36,8 +36,8 @@ public class Db2SideParser extends RdbSideParser { private static final String CURR_TYPE = "db2"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo tableInfo = super.getTableInfo(tableName, fieldsInfo, props); + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + AbstractTableInfo tableInfo = super.getTableInfo(tableName, fieldsInfo, props); tableInfo.setType(CURR_TYPE); return tableInfo; diff --git a/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/DbSink.java b/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/DbSink.java index 573a0896a..09844d1a3 100644 --- a/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/DbSink.java +++ b/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/DbSink.java @@ -1,13 +1,10 @@ package com.dtstack.flink.sql.sink.db; import com.dtstack.flink.sql.sink.rdb.JDBCOptions; -import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.AbstractRdbSink; import com.dtstack.flink.sql.sink.rdb.format.JDBCUpsertOutputFormat; -import java.util.List; -import java.util.Map; - -public class DbSink extends RdbSink { +public class DbSink extends AbstractRdbSink { public DbSink() { super(new DbDialect()); @@ -15,7 +12,7 @@ public DbSink() { @Override public JDBCUpsertOutputFormat getOutputFormat() { JDBCOptions jdbcOptions = JDBCOptions.builder() - .setDBUrl(dbURL) + .setDbUrl(dbUrl) .setDialect(jdbcDialect) .setUsername(userName) .setPassword(password) diff --git a/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/table/DbSinkParser.java b/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/table/DbSinkParser.java index ba11aabf3..3e823ab20 100644 --- a/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/table/DbSinkParser.java +++ b/db2/db2-sink/src/main/java/com/dtstack/flink/sql/sink/db/table/DbSinkParser.java @@ -1,7 +1,7 @@ package com.dtstack.flink.sql.sink.db.table; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import java.util.Map; @@ -10,8 +10,8 @@ public class DbSinkParser extends RdbSinkParser { private static final String CURR_TYPE = "db2"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo tableInfo = super.getTableInfo(tableName, fieldsInfo, props); + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + AbstractTableInfo tableInfo = super.getTableInfo(tableName, fieldsInfo, props); tableInfo.setType(CURR_TYPE); return tableInfo; } diff --git a/docs/kafkaSink.md b/docs/kafkaSink.md new file mode 100644 index 000000000..5d7c7b2a7 --- /dev/null +++ b/docs/kafkaSink.md @@ -0,0 +1,223 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + ... + function(colNameX) AS aliasName, + WATERMARK FOR colName AS withOffset( colName , delayTime ) + )WITH( + type ='kafka11', + bootstrapServers ='ip:port,ip:port...', + zookeeperQuorum ='ip:port,ip:port/zkparent', + offsetReset ='latest', + topic ='topicName', + groupId='test', + parallelism ='parllNum', + ); +``` + +## 2.支持的版本 + kafka09,kafka10,kafka11及以上版本 + **kafka读取和写入的版本必须一致,否则会有兼容性错误。** + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName | 在 sql 中使用的名称;即注册到flink-table-env上的名称| +| colName | 列名称| +| colType | 列类型 [colType支持的类型](colType.md)| + +## 4.参数: + +|参数名称|含义|是否必填|默认值| +|----|---|---|---| +|type | kafka09 | 是|kafka09、kafka10、kafka11、kafka(对应kafka1.0及以上版本)| +|groupId | 需要读取的 groupId 名称|否|| +|bootstrapServers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| +|zookeeperQuorum | kafka zk地址信息(多个之间用逗号分隔)|是|| +|topic | 需要读取的 topic 名称|是|| +|parallelism | 并行度设置|否|1| +|partitionKeys | 用来分区的字段|否|| +|updateMode | 回溯流数据下发模式,append,upsert.upsert模式下会将是否为回溯信息以字段形式进行下发。|否|append| +|sinkdatatype | 写入kafka数据格式,json,avro,csv|否|json| +|fieldDelimiter | csv数据分隔符|否| , | + + +**kafka相关参数可以自定义,使用kafka.开头即可。** +``` +kafka.consumer.id +kafka.socket.timeout.ms +kafka.fetch.message.max.bytes +kafka.num.consumer.fetchers +kafka.auto.commit.enable +kafka.auto.commit.interval.ms +kafka.queued.max.message.chunks +kafka.rebalance.max.retries +kafka.fetch.min.bytes +kafka.fetch.wait.max.ms +kafka.rebalance.backoff.ms +kafka.refresh.leader.backoff.ms +kafka.consumer.timeout.ms +kafka.exclude.internal.topics +kafka.partition.assignment.strategy +kafka.client.id +kafka.zookeeper.session.timeout.ms +kafka.zookeeper.connection.timeout.ms +kafka.zookeeper.sync.time.ms +kafka.offsets.storage +kafka.offsets.channel.backoff.ms +kafka.offsets.channel.socket.timeout.ms +kafka.offsets.commit.max.retries +kafka.dual.commit.enabled +kafka.partition.assignment.strategy +kafka.socket.receive.buffer.bytes +kafka.fetch.min.bytes +``` + +## 5.样例: + +### json格式: +``` +CREATE TABLE MyResult( + channel varchar, + pv varchar + )WITH( + type='kafka', + bootstrapServers='172.16.8.107:9092', + topic='mqTest02', + parallelism ='2', + partitionKeys = 'channel,pv', + updateMode='upsert' + ); + +upsert模式下发的数据格式:{"channel":"zs","pv":"330",retract:true} +append模式下发的数据格式:{"channel":"zs","pv":"330"} + +``` + +### avro格式: + +如果updateMode='upsert',schemaInfo需要包含retract属性信息。 + +``` +CREATE TABLE MyTable( + channel varchar, + pv varchar + --xctime bigint + )WITH( + type='kafka', + bootstrapServers='172.16.8.107:9092', + groupId='mqTest01', + offsetReset='latest', + topic='mqTest01', + parallelism ='1', + topicIsPattern ='false' + ); + +create table sideTable( + channel varchar, + xccount int, + PRIMARY KEY(channel), + PERIOD FOR SYSTEM_TIME + )WITH( + type='mysql', + url='jdbc:mysql://172.16.8.109:3306/test?charset=utf8', + userName='dtstack', + password='abc123', + tableName='sidetest', + cache = 'LRU', + cacheTTLMs='10000', + parallelism ='1' + + ); + + +CREATE TABLE MyResult( + channel varchar, + pv varchar + )WITH( + --type='console' + type='kafka', + bootstrapServers='172.16.8.107:9092', + topic='mqTest02', + parallelism ='1', + updateMode='upsert', + sinkdatatype = 'avro', + schemaInfo = '{"type":"record","name":"MyResult","fields":[{"name":"channel","type":"string"} + ,{"name":"pv","type":"string"},{"name":"channel","type":"string"}, + {"name":"retract","type":"boolean"}]}' + + ); + + +insert +into + MyResult + select + a.channel as channel, + a.pv as pv + from + MyTable a +``` +### csv格式: + +``` +CREATE TABLE MyTable( + channel varchar, + pv varchar + --xctime bigint + )WITH( + type='kafka', + bootstrapServers='172.16.8.107:9092', + groupId='mqTest01', + offsetReset='latest', + topic='mqTest01', + parallelism ='2', + topicIsPattern ='false' + ); + +create table sideTable( + channel varchar, + xccount int, + PRIMARY KEY(channel), + PERIOD FOR SYSTEM_TIME + )WITH( + type='mysql', + url='jdbc:mysql://172.16.8.109:3306/test?charset=utf8', + userName='dtstack', + password='abc123', + tableName='sidetest', + cache = 'LRU', + cacheTTLMs='10000', + parallelism ='1' + + ); + + +CREATE TABLE MyResult( + channel varchar, + pv varchar + )WITH( + type='kafka', + bootstrapServers='172.16.8.107:9092', + topic='mqTest02', + parallelism ='2', + updateMode='upsert', + sinkdatatype = 'csv', + fieldDelimiter='*' + + + + ); + + +insert +into + MyResult + select + a.channel as channel, + a.pv as pv + from + MyTable a +``` diff --git a/docs/plugin/kafkaSource.md b/docs/plugin/kafkaSource.md index 035e39ffb..95a60a3ff 100644 --- a/docs/plugin/kafkaSource.md +++ b/docs/plugin/kafkaSource.md @@ -1,6 +1,5 @@ ## 1.格式: ``` -数据现在支持json格式{"xx":"bb","cc":"dd"} CREATE TABLE tableName( colName colType, @@ -15,9 +14,8 @@ CREATE TABLE tableName( topic ='topicName', groupId='test', parallelism ='parllNum', - --timezone='America/Los_Angeles', timezone='Asia/Shanghai', - sourcedatatype ='json' #可不设置 + sourcedatatype ='dt_nest' #可不设置 ); ``` @@ -47,8 +45,10 @@ CREATE TABLE tableName( |topicIsPattern | topic是否是正则表达式格式(true|false) |否| false |offsetReset | 读取的topic 的offset初始位置[latest|earliest|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| |parallelism | 并行度设置|否|1| -|sourcedatatype | 数据类型|否|json| -|timezone|时区设置[timezone支持的参数](../timeZone.md)|否|'Asia/Shanghai' +|sourcedatatype | 数据类型,avro,csv,json,dt_nest。dt_nest为默认JSON解析器,能够解析嵌套JSON数据类型,其他仅支持非嵌套格式|否|dt_nest| +|schemaInfo | avro类型使用的schema信息|否|| +|fieldDelimiter |csv类型使用的数据分隔符|否| | | +|timezone|时区设置[timezone支持的参数](timeZone.md)|否|'Asia/Shanghai' **kafka相关参数可以自定义,使用kafka.开头即可。** ``` kafka.consumer.id @@ -169,24 +169,10 @@ CREATE TABLE MyTable( parallelism ='1' ); ``` -# 二、csv格式数据源 -根据字段分隔符进行数据分隔,按顺序匹配sql中配置的列。如数据分隔列数和sql中配置的列数相等直接匹配;如不同参照lengthcheckpolicy策略处理。 -## 1.参数: - -|参数名称|含义|是否必填|默认值| -|----|---|---|---| -|type | kafka09 | 是|| -|bootstrapServers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| -|zookeeperQuorum | kafka zk地址信息(多个之间用逗号分隔)|是|| -|topic | 需要读取的 topic 名称|是|| -|offsetReset | 读取的topic 的offset初始位置[latest|earliest]|否|latest| -|parallelism | 并行度设置 |否|1| -|sourcedatatype | 数据类型|是 |csv| -|fielddelimiter | 字段分隔符|是 || -|lengthcheckpolicy | 单行字段条数检查策略 |否|可选,默认为SKIP,其它可选值为EXCEPTION、PAD。SKIP:字段数目不符合时跳过 。EXCEPTION:字段数目不符合时抛出异常。PAD:按顺序填充,不存在的置为null。| -**kafka相关参数可以自定义,使用kafka.开头即可。** -## 2.样例: +## 7.csv格式数据源 + + ``` CREATE TABLE MyTable( name varchar, @@ -203,186 +189,28 @@ CREATE TABLE MyTable( --topic ='mqTest.*', --topicIsPattern='true' parallelism ='1', - sourcedatatype ='csv', - fielddelimiter ='\|', - lengthcheckpolicy = 'PAD' + sourceDatatype ='csv' ); ``` -# 三、text格式数据源UDF自定义拆分 -Kafka源表数据解析流程:Kafka Source Table -> UDTF ->Realtime Compute -> SINK。从Kakfa读入的数据,都是VARBINARY(二进制)格式,对读入的每条数据,都需要用UDTF将其解析成格式化数据。 - 与其他格式不同,本格式定义DDL必须与以下SQL一摸一样,表中的五个字段顺序务必保持一致: - -## 1. 定义源表,注意:kafka源表DDL字段必须与以下例子一模一样。WITH中参数可改。 -``` -create table kafka_stream( - _topic STRING, - _messageKey STRING, - _message STRING, - _partition INT, - _offset BIGINT, -) with ( - type ='kafka09', - bootstrapServers ='172.16.8.198:9092', - zookeeperQuorum ='172.16.8.198:2181/kafka', - offsetReset ='latest', - topic ='nbTest1', - parallelism ='1', - sourcedatatype='text' - ) -``` -## 2.参数: - -|参数名称|含义|是否必填|默认值| -|----|---|---|---| -|type | kafka09 | 是|| -|bootstrapServers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| -|zookeeperQuorum | kafka zk地址信息(多个之间用逗号分隔)|是|| -|topic | 需要读取的 topic 名称|是|| -|offsetReset | 读取的topic 的offset初始位置[latest|earliest]|否|latest| -|parallelism | 并行度设置|否|1| -|sourcedatatype | 数据类型|否|text| -**kafka相关参数可以自定义,使用kafka.开头即可。** +## 8.avro格式数据源 -## 2.自定义: -从kafka读出的数据,需要进行窗口计算。 按照实时计算目前的设计,滚窗/滑窗等窗口操作,需要(且必须)在源表DDL上定义Watermark。Kafka源表比较特殊。如果要以kafka中message字段中的的Event Time进行窗口操作, -需要先从message字段,使用UDX解析出event time,才能定义watermark。 在kafka源表场景中,需要使用计算列。 假设,kafka中写入的数据如下: -2018-11-11 00:00:00|1|Anna|female整个计算流程为:Kafka SOURCE->UDTF->Realtime Compute->RDS SINK(单一分隔符可直接使用类csv格式模板,自定义适用于更复杂的数据类型,本说明只做参考) - -**SQL** ``` --- 定义解析Kakfa message的UDTF - CREATE FUNCTION kafkapaser AS 'com.XXXX.kafkaUDTF'; - CREATE FUNCTION kafkaUDF AS 'com.XXXX.kafkaUDF'; - -- 定义源表,注意:kafka源表DDL字段必须与以下例子一模一样。WITH中参数可改。 - create table kafka_src ( - _topic STRING, - _messageKey STRING, - _message STRING, - _partition INT, - _offset BIGINT, - ctime AS TO_TIMESTAMP(kafkaUDF(_message)), -- 定义计算列,计算列可理解为占位符,源表中并没有这一列,其中的数据可经过下游计算得出。注意计算里的类型必须为timestamp才能在做watermark。 - watermark for ctime as withoffset(ctime,0) -- 在计算列上定义watermark - ) WITH ( - type = 'kafka010', -- Kafka Source类型,与Kafka版本强相关,目前支持的Kafka版本请参考本文档 - topic = 'test_kafka_topic', - ... - ); - create table rds_sink ( - name VARCHAR, - age INT, - grade VARCHAR, - updateTime TIMESTAMP - ) WITH( - type='mysql', - url='jdbc:mysql://localhost:3306/test', - tableName='test4', - userName='test', - password='XXXXXX' +CREATE TABLE MyTable( + channel varchar, + pv varchar + --xctime bigint + )WITH( + type='kafka', + bootstrapServers='172.16.8.107:9092', + groupId='mqTest01', + offsetReset='latest', + topic='mqTest01', + parallelism ='1', + topicIsPattern ='false', + kafka.group.id='mqTest', + sourceDataType ='avro', + schemaInfo = '{"type":"record","name":"MyResult","fields":[{"name":"channel","type":"string"},{"name":"pv","type":"string"}]}' ); - -- 使用UDTF,将二进制数据解析成格式化数据 - CREATE VIEW input_view ( - name, - age, - grade, - updateTime - ) AS - SELECT - COUNT(*) as cnt, - T.ctime, - T.order, - T.name, - T.sex - from - kafka_src as S, - LATERAL TABLE (kafkapaser _message)) as T ( - ctime, - order, - name, - sex - ) - Group BY T.sex, - TUMBLE(ctime, INTERVAL '1' MINUTE); - -- 对input_view中输出的数据做计算 - CREATE VIEW view2 ( - cnt, - sex - ) AS - SELECT - COUNT(*) as cnt, - T.sex - from - input_view - Group BY sex, TUMBLE(ctime, INTERVAL '1' MINUTE); - -- 使用解析出的格式化数据进行计算,并将结果输出到RDS中 - insert into rds_sink - SELECT - cnt,sex - from view2; - ``` -**UDF&UDTF** + ``` -package com.XXXX; - import com.XXXX.fastjson.JSONObject; - import org.apache.flink.table.functions.TableFunction; - import org.apache.flink.table.types.DataType; - import org.apache.flink.table.types.DataTypes; - import org.apache.flink.types.Row; - import java.io.UnsupportedEncodingException; - /** - 以下例子解析输入Kafka中的JSON字符串,并将其格式化输出 - **/ - public class kafkaUDTF extends TableFunction { - public void eval(byte[] message) { - try { - // 读入一个二进制数据,并将其转换为String格式 - String msg = new String(message, "UTF-8"); - // 提取JSON Object中各字段 - String ctime = Timestamp.valueOf(data.split('\\|')[0]); - String order = data.split('\\|')[1]; - String name = data.split('\\|')[2]; - String sex = data.split('\\|')[3]; - // 将解析出的字段放到要输出的Row()对象 - Row row = new Row(4); - row.setField(0, ctime); - row.setField(1, age); - row.setField(2, grade); - row.setField(3, updateTime); - System.out.println("Kafka message str ==>" + row.toString()); - // 输出一行 - collect(row); - } catch (ClassCastException e) { - System.out.println("Input data format error. Input data " + msg + "is not json string"); - } - } catch (UnsupportedEncodingException e) { - e.printStackTrace(); - } - } - @Override - // 如果返回值是Row,就必须重载实现这个方法,显式地告诉系统返回的字段类型 - // 定义输出Row()对象的字段类型 - public DataType getResultType(Object[] arguments, Class[] argTypes) { - return DataTypes.createRowType(DataTypes.TIMESTAMP,DataTypes.STRING, DataTypes.Integer, DataTypes.STRING,DataTypes.STRING); - } - } - - package com.dp58; - package com.dp58.sql.udx; - import org.apache.flink.table.functions.FunctionContext; - import org.apache.flink.table.functions.ScalarFunction; - public class KafkaUDF extends ScalarFunction { - // 可选,open方法可以不写 - // 需要import org.apache.flink.table.functions.FunctionContext; - public String eval(byte[] message) { - // 读入一个二进制数据,并将其转换为String格式 - String msg = new String(message, "UTF-8"); - return msg.split('\\|')[0]; - } - public long eval(String b, String c) { - return eval(b) + eval(c); - } - //可选,close方法可以不写 - @Override - public void close() { - } - } - ``` + diff --git a/docs/plugin/mongoSide.md b/docs/plugin/mongoSide.md index 90ffa2582..62d2d8698 100644 --- a/docs/plugin/mongoSide.md +++ b/docs/plugin/mongoSide.md @@ -40,8 +40,6 @@ |----|---|---|----| | type |表明 输出表类型 mongo|是|| | address | 连接mongo数据库 jdbcUrl |是|| - | userName | mongo连接用户名|否|| - | password | mongo连接密码|否|| | tableName | mongo表名称|是|| | database | mongo表名称|是|| | cache | 维表缓存策略(NONE/LRU)|否|NONE| @@ -64,7 +62,8 @@ create table sideTable( PERIOD FOR SYSTEM_TIME )WITH( type ='mongo', - address ='172.21.32.1:27017,172.21.32.1:27017', + //mongodb://[username:password@]host1[:port1][,host2[:port2],...[,hostN[:portN]]]/[?options]] + address ='mongodb://172.21.32.1:27017,172.21.32.1:27017', database ='test', tableName ='sidetest', cache ='LRU', diff --git a/docs/plugin/mongoSink.md b/docs/plugin/mongoSink.md index 9900dea12..fdfe93d22 100644 --- a/docs/plugin/mongoSink.md +++ b/docs/plugin/mongoSink.md @@ -33,8 +33,6 @@ CREATE TABLE tableName( |----|----|----|----| |type |表明 输出表类型 mongo|是|| |address | 连接mongo数据库 jdbcUrl |是|| -|userName | mongo连接用户名|否|| -|password | mongo连接密码|否|| |tableName | mongo表名称|是|| |database | mongo表名称|是|| |parallelism | 并行度设置|否|1| @@ -46,9 +44,8 @@ CREATE TABLE MyResult( pv VARCHAR )WITH( type ='mongo', - address ='172.21.32.1:27017,172.21.32.1:27017', - userName ='dtstack', - password ='abc123', + //mongodb://[username:password@]host1[:port1][,host2[:port2],...[,hostN[:portN]]]/[?options]] + address ='mongodb://172.21.32.1:27017,172.21.32.1:27017', database ='test', tableName ='pv', parallelism ='1' diff --git a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java index 491d1af27..5ca81c5ed 100644 --- a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java +++ b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.sink.elasticsearch; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -33,7 +34,6 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.elasticsearch.table.ElasticsearchTableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -164,7 +164,7 @@ public void setBulkFlushMaxActions(int bulkFlushMaxActions) { } @Override - public ElasticsearchSink genStreamSink(TargetTableInfo targetTableInfo) { + public ElasticsearchSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { ElasticsearchTableInfo elasticsearchTableInfo = (ElasticsearchTableInfo) targetTableInfo; esTableInfo = elasticsearchTableInfo; clusterName = elasticsearchTableInfo.getClusterName(); 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 93682de46..7988e597c 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 @@ -21,8 +21,8 @@ package com.dtstack.flink.sql.sink.elasticsearch.table; -import com.dtstack.flink.sql.table.AbsTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; @@ -32,7 +32,7 @@ * @author sishu.yss * @Company: www.dtstack.com */ -public class ElasticsearchSinkParser extends AbsTableParser { +public class ElasticsearchSinkParser extends AbstractTableParser { private static final String KEY_ES_ADDRESS = "address"; @@ -56,7 +56,7 @@ protected boolean fieldNameNeedsUpperCase() { } @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { ElasticsearchTableInfo elasticsearchTableInfo = new ElasticsearchTableInfo(); elasticsearchTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, elasticsearchTableInfo); diff --git a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java index 9681742c2..f254d5b4e 100644 --- a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java +++ b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java @@ -21,7 +21,7 @@ package com.dtstack.flink.sql.sink.elasticsearch.table; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import com.google.common.base.Preconditions; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang3.math.NumberUtils; @@ -33,7 +33,7 @@ * @author sishu.yss * @Company: www.dtstack.com */ -public class ElasticsearchTableInfo extends TargetTableInfo { +public class ElasticsearchTableInfo extends AbstractTargetTableInfo { private static final String CURR_TYPE = "elasticsearch"; diff --git a/elasticsearch6/elasticsearch6-side/elasticsearch6-all-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AllReqRow.java b/elasticsearch6/elasticsearch6-side/elasticsearch6-all-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AllReqRow.java index 648f09fba..b8cf2c46a 100644 --- a/elasticsearch6/elasticsearch6-side/elasticsearch6-all-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AllReqRow.java +++ b/elasticsearch6/elasticsearch6-side/elasticsearch6-all-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AllReqRow.java @@ -18,24 +18,22 @@ package com.dtstack.flink.sql.side.elasticsearch6; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.table.runtime.types.CRow; -import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; -import org.apache.flink.types.Row; -import org.apache.flink.util.Collector; - -import com.dtstack.flink.sql.side.AllReqRow; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.BaseAllReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.elasticsearch6.table.Elasticsearch6SideTableInfo; import com.dtstack.flink.sql.side.elasticsearch6.util.Es6Util; import com.dtstack.flink.sql.side.elasticsearch6.util.SwitchUtil; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import org.apache.calcite.sql.JoinType; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.runtime.types.CRow; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.RequestOptions; @@ -49,7 +47,6 @@ import java.io.IOException; import java.io.Serializable; -import java.sql.SQLException; import java.sql.Timestamp; import java.util.Calendar; import java.util.List; @@ -60,7 +57,7 @@ * @author yinxi * @date 2020/1/13 - 1:00 */ -public class Elasticsearch6AllReqRow extends AllReqRow implements Serializable { +public class Elasticsearch6AllReqRow extends BaseAllReqRow implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch6AllReqRow.class); @@ -70,7 +67,7 @@ public class Elasticsearch6AllReqRow extends AllReqRow implements Serializable { private SearchRequest searchRequest; private BoolQueryBuilder boolQueryBuilder; - public Elasticsearch6AllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public Elasticsearch6AllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new Elasticsearch6AllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } diff --git a/elasticsearch6/elasticsearch6-side/elasticsearch6-all-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AllSideInfo.java b/elasticsearch6/elasticsearch6-side/elasticsearch6-all-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AllSideInfo.java index bf21b3ca8..d353a583d 100644 --- a/elasticsearch6/elasticsearch6-side/elasticsearch6-all-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AllSideInfo.java +++ b/elasticsearch6/elasticsearch6-side/elasticsearch6-all-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AllSideInfo.java @@ -18,12 +18,13 @@ package com.dtstack.flink.sql.side.elasticsearch6; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.BaseSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; + import com.dtstack.flink.sql.util.ParseUtils; import com.google.common.collect.Lists; import org.apache.calcite.sql.SqlNode; @@ -35,15 +36,15 @@ * @author yinxi * @date 2020/1/13 - 1:01 */ -public class Elasticsearch6AllSideInfo extends SideInfo { +public class Elasticsearch6AllSideInfo extends BaseSideInfo { - public Elasticsearch6AllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public Elasticsearch6AllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + public void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo) { } diff --git a/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java b/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java index f67177fe4..1f4bd1bf1 100644 --- a/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java +++ b/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java @@ -18,6 +18,12 @@ package com.dtstack.flink.sql.side.elasticsearch6; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.BaseAsyncReqRow; +import com.dtstack.flink.sql.side.CacheMissVal; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.PredicateInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.ResultFuture; @@ -26,7 +32,6 @@ import org.apache.flink.types.Row; import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.*; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.elasticsearch6.table.Elasticsearch6SideTableInfo; import com.dtstack.flink.sql.side.elasticsearch6.util.Es6Util; @@ -57,14 +62,14 @@ * @author yinxi * @date 2020/2/13 - 13:10 */ -public class Elasticsearch6AsyncReqRow extends AsyncReqRow implements Serializable { +public class Elasticsearch6AsyncReqRow extends BaseAsyncReqRow implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch6AsyncReqRow.class); private transient RestHighLevelClient rhlClient; private SearchRequest searchRequest; private List sqlJoinCompareOperate = Lists.newArrayList(); - public Elasticsearch6AsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public Elasticsearch6AsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new Elasticsearch6AsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); SqlNode conditionNode = joinInfo.getCondition(); ParseUtils.parseJoinCompareOperate(conditionNode, sqlJoinCompareOperate); diff --git a/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncSideInfo.java b/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncSideInfo.java index 7b3a2f9e7..4118988ab 100644 --- a/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncSideInfo.java +++ b/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncSideInfo.java @@ -18,18 +18,17 @@ package com.dtstack.flink.sql.side.elasticsearch6; -import org.apache.flink.api.java.typeutils.RowTypeInfo; - +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.BaseSideInfo; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; import com.google.common.collect.Lists; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; import java.util.List; @@ -37,15 +36,15 @@ * @author yinxi * @date 2020/2/13 - 13:09 */ -public class Elasticsearch6AsyncSideInfo extends SideInfo { +public class Elasticsearch6AsyncSideInfo extends BaseSideInfo { - public Elasticsearch6AsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public Elasticsearch6AsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + public void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo) { String sideTableName = joinInfo.getSideTableName(); SqlNode conditionNode = joinInfo.getCondition(); diff --git a/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/table/Elasticsearch6SideParser.java b/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/table/Elasticsearch6SideParser.java index 13dfe2995..1b39bbf0f 100644 --- a/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/table/Elasticsearch6SideParser.java +++ b/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/table/Elasticsearch6SideParser.java @@ -19,8 +19,8 @@ package com.dtstack.flink.sql.side.elasticsearch6.table; import com.dtstack.flink.sql.side.elasticsearch6.util.ClassUtil; -import com.dtstack.flink.sql.table.AbsSideTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractSideTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import org.apache.commons.lang3.StringUtils; @@ -30,7 +30,7 @@ * @author yinxi * @date 2020/1/13 - 1:07 */ -public class Elasticsearch6SideParser extends AbsSideTableParser { +public class Elasticsearch6SideParser extends AbstractSideTableParser { private static final String KEY_ES6_ADDRESS = "address"; @@ -55,7 +55,7 @@ protected boolean fieldNameNeedsUpperCase() { } @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { Elasticsearch6SideTableInfo elasticsearch6SideTableInfo = new Elasticsearch6SideTableInfo(); elasticsearch6SideTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, elasticsearch6SideTableInfo); diff --git a/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/table/Elasticsearch6SideTableInfo.java b/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/table/Elasticsearch6SideTableInfo.java index 0afe2d59e..a01c90a53 100644 --- a/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/table/Elasticsearch6SideTableInfo.java +++ b/elasticsearch6/elasticsearch6-side/elasticsearch6-side-core/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/table/Elasticsearch6SideTableInfo.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.side.elasticsearch6.table; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.google.common.base.Preconditions; import org.elasticsearch.search.builder.SearchSourceBuilder; @@ -26,7 +26,7 @@ * @author yinxi * @date 2020/1/13 - 15:00 */ -public class Elasticsearch6SideTableInfo extends SideTableInfo { +public class Elasticsearch6SideTableInfo extends AbstractSideTableInfo { private static final String CURR_TYPE = "elasticsearch6"; 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 cc591789e..3d9e4d62a 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 @@ -18,8 +18,8 @@ package com.dtstack.flink.sql.side.elasticsearch6.util; +import com.dtstack.flink.sql.side.BaseSideInfo; import com.dtstack.flink.sql.side.PredicateInfo; -import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.elasticsearch6.table.Elasticsearch6SideTableInfo; import org.apache.commons.lang3.StringUtils; import org.apache.http.HttpHost; @@ -98,7 +98,7 @@ public static RestHighLevelClient getClient(String esAddress, Boolean isAuthMesh } // add index and type to search request - public static SearchRequest setSearchRequest(SideInfo sideInfo) { + public static SearchRequest setSearchRequest(BaseSideInfo sideInfo) { SearchRequest searchRequest = new SearchRequest(); Elasticsearch6SideTableInfo tableInfo = (Elasticsearch6SideTableInfo) sideInfo.getSideTableInfo(); // determine existence of index @@ -129,7 +129,7 @@ public static SearchRequest setSearchRequest(SideInfo sideInfo) { } // build where cause - public static BoolQueryBuilder setPredicateclause(SideInfo sideInfo) { + public static BoolQueryBuilder setPredicateclause(BaseSideInfo sideInfo) { BoolQueryBuilder boolQueryBuilder = null; List predicateInfoes = sideInfo.getSideTableInfo().getPredicateInfoes(); @@ -144,7 +144,7 @@ public static BoolQueryBuilder setPredicateclause(SideInfo sideInfo) { } // build filter condition - public static BoolQueryBuilder buildFilterCondition(BoolQueryBuilder boolQueryBuilder, PredicateInfo info, SideInfo sideInfo) { + public static BoolQueryBuilder buildFilterCondition(BoolQueryBuilder boolQueryBuilder, PredicateInfo info, BaseSideInfo sideInfo) { switch (info.getOperatorKind()) { case "IN": return boolQueryBuilder.must(QueryBuilders.termsQuery(textConvertToKeyword(info.getFieldName(), sideInfo), removeSpaceAndApostrophe(info.getCondition()))); @@ -202,7 +202,7 @@ public static String[] removeSpaceAndApostrophe(String str) { } // prevent word segmentation - public static String textConvertToKeyword(String fieldName, SideInfo sideInfo) { + public static String textConvertToKeyword(String fieldName, BaseSideInfo sideInfo) { String[] sideFieldTypes = sideInfo.getSideTableInfo().getFieldTypes(); int fieldIndex = sideInfo.getSideTableInfo().getFieldList().indexOf(fieldName.trim()); String fieldType = sideFieldTypes[fieldIndex]; 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 2646c50e9..b7d9de6fc 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 @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.sink.elasticsearch; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -31,7 +32,6 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.elasticsearch.table.ElasticsearchTableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; import com.google.common.collect.Maps; import org.apache.commons.lang.StringUtils; import org.apache.http.HttpHost; @@ -128,7 +128,7 @@ public void emitDataStream(DataStream> dataStream) { } @Override - public ElasticsearchSink genStreamSink(TargetTableInfo targetTableInfo) { + public ElasticsearchSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { esTableInfo = (ElasticsearchTableInfo) targetTableInfo; clusterName = esTableInfo.getClusterName(); index = esTableInfo.getIndex(); 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 8902b953c..22c2b72bc 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 @@ -18,8 +18,8 @@ package com.dtstack.flink.sql.sink.elasticsearch.table; -import com.dtstack.flink.sql.table.AbsTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; +import com.dtstack.flink.sql.table.AbstractTableParser; import com.dtstack.flink.sql.util.MathUtil; import org.apache.commons.lang3.StringUtils; @@ -29,7 +29,7 @@ * @author yinxi * @date 2020/1/9 - 15:06 */ -public class ElasticsearchSinkParser extends AbsTableParser { +public class ElasticsearchSinkParser extends AbstractTableParser { private static final String KEY_ES6_ADDRESS = "address"; @@ -55,7 +55,7 @@ protected boolean fieldNameNeedsUpperCase() { } @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { ElasticsearchTableInfo elasticsearchTableInfo = new ElasticsearchTableInfo(); elasticsearchTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, elasticsearchTableInfo); diff --git a/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java b/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java index 3cc3dd9ff..d0eef18f0 100644 --- a/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java +++ b/elasticsearch6/elasticsearch6-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.sink.elasticsearch.table; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import com.google.common.base.Preconditions; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang3.math.NumberUtils; @@ -29,7 +29,7 @@ * @author yinxi * @date 2020/1/9 - 15:06 */ -public class ElasticsearchTableInfo extends TargetTableInfo { +public class ElasticsearchTableInfo extends AbstractTargetTableInfo { private static final String CURR_TYPE = "elasticsearch6"; diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java index 11380eea6..ed2931fff 100644 --- a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java @@ -45,7 +45,7 @@ import java.util.*; import java.util.concurrent.atomic.AtomicReference; -public class HbaseAllReqRow extends AllReqRow { +public class HbaseAllReqRow extends BaseAllReqRow { private static final Logger LOG = LoggerFactory.getLogger(HbaseAllReqRow.class); @@ -55,7 +55,7 @@ public class HbaseAllReqRow extends AllReqRow { private AtomicReference>> cacheRef = new AtomicReference<>(); - public HbaseAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public HbaseAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new HbaseAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); tableName = ((HbaseSideTableInfo)sideTableInfo).getTableName(); @@ -134,7 +134,7 @@ public void flatMap(CRow input, Collector out) throws Exception { Map cacheList = null; - SideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); + AbstractSideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; if (hbaseSideTableInfo.isPreRowKey()) { for (Map.Entry> entry : cacheRef.get().entrySet()) { @@ -153,7 +153,7 @@ public void flatMap(CRow input, Collector out) throws Exception { } private void loadData(Map> tmpCache) throws SQLException { - SideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); + AbstractSideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; Configuration conf = new Configuration(); conf.set("hbase.zookeeper.quorum", hbaseSideTableInfo.getHost()); @@ -182,9 +182,17 @@ private void loadData(Map> tmpCache) throws SQLExcep LOG.error("", e); } finally { try { - conn.close(); - table.close(); - resultScanner.close(); + if (null != conn && !conn.isClosed()) { + conn.close(); + } + + if (null != table) { + table.close(); + } + + if (null != resultScanner) { + resultScanner.close(); + } } catch (IOException e) { LOG.error("", e); } diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java index ea51f46e4..663c2927b 100644 --- a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java @@ -22,8 +22,8 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.BaseSideInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -31,16 +31,16 @@ import java.util.List; -public class HbaseAllSideInfo extends SideInfo { +public class HbaseAllSideInfo extends BaseSideInfo { private RowKeyBuilder rowKeyBuilder; - public HbaseAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public HbaseAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + public void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo) { rowKeyBuilder = new RowKeyBuilder(); if(sideTableInfo.getPrimaryKeys().size() < 1){ throw new RuntimeException("Primary key dimension table must be filled"); 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 251b88034..24ee01786 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 @@ -21,12 +21,12 @@ package com.dtstack.flink.sql.side.hbase; import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.AsyncReqRow; +import com.dtstack.flink.sql.side.BaseAsyncReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.cache.CacheObj; -import com.dtstack.flink.sql.side.hbase.rowkeydealer.AbsRowKeyModeDealer; +import com.dtstack.flink.sql.side.hbase.rowkeydealer.AbstractRowKeyModeDealer; import com.dtstack.flink.sql.side.hbase.rowkeydealer.PreRowKeyModeDealerDealer; import com.dtstack.flink.sql.side.hbase.rowkeydealer.RowKeyEqualModeDealer; import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; @@ -58,7 +58,7 @@ * @author xuchao */ -public class HbaseAsyncReqRow extends AsyncReqRow { +public class HbaseAsyncReqRow extends BaseAsyncReqRow { private static final long serialVersionUID = 2098635104857937717L; @@ -73,13 +73,13 @@ public class HbaseAsyncReqRow extends AsyncReqRow { private transient HBaseClient hBaseClient; - private transient AbsRowKeyModeDealer rowKeyMode; + private transient AbstractRowKeyModeDealer rowKeyMode; private String tableName; private String[] colNames; - public HbaseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public HbaseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new HbaseAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); tableName = ((HbaseSideTableInfo)sideTableInfo).getTableName(); @@ -89,7 +89,7 @@ public HbaseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List colRefType; - public HbaseAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public HbaseAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + public void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo) { rowKeyBuilder = new RowKeyBuilder(); if(sideTableInfo.getPrimaryKeys().size() < 1){ throw new RuntimeException("Primary key dimension table must be filled"); diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbstractRowKeyModeDealer.java similarity index 87% rename from hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java rename to hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbstractRowKeyModeDealer.java index 1506bc440..90ee289bd 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbstractRowKeyModeDealer.java @@ -21,7 +21,7 @@ package com.dtstack.flink.sql.side.hbase.rowkeydealer; import com.dtstack.flink.sql.side.FieldInfo; -import com.dtstack.flink.sql.side.cache.AbsSideCache; +import com.dtstack.flink.sql.side.cache.AbstractSideCache; import org.apache.calcite.sql.JoinType; import com.google.common.collect.Maps; import org.apache.flink.streaming.api.functions.async.ResultFuture; @@ -41,7 +41,7 @@ * @author xuchao */ -public abstract class AbsRowKeyModeDealer { +public abstract class AbstractRowKeyModeDealer { protected Map colRefType; @@ -60,9 +60,9 @@ public abstract class AbsRowKeyModeDealer { protected Map sideFieldIndex = Maps.newHashMap(); - public AbsRowKeyModeDealer(Map colRefType, String[] colNames, HBaseClient hBaseClient, - boolean openCache, JoinType joinType, List outFieldInfoList, - Map inFieldIndex, Map sideFieldIndex){ + public AbstractRowKeyModeDealer(Map colRefType, String[] colNames, HBaseClient hBaseClient, + boolean openCache, JoinType joinType, List outFieldInfoList, + Map inFieldIndex, Map sideFieldIndex){ this.colRefType = colRefType; this.colNames = colNames; this.hBaseClient = hBaseClient; @@ -111,5 +111,5 @@ protected Row fillData(Row input, Object sideInput){ } public abstract void asyncGetData(String tableName, String rowKeyStr, CRow input, ResultFuture resultFuture, - AbsSideCache sideCache); + AbstractSideCache sideCache); } diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java index a41ffe916..dcd50131d 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java @@ -23,7 +23,7 @@ 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.cache.AbsSideCache; +import com.dtstack.flink.sql.side.cache.AbstractSideCache; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.hbase.utils.HbaseUtils; import com.google.common.collect.Maps; @@ -44,7 +44,6 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; @@ -55,7 +54,7 @@ * @author xuchao */ -public class PreRowKeyModeDealerDealer extends AbsRowKeyModeDealer { +public class PreRowKeyModeDealerDealer extends AbstractRowKeyModeDealer { private static final Logger LOG = LoggerFactory.getLogger(PreRowKeyModeDealerDealer.class); @@ -67,7 +66,7 @@ public PreRowKeyModeDealerDealer(Map colRefType, String[] colNam @Override public void asyncGetData(String tableName, String rowKeyStr, CRow input, ResultFuture resultFuture, - AbsSideCache sideCache) { + AbstractSideCache sideCache) { Scanner prefixScanner = hBaseClient.newScanner(tableName); ScanFilter scanFilter = new RowFilter(CompareFilter.CompareOp.EQUAL, new BinaryPrefixComparator(Bytes.UTF8(rowKeyStr))); prefixScanner.setFilter(scanFilter); @@ -80,7 +79,7 @@ public void asyncGetData(String tableName, String rowKeyStr, CRow input, ResultF } - private String dealOneRow(ArrayList> args, String rowKeyStr, CRow input, ResultFuture resultFuture, AbsSideCache sideCache) { + private String dealOneRow(ArrayList> args, String rowKeyStr, CRow input, ResultFuture resultFuture, AbstractSideCache sideCache) { if(args == null || args.size() == 0){ dealMissKey(input, resultFuture); if (openCache) { @@ -111,8 +110,7 @@ private String dealOneRow(ArrayList> args, String rowKeyStr, for (String key : colNames) { Object val = sideMap.get(key); if (val == null) { - System.out.println("can't get data with column " + key); - LOG.error("can't get data with column " + key); + LOG.error("can't get data with column {}", key); } sideVal.add(val); diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java index b20c316db..41208c7f3 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java @@ -23,7 +23,7 @@ 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.cache.AbsSideCache; +import com.dtstack.flink.sql.side.cache.AbstractSideCache; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.hbase.utils.HbaseUtils; import com.google.common.collect.Maps; @@ -49,7 +49,7 @@ * @author xuchao */ -public class RowKeyEqualModeDealer extends AbsRowKeyModeDealer { +public class RowKeyEqualModeDealer extends AbstractRowKeyModeDealer { private static final Logger LOG = LoggerFactory.getLogger(RowKeyEqualModeDealer.class); @@ -62,7 +62,7 @@ public RowKeyEqualModeDealer(Map colRefType, String[] colNames, @Override public void asyncGetData(String tableName, String rowKeyStr, CRow input, ResultFuture resultFuture, - AbsSideCache sideCache){ + AbstractSideCache sideCache){ //TODO 是否有查询多个col family 和多个col的方法 GetRequest getRequest = new GetRequest(tableName, rowKeyStr); hBaseClient.get(getRequest).addCallbacks(arg -> { @@ -86,8 +86,7 @@ public void asyncGetData(String tableName, String rowKeyStr, CRow input, ResultF for(String key : colNames){ Object val = sideMap.get(key); if(val == null){ - System.out.println("can't get data with column " + key); - LOG.error("can't get data with column " + key); + LOG.error("can't get data with column {}", key); } sideVal.add(val); diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/AbstractReplaceOperator.java similarity index 93% rename from hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java rename to hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/AbstractReplaceOperator.java index c3c9b5cfe..c75fa43c1 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/AbstractReplaceOperator.java @@ -31,11 +31,11 @@ * @author xuchao */ -public abstract class ReplaceOperator { +public abstract class AbstractReplaceOperator { private EReplaceOpType opType; - public ReplaceOperator(EReplaceOpType opType){ + public AbstractReplaceOperator(EReplaceOpType opType){ this.opType = opType; } diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/Md5ReplaceOperator.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/Md5ReplaceOperator.java index af5e24dd6..ab826c98f 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/Md5ReplaceOperator.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/Md5ReplaceOperator.java @@ -30,7 +30,7 @@ * @author xuchao */ -public class Md5ReplaceOperator extends ReplaceOperator{ +public class Md5ReplaceOperator extends AbstractReplaceOperator { public Md5ReplaceOperator(EReplaceOpType opType) { super(opType); diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceOpType.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceOpType.java index 834edf9ae..ac9a3b87c 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceOpType.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceOpType.java @@ -28,5 +28,12 @@ */ public enum EReplaceOpType { - NO_FUNC, MD5_FUNC; + /** + * 没有func + */ + NO_FUNC, + /** + * md5 func + */ + MD5_FUNC; } diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceType.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceType.java index f4b039ef8..bb4b1556b 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceType.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceType.java @@ -28,7 +28,16 @@ * @author xuchao */ public enum EReplaceType { + /** + * 参数 + */ PARAM, + /** + * 函数 + */ FUNC, + /** + * 常量 + */ CONSTANT; } diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java index 7b627da4d..cf44c181f 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java @@ -20,15 +20,15 @@ package com.dtstack.flink.sql.side.hbase.table; -import com.dtstack.flink.sql.table.AbsSideTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractSideTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; -import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; +import static com.dtstack.flink.sql.table.AbstractTableInfo.PARALLELISM_KEY; /** * hbase field information must include the definition of an alias -> sql which does not allow ":" @@ -37,7 +37,7 @@ * @author xuchao */ -public class HbaseSideParser extends AbsSideTableParser { +public class HbaseSideParser extends AbstractSideTableParser { private final static String FIELD_KEY = "fieldKey"; @@ -58,7 +58,7 @@ public HbaseSideParser() { } @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { HbaseSideTableInfo hbaseTableInfo = new HbaseSideTableInfo(); hbaseTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, hbaseTableInfo); @@ -77,7 +77,7 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map { +public class HbaseOutputFormat extends AbstractDtRichOutputFormat { private static final Logger LOG = LoggerFactory.getLogger(HbaseOutputFormat.class); 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 447b02921..9308725a2 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 @@ -22,7 +22,7 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.hbase.table.HbaseTableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; 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,7 +30,6 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.RetractStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; @@ -59,7 +58,7 @@ public HbaseSink() { } @Override - public HbaseSink genStreamSink(TargetTableInfo targetTableInfo) { + public HbaseSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { HbaseTableInfo hbaseTableInfo = (HbaseTableInfo) targetTableInfo; this.zookeeperQuorum = hbaseTableInfo.getHost(); this.port = hbaseTableInfo.getPort(); diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java index 064b13e69..8d8996a23 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java @@ -22,8 +22,8 @@ import com.dtstack.flink.sql.enums.EUpdateMode; -import com.dtstack.flink.sql.table.AbsTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.MathUtil; import org.apache.commons.lang3.StringUtils; @@ -32,14 +32,14 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; +import static com.dtstack.flink.sql.table.AbstractTableInfo.PARALLELISM_KEY; /** * Date: 2018/09/14 * Company: www.dtstack.com * @author sishu.yss */ -public class HbaseSinkParser extends AbsTableParser { +public class HbaseSinkParser extends AbstractTableParser { public static final String HBASE_ZOOKEEPER_QUORUM = "zookeeperQuorum"; @@ -60,7 +60,7 @@ protected boolean fieldNameNeedsUpperCase() { } @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { HbaseTableInfo hbaseTableInfo = new HbaseTableInfo(); hbaseTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, hbaseTableInfo); diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java index 610eb5c30..62f7c9936 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java @@ -21,7 +21,7 @@ package com.dtstack.flink.sql.sink.hbase.table; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import com.google.common.base.Preconditions; import java.util.Map; @@ -30,7 +30,7 @@ * Company: www.dtstack.com * @author sishu.yss */ -public class HbaseTableInfo extends TargetTableInfo { +public class HbaseTableInfo extends AbstractTargetTableInfo { private static final String CURR_TYPE = "hbase"; 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 9cd8c9194..10938308a 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 @@ -20,12 +20,10 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.impala.table.ImpalaSideTableInfo; -import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; -import com.dtstack.flink.sql.util.DtStringUtil; +import com.dtstack.flink.sql.side.rdb.all.AbstractRdbAllReqRow; import com.dtstack.flink.sql.util.JDBCUtils; -import com.google.common.collect.Maps; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; @@ -36,7 +34,6 @@ import java.sql.Connection; import java.sql.DriverManager; import java.util.List; -import java.util.Map; /** * side operator with cache for all(period reload) @@ -46,7 +43,7 @@ * @author xiuzhu */ -public class ImpalaAllReqRow extends RdbAllReqRow { +public class ImpalaAllReqRow extends AbstractRdbAllReqRow { private static final long serialVersionUID = 2098635140857937717L; @@ -56,13 +53,13 @@ public class ImpalaAllReqRow extends RdbAllReqRow { private ImpalaSideTableInfo impalaSideTableInfo; - public ImpalaAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public ImpalaAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new ImpalaAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); this.impalaSideTableInfo = (ImpalaSideTableInfo) sideTableInfo; } @Override - public Connection getConn(String dbURL, String userName, String password) { + public Connection getConn(String dbUrl, String userName, String password) { try { Connection connection ; String url = getUrl(); diff --git a/impala/impala-side/impala-all-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAllSideInfo.java b/impala/impala-side/impala-all-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAllSideInfo.java index d40c5f48c..a5e643967 100644 --- a/impala/impala-side/impala-all-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAllSideInfo.java +++ b/impala/impala-side/impala-all-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAllSideInfo.java @@ -20,10 +20,9 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.impala.table.ImpalaSideTableInfo; import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; -import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import java.util.List; @@ -32,7 +31,7 @@ public class ImpalaAllSideInfo extends RdbAllSideInfo { - public ImpalaAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public ImpalaAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @@ -61,7 +60,7 @@ private String buildPartitionCondition(ImpalaSideTableInfo impalaSideTableInfo) private String getPartitionVaule(String fieldType, List values) { String partitionVaule = values.stream().map(val -> { - return (fieldType.toLowerCase().equals("string") || fieldType.toLowerCase().equals("varchar")) ? "'" + val + "'" : val.toString(); + return ("string".equals(fieldType.toLowerCase()) || "varchar".equals(fieldType.toLowerCase())) ? "'" + val + "'" : val.toString(); }).collect(Collectors.joining(" , ")).toString(); return partitionVaule; 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 2b76ec26c..b5e0ed5fb 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 @@ -20,7 +20,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +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 io.vertx.core.Vertx; @@ -50,7 +50,7 @@ public class ImpalaAsyncReqRow extends RdbAsyncReqRow { private final static String IMPALA_DRIVER = "com.cloudera.impala.jdbc41.Driver"; - public ImpalaAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public ImpalaAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new ImpalaAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @@ -66,7 +66,10 @@ public void open(Configuration parameters) throws Exception { .put("provider_class", DT_PROVIDER_CLASS) .put("idle_connection_test_period", 300) .put("test_connection_on_checkin", DEFAULT_TEST_CONNECTION_ON_CHECKIN) - .put("max_idle_time", 600); + .put("max_idle_time", 600) + .put("preferred_test_query", PREFERRED_TEST_QUERY_SQL) + .put("idle_connection_test_period", DEFAULT_IDLE_CONNECTION_TEST_PEROID) + .put("test_connection_on_checkin", DEFAULT_TEST_CONNECTION_ON_CHECKIN); System.setProperty("vertx.disableFileCPResolving", "true"); @@ -75,8 +78,7 @@ public void open(Configuration parameters) throws Exception { vo.setWorkerPoolSize(impalaSideTableInfo.getAsyncPoolSize()); vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); - - setRdbSQLClient(JDBCClient.createNonShared(vertx, impalaClientConfig)); + setRdbSqlClient(JDBCClient.createNonShared(vertx, impalaClientConfig)); } diff --git a/impala/impala-side/impala-async-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAsyncSideInfo.java b/impala/impala-side/impala-async-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAsyncSideInfo.java index 53559b9a7..14fbf0ed3 100644 --- a/impala/impala-side/impala-async-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAsyncSideInfo.java +++ b/impala/impala-side/impala-async-side/src/main/java/com/dtstack/flink/sql/side/impala/ImpalaAsyncSideInfo.java @@ -20,7 +20,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.impala.table.ImpalaSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -38,7 +38,7 @@ public class ImpalaAsyncSideInfo extends RdbAsyncSideInfo { - public ImpalaAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public ImpalaAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @@ -69,7 +69,7 @@ private String buildPartitionCondition(ImpalaSideTableInfo impalaSideTableInfo) private String getPartitionVaule(String fieldType, List values) { String partitionVaule = values.stream().map(val -> { - return (fieldType.toLowerCase().equals("string") || fieldType.toLowerCase().equals("varchar")) ? "'" + val + "'" : val.toString(); + return ("string".equals(fieldType.toLowerCase()) || "varchar".equals(fieldType.toLowerCase())) ? "'" + val + "'" : val.toString(); }).collect(Collectors.joining(" , ")).toString(); return partitionVaule; diff --git a/impala/impala-side/impala-side-core/src/main/java/com/dtstack/flink/sql/side/impala/table/ImpalaSideParser.java b/impala/impala-side/impala-side-core/src/main/java/com/dtstack/flink/sql/side/impala/table/ImpalaSideParser.java index b6a54ca3f..2f1d73be7 100644 --- a/impala/impala-side/impala-side-core/src/main/java/com/dtstack/flink/sql/side/impala/table/ImpalaSideParser.java +++ b/impala/impala-side/impala-side-core/src/main/java/com/dtstack/flink/sql/side/impala/table/ImpalaSideParser.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.side.impala.table; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.lang3.StringUtils; @@ -48,7 +48,7 @@ public class ImpalaSideParser extends RdbSideParser { private static final String CURR_TYPE = "impala"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { ImpalaSideTableInfo impalaSideTableInfo = new ImpalaSideTableInfo(); impalaSideTableInfo.setType(CURR_TYPE); impalaSideTableInfo.setName(tableName); @@ -147,6 +147,8 @@ public Class dbTypeConvertToJavaType(String fieldType) { 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/ImpalaSink.java b/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/ImpalaSink.java index 41fa00926..cd1e1e945 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 @@ -21,9 +21,9 @@ 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.RdbSink; +import com.dtstack.flink.sql.sink.rdb.AbstractRdbSink; import com.dtstack.flink.sql.sink.rdb.format.JDBCUpsertOutputFormat; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; @@ -36,7 +36,7 @@ * @author xiuzhu */ -public class ImpalaSink extends RdbSink implements IStreamSinkGener { +public class ImpalaSink extends AbstractRdbSink implements IStreamSinkGener { private ImpalaTableInfo impalaTableInfo; @@ -47,7 +47,7 @@ public ImpalaSink() { @Override public JDBCUpsertOutputFormat getOutputFormat() { JDBCOptions jdbcOptions = JDBCOptions.builder() - .setDBUrl(getImpalaJdbcUrl()) + .setDbUrl(getImpalaJdbcUrl()) .setDialect(jdbcDialect) .setUsername(userName) .setPassword(password) @@ -70,8 +70,8 @@ public JDBCUpsertOutputFormat getOutputFormat() { public String getImpalaJdbcUrl() { Integer authMech = impalaTableInfo.getAuthMech(); - String newUrl = dbURL; - StringBuffer urlBuffer = new StringBuffer(dbURL); + String newUrl = dbUrl; + StringBuffer urlBuffer = new StringBuffer(dbUrl); if (authMech == EAuthMech.NoAuthentication.getType()) { return newUrl; } else if (authMech == EAuthMech.Kerberos.getType()) { @@ -121,7 +121,7 @@ public String getImpalaJdbcUrl() { } @Override - public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { + public AbstractRdbSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { super.genStreamSink(targetTableInfo); this.impalaTableInfo = (ImpalaTableInfo) targetTableInfo; return this; 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 4921f5e51..5b6fdeafe 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 @@ -19,12 +19,11 @@ package com.dtstack.flink.sql.sink.impala.table; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.math.BigDecimal; import java.sql.Timestamp; -import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -42,7 +41,7 @@ public class ImpalaSinkParser extends RdbSinkParser { private static final String CURR_TYPE = "impala"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { ImpalaTableInfo impalaTableInfo = new ImpalaTableInfo(); impalaTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, impalaTableInfo); @@ -117,6 +116,8 @@ public Class dbTypeConvertToJavaType(String fieldType) { return String.class; case "timestamp": return Timestamp.class; + default: + break; } throw new RuntimeException("不支持 " + fieldType + " 类型"); diff --git a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaProducerFactory.java b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaProducerFactory.java index 88c2ca939..9958a2544 100644 --- a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaProducerFactory.java +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaProducerFactory.java @@ -19,16 +19,16 @@ import com.dtstack.flink.sql.format.FormatType; import com.dtstack.flink.sql.format.SerializationMetricWrapper; +import com.dtstack.flink.sql.sink.kafka.serialization.AvroCRowSerializationSchema; +import com.dtstack.flink.sql.sink.kafka.serialization.CsvCRowSerializationSchema; +import com.dtstack.flink.sql.sink.kafka.serialization.JsonCRowSerializationSchema; import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.avro.AvroRowSerializationSchema; -import org.apache.flink.formats.csv.CsvRowSerializationSchema; -import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.types.Row; +import org.apache.flink.table.runtime.types.CRow; import java.util.Optional; import java.util.Properties; @@ -51,42 +51,37 @@ public abstract class AbstractKafkaProducerFactory { * @param partitioner * @return */ - public abstract RichSinkFunction createKafkaProducer(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation typeInformation, Properties properties, Optional> partitioner, String[] partitionKeys); + public abstract RichSinkFunction createKafkaProducer(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation typeInformation, Properties properties, Optional> partitioner, String[] partitionKeys); - protected SerializationMetricWrapper createSerializationMetricWrapper(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation typeInformation) { - return new SerializationMetricWrapper(createSerializationSchema(kafkaSinkTableInfo, typeInformation)); + protected SerializationMetricWrapper createSerializationMetricWrapper(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation typeInformation) { + SerializationSchema serializationSchema = createSerializationSchema(kafkaSinkTableInfo, typeInformation); + return new SerializationMetricWrapper(serializationSchema); } - private SerializationSchema createSerializationSchema(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation typeInformation) { - SerializationSchema serializationSchema = null; + private SerializationSchema createSerializationSchema(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation typeInformation) { + SerializationSchema serializationSchema = null; if (FormatType.JSON.name().equalsIgnoreCase(kafkaSinkTableInfo.getSinkDataType())) { - if (StringUtils.isNotBlank(kafkaSinkTableInfo.getSchemaString())) { - serializationSchema = new JsonRowSerializationSchema(kafkaSinkTableInfo.getSchemaString()); + serializationSchema = new JsonCRowSerializationSchema(kafkaSinkTableInfo.getSchemaString(), kafkaSinkTableInfo.getUpdateMode()); } else if (typeInformation != null && typeInformation.getArity() != 0) { - serializationSchema = new JsonRowSerializationSchema(typeInformation); + serializationSchema = new JsonCRowSerializationSchema(typeInformation, kafkaSinkTableInfo.getUpdateMode()); } else { throw new IllegalArgumentException("sinkDataType:" + FormatType.JSON.name() + " must set schemaString(JSON Schema)or TypeInformation"); } - } else if (FormatType.CSV.name().equalsIgnoreCase(kafkaSinkTableInfo.getSinkDataType())) { - if (StringUtils.isBlank(kafkaSinkTableInfo.getFieldDelimiter())) { throw new IllegalArgumentException("sinkDataType:" + FormatType.CSV.name() + " must set fieldDelimiter"); } - - final CsvRowSerializationSchema.Builder serSchemaBuilder = new CsvRowSerializationSchema.Builder(typeInformation); + final CsvCRowSerializationSchema.Builder serSchemaBuilder = new CsvCRowSerializationSchema.Builder(typeInformation); serSchemaBuilder.setFieldDelimiter(kafkaSinkTableInfo.getFieldDelimiter().toCharArray()[0]); - serializationSchema = serSchemaBuilder.build(); + serSchemaBuilder.setUpdateMode(kafkaSinkTableInfo.getUpdateMode()); + serializationSchema = serSchemaBuilder.build(); } else if (FormatType.AVRO.name().equalsIgnoreCase(kafkaSinkTableInfo.getSinkDataType())) { - if (StringUtils.isBlank(kafkaSinkTableInfo.getSchemaString())) { throw new IllegalArgumentException("sinkDataType:" + FormatType.AVRO.name() + " must set schemaString"); } - - serializationSchema = new AvroRowSerializationSchema(kafkaSinkTableInfo.getSchemaString()); - + serializationSchema = new AvroCRowSerializationSchema(kafkaSinkTableInfo.getSchemaString(), kafkaSinkTableInfo.getUpdateMode()); } if (null == serializationSchema) { 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 new file mode 100644 index 000000000..7234216a7 --- /dev/null +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/AbstractKafkaSink.java @@ -0,0 +1,148 @@ +/* + * 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.kafka; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; +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; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.runtime.types.CRow; +import org.apache.flink.table.runtime.types.CRowTypeInfo; +import org.apache.flink.table.sinks.RetractStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; +import org.apache.kafka.clients.consumer.ConsumerConfig; + +import java.util.Optional; +import java.util.Properties; +import java.util.stream.IntStream; + +/** + * Date: 2020/4/1 + * Company: www.dtstack.com + * @author maqi + */ +public abstract class AbstractKafkaSink implements RetractStreamTableSink, IStreamSinkGener { + + public static final String SINK_OPERATOR_NAME_TPL = "${topic}_${table}"; + + protected String[] fieldNames; + protected TypeInformation[] fieldTypes; + + protected String[] partitionKeys; + protected String sinkOperatorName; + protected Properties properties; + protected int parallelism; + protected String topic; + protected String tableName; + + protected TableSchema schema; + protected SinkFunction kafkaProducer; + + + protected Optional> partitioner; + + protected Properties getKafkaProperties(KafkaSinkTableInfo KafkaSinkTableInfo) { + Properties props = new Properties(); + props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, KafkaSinkTableInfo.getBootstrapServers()); + + for (String key : KafkaSinkTableInfo.getKafkaParamKeys()) { + props.setProperty(key, KafkaSinkTableInfo.getKafkaParam(key)); + } + return props; + } + + protected TypeInformation[] getTypeInformations(KafkaSinkTableInfo kafka11SinkTableInfo) { + Class[] fieldClasses = kafka11SinkTableInfo.getFieldClasses(); + TypeInformation[] types = IntStream.range(0, fieldClasses.length) + .mapToObj(i -> TypeInformation.of(fieldClasses[i])) + .toArray(TypeInformation[]::new); + return types; + } + + + protected TableSchema buildTableSchema(String[] fieldNames, TypeInformation[] fieldTypes) { + Preconditions.checkArgument(fieldNames.length == fieldTypes.length, "fieldNames length must equals fieldTypes length !"); + + TableSchema.Builder builder = TableSchema.builder(); + IntStream.range(0, fieldTypes.length) + .forEach(i -> builder.field(fieldNames[i], fieldTypes[i])); + + return builder.build(); + } + + + @Override + public void emitDataStream(DataStream> dataStream) { + DataStream mapDataStream = dataStream + .map((Tuple2 record) -> new CRow(record.f1, record.f0)) + .returns(getRowTypeInfo()) + .setParallelism(parallelism); + + mapDataStream.addSink(kafkaProducer).name(sinkOperatorName); + } + + public CRowTypeInfo getRowTypeInfo() { + return new CRowTypeInfo(new RowTypeInfo(fieldTypes, fieldNames)); + } + + protected String[] getPartitionKeys(KafkaSinkTableInfo kafkaSinkTableInfo) { + if (StringUtils.isNotBlank(kafkaSinkTableInfo.getPartitionKeys())) { + return StringUtils.split(kafkaSinkTableInfo.getPartitionKeys(), ','); + } + return null; + } + + @Override + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), new RowTypeInfo(fieldTypes, fieldNames)); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + + @Override + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + +} diff --git a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkPartition.java b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkPartition.java index e212d1f57..90dfe996b 100644 --- a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkPartition.java +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkPartition.java @@ -11,13 +11,13 @@ public class CustomerFlinkPartition extends FlinkKafkaPartitioner { public CustomerFlinkPartition() { } - + @Override public void open(int parallelInstanceId, int parallelInstances) { Preconditions.checkArgument(parallelInstanceId >= 0, "Id of this subtask cannot be negative."); Preconditions.checkArgument(parallelInstances > 0, "Number of subtasks must be larger than 0."); this.parallelInstanceId = parallelInstanceId; } - + @Override public int partition(T record, byte[] key, byte[] value, String targetTopic, int[] partitions) { Preconditions.checkArgument(partitions != null && partitions.length > 0, "Partitions of the target topic is empty."); if(key == null){ @@ -25,11 +25,11 @@ public int partition(T record, byte[] key, byte[] value, String targetTopic, int } return partitions[Math.abs(new String(key).hashCode()) % partitions.length]; } - + @Override public boolean equals(Object o) { return this == o || o instanceof CustomerFlinkPartition; } - + @Override public int hashCode() { return CustomerFlinkPartition.class.hashCode(); } diff --git a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKeyedSerializationSchema.java b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKeyedSerializationSchema.java index 498766564..cde8d1b1d 100644 --- a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKeyedSerializationSchema.java +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKeyedSerializationSchema.java @@ -2,18 +2,20 @@ import com.dtstack.flink.sql.format.SerializationMetricWrapper; +import com.dtstack.flink.sql.sink.kafka.serialization.JsonCRowSerializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; +import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.concurrent.atomic.AtomicLong; -public class CustomerKeyedSerializationSchema implements KeyedSerializationSchema { +public class CustomerKeyedSerializationSchema implements KeyedSerializationSchema { private static final Logger LOG = LoggerFactory.getLogger(CustomerKeyedSerializationSchema.class); @@ -30,38 +32,41 @@ public CustomerKeyedSerializationSchema(SerializationMetricWrapper serialization this.mapper = new ObjectMapper(); } - public byte[] serializeKey(Row element) { - if(partitionKeys == null || partitionKeys.length <=0){ + @Override + public byte[] serializeKey(CRow element) { + if (partitionKeys == null || partitionKeys.length <= 0) { return null; - } - SerializationSchema serializationSchema = serializationMetricWrapper.getSerializationSchema(); - if(serializationSchema instanceof JsonRowSerializationSchema){ - return serializeJsonKey((JsonRowSerializationSchema) serializationSchema, element); + } + SerializationSchema serializationSchema = serializationMetricWrapper.getSerializationSchema(); + if (serializationSchema instanceof JsonCRowSerializationSchema) { + return serializeJsonKey((JsonCRowSerializationSchema) serializationSchema, element); } return null; } - public byte[] serializeValue(Row element) { + @Override + public byte[] serializeValue(CRow element) { return this.serializationMetricWrapper.serialize(element); } - public String getTargetTopic(Row element) { + @Override + public String getTargetTopic(CRow element) { return null; } - private byte[] serializeJsonKey(JsonRowSerializationSchema jsonRowSerializationSchema, Row element) { + private byte[] serializeJsonKey(JsonCRowSerializationSchema jsonCRowSerializationSchema, CRow element) { try { - byte[] data = jsonRowSerializationSchema.serialize(element); + byte[] data = jsonCRowSerializationSchema.serialize(element); ObjectNode objectNode = mapper.readValue(data, ObjectNode.class); StringBuilder sb = new StringBuilder(); - for(String key : partitionKeys){ - if(objectNode.has(key)){ + for (String key : partitionKeys) { + if (objectNode.has(key)) { sb.append(objectNode.get(key.trim())); } } return sb.toString().getBytes(); - } catch (Exception e){ - if(COUNTER.getAndIncrement() % 1000 == 0){ + } catch (Exception e) { + if (COUNTER.getAndIncrement() % 1000 == 0) { LOG.error("serializeJsonKey error", e); } } diff --git a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/AvroCRowSerializationSchema.java b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/AvroCRowSerializationSchema.java new file mode 100644 index 000000000..692e208b5 --- /dev/null +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/AvroCRowSerializationSchema.java @@ -0,0 +1,366 @@ +/* + * 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.kafka.serialization; + +import com.dtstack.flink.sql.enums.EUpdateMode; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.SchemaParseException; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.Encoder; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.specific.SpecificData; +import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.avro.specific.SpecificRecord; +import org.apache.avro.util.Utf8; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.table.runtime.types.CRow; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.TimeZone; +import java.util.stream.Collectors; + +/** + * Serialization schema that serializes CROW into Avro bytes. + * + *

Serializes objects that are represented in (nested) Flink rows. It support types that + * are compatible with Flink's Table & SQL API. + ** + * @author maqi + */ +public class AvroCRowSerializationSchema implements SerializationSchema { + + /** + * Used for time conversions from SQL types. + */ + private static final TimeZone LOCAL_TZ = TimeZone.getDefault(); + + /** + * Avro record class for serialization. Might be null if record class is not available. + */ + private Class recordClazz; + + /** + * Schema string for deserialization. + */ + private String schemaString; + + /** + * Avro serialization schema. + */ + private transient Schema schema; + + /** + * Writer to serialize Avro record into a byte array. + */ + private transient DatumWriter datumWriter; + + /** + * Output stream to serialize records into byte array. + */ + private transient ByteArrayOutputStream arrayOutputStream; + + /** + * Low-level class for serialization of Avro values. + */ + private transient Encoder encoder; + + private String updateMode; + + private String retractKey = "retract"; + + /** + * Creates an Avro serialization schema for the given specific record class. + * + * @param recordClazz Avro record class used to serialize Flink's row to Avro's record + */ + public AvroCRowSerializationSchema(Class recordClazz, String updateMode) { + Preconditions.checkNotNull(recordClazz, "Avro record class must not be null."); + this.recordClazz = recordClazz; + this.schema = SpecificData.get().getSchema(recordClazz); + this.schemaString = schema.toString(); + this.datumWriter = new SpecificDatumWriter<>(schema); + this.arrayOutputStream = new ByteArrayOutputStream(); + this.encoder = EncoderFactory.get().binaryEncoder(arrayOutputStream, null); + this.updateMode = updateMode; + } + + /** + * Creates an Avro serialization schema for the given Avro schema string. + * + * @param avroSchemaString Avro schema string used to serialize Flink's row to Avro's record + */ + public AvroCRowSerializationSchema(String avroSchemaString,String updateMode) { + Preconditions.checkNotNull(avroSchemaString, "Avro schema must not be null."); + this.recordClazz = null; + this.schemaString = avroSchemaString; + try { + this.schema = new Schema.Parser().parse(avroSchemaString); + } catch (SchemaParseException e) { + throw new IllegalArgumentException("Could not parse Avro schema string.", e); + } + this.datumWriter = new GenericDatumWriter<>(schema); + this.arrayOutputStream = new ByteArrayOutputStream(); + this.encoder = EncoderFactory.get().binaryEncoder(arrayOutputStream, null); + this.updateMode = updateMode; + } + + @Override + public byte[] serialize(CRow crow) { + try { + Row row = crow.row(); + boolean change = crow.change(); + + // convert to record + final GenericRecord record = convertRowToAvroRecord(schema, row); + + dealRetractField(change, record); + + arrayOutputStream.reset(); + datumWriter.write(record, encoder); + encoder.flush(); + return arrayOutputStream.toByteArray(); + } catch (Exception e) { + throw new RuntimeException("Failed to serialize row.", e); + } + } + + protected void dealRetractField(boolean change, GenericRecord record) { + schema.getFields() + .stream() + .filter(field -> StringUtils.equalsIgnoreCase(field.name(), retractKey)) + .findFirst() + .ifPresent(field -> { + if (StringUtils.equalsIgnoreCase(updateMode, EUpdateMode.UPSERT.name())) { + record.put(retractKey, convertFlinkType(field.schema(), change)); + } + }); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final AvroCRowSerializationSchema that = (AvroCRowSerializationSchema) o; + return Objects.equals(recordClazz, that.recordClazz) && Objects.equals(schemaString, that.schemaString); + } + + @Override + public int hashCode() { + return Objects.hash(recordClazz, schemaString); + } + + // -------------------------------------------------------------------------------------------- + + private GenericRecord convertRowToAvroRecord(Schema schema, Row row) { + + final List fields = schema.getFields() + .stream() + .filter(field -> !StringUtils.equalsIgnoreCase(field.name(), retractKey)) + .collect(Collectors.toList()); + + final int length = fields.size(); + final GenericRecord record = new GenericData.Record(schema); + for (int i = 0; i < length; i++) { + final Schema.Field field = fields.get(i); + record.put(i, convertFlinkType(field.schema(), row.getField(i))); + } + return record; + } + + private Object convertFlinkType(Schema schema, Object object) { + if (object == null) { + return null; + } + switch (schema.getType()) { + case RECORD: + if (object instanceof Row) { + return convertRowToAvroRecord(schema, (Row) object); + } + throw new IllegalStateException("Row expected but was: " + object.getClass()); + case ENUM: + return new GenericData.EnumSymbol(schema, object.toString()); + case ARRAY: + final Schema elementSchema = schema.getElementType(); + final Object[] array = (Object[]) object; + final GenericData.Array convertedArray = new GenericData.Array<>(array.length, schema); + for (Object element : array) { + convertedArray.add(convertFlinkType(elementSchema, element)); + } + return convertedArray; + case MAP: + final Map map = (Map) object; + final Map convertedMap = new HashMap<>(); + for (Map.Entry entry : map.entrySet()) { + convertedMap.put( + new Utf8(entry.getKey().toString()), + convertFlinkType(schema.getValueType(), entry.getValue())); + } + return convertedMap; + case UNION: + final List types = schema.getTypes(); + final int size = types.size(); + final Schema actualSchema; + if (size == 2 && types.get(0).getType() == Schema.Type.NULL) { + actualSchema = types.get(1); + } else if (size == 2 && types.get(1).getType() == Schema.Type.NULL) { + actualSchema = types.get(0); + } else if (size == 1) { + actualSchema = types.get(0); + } else { + // generic type + return object; + } + return convertFlinkType(actualSchema, object); + case FIXED: + // check for logical type + if (object instanceof BigDecimal) { + return new GenericData.Fixed( + schema, + convertFromDecimal(schema, (BigDecimal) object)); + } + return new GenericData.Fixed(schema, (byte[]) object); + case STRING: + return new Utf8(object.toString()); + case BYTES: + // check for logical type + if (object instanceof BigDecimal) { + return ByteBuffer.wrap(convertFromDecimal(schema, (BigDecimal) object)); + } + return ByteBuffer.wrap((byte[]) object); + case INT: + // check for logical types + if (object instanceof Date) { + return convertFromDate(schema, (Date) object); + } else if (object instanceof Time) { + return convertFromTime(schema, (Time) object); + } + return object; + case LONG: + // check for logical type + if (object instanceof Timestamp) { + return convertFromTimestamp(schema, (Timestamp) object); + } + return object; + case FLOAT: + case DOUBLE: + case BOOLEAN: + return object; + } + throw new RuntimeException("Unsupported Avro type:" + schema); + } + + private byte[] convertFromDecimal(Schema schema, BigDecimal decimal) { + final LogicalType logicalType = schema.getLogicalType(); + if (logicalType instanceof LogicalTypes.Decimal) { + final LogicalTypes.Decimal decimalType = (LogicalTypes.Decimal) logicalType; + // rescale to target type + final BigDecimal rescaled = decimal.setScale(decimalType.getScale(), BigDecimal.ROUND_UNNECESSARY); + // byte array must contain the two's-complement representation of the + // unscaled integer value in big-endian byte order + return decimal.unscaledValue().toByteArray(); + } else { + throw new RuntimeException("Unsupported decimal type."); + } + } + + private int convertFromDate(Schema schema, Date date) { + final LogicalType logicalType = schema.getLogicalType(); + if (logicalType == LogicalTypes.date()) { + // adopted from Apache Calcite + final long time = date.getTime(); + final long converted = time + (long) LOCAL_TZ.getOffset(time); + return (int) (converted / 86400000L); + } else { + throw new RuntimeException("Unsupported date type."); + } + } + + private int convertFromTime(Schema schema, Time date) { + final LogicalType logicalType = schema.getLogicalType(); + if (logicalType == LogicalTypes.timeMillis()) { + // adopted from Apache Calcite + final long time = date.getTime(); + final long converted = time + (long) LOCAL_TZ.getOffset(time); + return (int) (converted % 86400000L); + } else { + throw new RuntimeException("Unsupported time type."); + } + } + + private long convertFromTimestamp(Schema schema, Timestamp date) { + final LogicalType logicalType = schema.getLogicalType(); + if (logicalType == LogicalTypes.timestampMillis()) { + // adopted from Apache Calcite + final long time = date.getTime(); + return time + (long) LOCAL_TZ.getOffset(time); + } else { + throw new RuntimeException("Unsupported timestamp type."); + } + } + + private void writeObject(ObjectOutputStream outputStream) throws IOException { + outputStream.writeObject(recordClazz); + outputStream.writeObject(schemaString); // support for null + outputStream.writeObject(retractKey); + outputStream.writeObject(updateMode); + } + + @SuppressWarnings("unchecked") + private void readObject(ObjectInputStream inputStream) throws ClassNotFoundException, IOException { + recordClazz = (Class) inputStream.readObject(); + schemaString = (String) inputStream.readObject(); + if (recordClazz != null) { + schema = SpecificData.get().getSchema(recordClazz); + } else { + schema = new Schema.Parser().parse(schemaString); + } + retractKey = (String) inputStream.readObject(); + updateMode = (String) inputStream.readObject(); + + datumWriter = new SpecificDatumWriter<>(schema); + arrayOutputStream = new ByteArrayOutputStream(); + encoder = EncoderFactory.get().binaryEncoder(arrayOutputStream, null); + } +} diff --git a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/CsvCRowSerializationSchema.java b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/CsvCRowSerializationSchema.java new file mode 100644 index 000000000..4e57b6f2a --- /dev/null +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/CsvCRowSerializationSchema.java @@ -0,0 +1,374 @@ +/* + * 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.kafka.serialization; + +import com.dtstack.flink.sql.enums.EUpdateMode; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.csv.CsvRowDeserializationSchema; +import org.apache.flink.formats.csv.CsvRowSchemaConverter; +import org.apache.flink.formats.csv.CsvRowSerializationSchema; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectWriter; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ContainerNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; +import org.apache.flink.table.runtime.types.CRow; +import org.apache.flink.table.runtime.types.CRowTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.Arrays; +import java.util.Iterator; +import java.util.Objects; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +/** + * Serialization schema that serializes an object of Flink types into a CSV bytes. + * + *

Serializes the input row into a {@link ObjectNode} and + * converts it into byte[]. + * + *

Result byte[] messages can be deserialized using {@link CsvRowDeserializationSchema}. + */ +@PublicEvolving +public final class CsvCRowSerializationSchema implements SerializationSchema { + + private static final long serialVersionUID = 2098447220136965L; + + /** Type information describing the input CSV data. */ + private final RowTypeInfo typeInfo; + + /** Runtime instance that performs the actual work. */ + private final RuntimeConverter runtimeConverter; + + /** CsvMapper used to write {@link JsonNode} into bytes. */ + private final CsvMapper csvMapper; + + /** Schema describing the input CSV data. */ + private CsvSchema csvSchema; + + /** Object writer used to write rows. It is configured by {@link CsvSchema}. */ + private ObjectWriter objectWriter; + + /** Reusable object node. */ + private transient ObjectNode root; + + private String updateMode; + + private String retractKey = "retract"; + + private CsvCRowSerializationSchema( + RowTypeInfo typeInfo, + CsvSchema csvSchema, + String updateMode) { + this.typeInfo = typeInfo; + this.runtimeConverter = createRowRuntimeConverter(typeInfo, true); + this.csvMapper = new CsvMapper(); + this.csvSchema = csvSchema; + this.updateMode = updateMode; + this.objectWriter = csvMapper.writer(csvSchema); + } + + /** + * A builder for creating a {@link CsvRowSerializationSchema}. + */ + @PublicEvolving + public static class Builder { + + private final RowTypeInfo typeInfo; + private CsvSchema csvSchema; + private String updateMode; + + /** + * Creates a {@link CsvRowSerializationSchema} expecting the given {@link TypeInformation}. + * + * @param typeInfo type information used to create schema. + */ + public Builder(TypeInformation typeInfo) { + Preconditions.checkNotNull(typeInfo, "Type information must not be null."); + + if (!(typeInfo instanceof CRowTypeInfo)) { + throw new IllegalArgumentException("Row type information expected."); + } + RowTypeInfo rowTypeInfo = ((CRowTypeInfo) typeInfo).rowType(); + this.typeInfo = rowTypeInfo; + this.csvSchema = CsvRowSchemaConverter.convert(rowTypeInfo); + } + + public Builder setFieldDelimiter(char c) { + this.csvSchema = this.csvSchema.rebuild().setColumnSeparator(c).build(); + return this; + } + + public Builder setLineDelimiter(String delimiter) { + Preconditions.checkNotNull(delimiter, "Delimiter must not be null."); + if (!delimiter.equals("\n") && !delimiter.equals("\r") && !delimiter.equals("\r\n")) { + throw new IllegalArgumentException( + "Unsupported new line delimiter. Only \\n, \\r, or \\r\\n are supported."); + } + this.csvSchema = this.csvSchema.rebuild().setLineSeparator(delimiter).build(); + return this; + } + + public Builder setArrayElementDelimiter(String delimiter) { + Preconditions.checkNotNull(delimiter, "Delimiter must not be null."); + this.csvSchema = this.csvSchema.rebuild().setArrayElementSeparator(delimiter).build(); + return this; + } + + public Builder setQuoteCharacter(char c) { + this.csvSchema = this.csvSchema.rebuild().setQuoteChar(c).build(); + return this; + } + + public Builder setEscapeCharacter(char c) { + this.csvSchema = this.csvSchema.rebuild().setEscapeChar(c).build(); + return this; + } + + public Builder setNullLiteral(String s) { + this.csvSchema = this.csvSchema.rebuild().setNullValue(s).build(); + return this; + } + + public Builder setUpdateMode(String updateMode) { + this.updateMode = updateMode; + return this; + } + + public CsvCRowSerializationSchema build() { + return new CsvCRowSerializationSchema( + typeInfo, + csvSchema, + updateMode); + } + } + + @Override + public byte[] serialize(CRow crow) { + Row row = crow.row(); + boolean change = crow.change(); + if (root == null) { + root = csvMapper.createObjectNode(); + } + try { + runtimeConverter.convert(csvMapper, root, row); + if (StringUtils.equalsIgnoreCase(updateMode, EUpdateMode.UPSERT.name())) { + fillRetractField(row, change); + } + + return objectWriter.writeValueAsBytes(root); + } catch (Throwable t) { + throw new RuntimeException("Could not serialize row '" + row + "'.", t); + } + } + + protected void fillRetractField(Row row, boolean change) { + root.put(retractKey, change); + CsvSchema.Builder newBuilder = new CsvSchema.Builder(csvSchema); + + CsvSchema.Column retractColumn = new CsvSchema.Column(row.getArity(), retractKey, CsvSchema.ColumnType.BOOLEAN); + newBuilder.addColumn(retractColumn); + csvSchema = newBuilder.build(); + + this.objectWriter = csvMapper.writer(csvSchema); + + } + + @Override + public boolean equals(Object o) { + if (o == null || o.getClass() != this.getClass()) { + return false; + } + if (this == o) { + return true; + } + final CsvCRowSerializationSchema that = (CsvCRowSerializationSchema) o; + final CsvSchema otherSchema = that.csvSchema; + + return typeInfo.equals(that.typeInfo) && + csvSchema.getColumnSeparator() == otherSchema.getColumnSeparator() && + Arrays.equals(csvSchema.getLineSeparator(), otherSchema.getLineSeparator()) && + csvSchema.getArrayElementSeparator().equals(otherSchema.getArrayElementSeparator()) && + csvSchema.getQuoteChar() == otherSchema.getQuoteChar() && + csvSchema.getEscapeChar() == otherSchema.getEscapeChar() && + Arrays.equals(csvSchema.getNullValue(), otherSchema.getNullValue()); + } + + @Override + public int hashCode() { + return Objects.hash( + typeInfo, + csvSchema.getColumnSeparator(), + csvSchema.getLineSeparator(), + csvSchema.getArrayElementSeparator(), + csvSchema.getQuoteChar(), + csvSchema.getEscapeChar(), + csvSchema.getNullValue()); + } + + // -------------------------------------------------------------------------------------------- + + private interface RuntimeConverter extends Serializable { + JsonNode convert(CsvMapper csvMapper, ContainerNode container, Object obj); + } + + private static RuntimeConverter createRowRuntimeConverter(RowTypeInfo rowTypeInfo, boolean isTopLevel) { + final TypeInformation[] fieldTypes = rowTypeInfo.getFieldTypes(); + final String[] fieldNames = rowTypeInfo.getFieldNames(); + + final RuntimeConverter[] fieldConverters = createFieldRuntimeConverters(fieldTypes); + + return assembleRowRuntimeConverter(isTopLevel, fieldNames, fieldConverters); + } + + private static RuntimeConverter[] createFieldRuntimeConverters(TypeInformation[] fieldTypes) { + final RuntimeConverter[] fieldConverters = new RuntimeConverter[fieldTypes.length]; + for (int i = 0; i < fieldTypes.length; i++) { + fieldConverters[i] = createNullableRuntimeConverter(fieldTypes[i]); + } + return fieldConverters; + } + + private static RuntimeConverter assembleRowRuntimeConverter( + boolean isTopLevel, + String[] fieldNames, + RuntimeConverter[] fieldConverters) { + final int rowArity = fieldNames.length; + // top level reuses the object node container + if (isTopLevel) { + return (csvMapper, container, obj) -> { + final Row row = (Row) obj; + + validateArity(rowArity, row.getArity()); + + final ObjectNode objectNode = (ObjectNode) container; + for (int i = 0; i < rowArity; i++) { + objectNode.set( + fieldNames[i], + fieldConverters[i].convert(csvMapper, container, row.getField(i))); + } + return objectNode; + }; + } else { + return (csvMapper, container, obj) -> { + final Row row = (Row) obj; + + validateArity(rowArity, row.getArity()); + + final ArrayNode arrayNode = csvMapper.createArrayNode(); + for (int i = 0; i < rowArity; i++) { + arrayNode.add(fieldConverters[i].convert(csvMapper, arrayNode, row.getField(i))); + } + return arrayNode; + }; + } + } + + private static RuntimeConverter createNullableRuntimeConverter(TypeInformation info) { + final RuntimeConverter valueConverter = createRuntimeConverter(info); + return (csvMapper, container, obj) -> { + if (obj == null) { + return container.nullNode(); + } + return valueConverter.convert(csvMapper, container, obj); + }; + } + + private static RuntimeConverter createRuntimeConverter(TypeInformation info) { + if (info.equals(Types.VOID)) { + return (csvMapper, container, obj) -> container.nullNode(); + } else if (info.equals(Types.STRING)) { + return (csvMapper, container, obj) -> container.textNode((String) obj); + } else if (info.equals(Types.BOOLEAN)) { + return (csvMapper, container, obj) -> container.booleanNode((Boolean) obj); + } else if (info.equals(Types.BYTE)) { + return (csvMapper, container, obj) -> container.numberNode((Byte) obj); + } else if (info.equals(Types.SHORT)) { + return (csvMapper, container, obj) -> container.numberNode((Short) obj); + } else if (info.equals(Types.INT)) { + return (csvMapper, container, obj) -> container.numberNode((Integer) obj); + } else if (info.equals(Types.LONG)) { + return (csvMapper, container, obj) -> container.numberNode((Long) obj); + } else if (info.equals(Types.FLOAT)) { + return (csvMapper, container, obj) -> container.numberNode((Float) obj); + } else if (info.equals(Types.DOUBLE)) { + return (csvMapper, container, obj) -> container.numberNode((Double) obj); + } else if (info.equals(Types.BIG_DEC)) { + return (csvMapper, container, obj) -> container.numberNode((BigDecimal) obj); + } else if (info.equals(Types.BIG_INT)) { + return (csvMapper, container, obj) -> container.numberNode((BigInteger) obj); + } else if (info.equals(Types.SQL_DATE)) { + return (csvMapper, container, obj) -> container.textNode(obj.toString()); + } else if (info.equals(Types.SQL_TIME)) { + return (csvMapper, container, obj) -> container.textNode(obj.toString()); + } else if (info.equals(Types.SQL_TIMESTAMP)) { + return (csvMapper, container, obj) -> container.textNode(obj.toString()); + } else if (info instanceof RowTypeInfo){ + return createRowRuntimeConverter((RowTypeInfo) info, false); + } else if (info instanceof BasicArrayTypeInfo) { + return createObjectArrayRuntimeConverter(((BasicArrayTypeInfo) info).getComponentInfo()); + } else if (info instanceof ObjectArrayTypeInfo) { + return createObjectArrayRuntimeConverter(((ObjectArrayTypeInfo) info).getComponentInfo()); + } else if (info instanceof PrimitiveArrayTypeInfo && + ((PrimitiveArrayTypeInfo) info).getComponentType() == Types.BYTE) { + return createByteArrayRuntimeConverter(); + } + else { + throw new RuntimeException("Unsupported type information '" + info + "'."); + } + } + + private static RuntimeConverter createObjectArrayRuntimeConverter(TypeInformation elementType) { + final RuntimeConverter elementConverter = createNullableRuntimeConverter(elementType); + return (csvMapper, container, obj) -> { + final Object[] array = (Object[]) obj; + final ArrayNode arrayNode = csvMapper.createArrayNode(); + for (Object element : array) { + arrayNode.add(elementConverter.convert(csvMapper, arrayNode, element)); + } + return arrayNode; + }; + } + + private static RuntimeConverter createByteArrayRuntimeConverter() { + return (csvMapper, container, obj) -> container.binaryNode((byte[]) obj); + } + + private static void validateArity(int expected, int actual) { + if (expected != actual) { + throw new RuntimeException("Row length mismatch. " + expected + + " fields expected but was " + actual + "."); + } + } +} diff --git a/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/JsonCRowSerializationSchema.java b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/JsonCRowSerializationSchema.java new file mode 100644 index 000000000..bfe801d52 --- /dev/null +++ b/kafka-base/kafka-base-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/serialization/JsonCRowSerializationSchema.java @@ -0,0 +1,234 @@ +/* + * 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.kafka.serialization; + +import com.dtstack.flink.sql.enums.EUpdateMode; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSchemaConverter; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ContainerNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.flink.table.runtime.types.CRow; +import org.apache.flink.table.runtime.types.CRowTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Time; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; +import java.util.Objects; + +/** + * + * Serialization schema that serializes an object of Flink types into a JSON bytes. + * + *

Serializes the input Flink object into a JSON string and + * converts it into byte[]. + * + */ +public class JsonCRowSerializationSchema implements SerializationSchema { + + private static final long serialVersionUID = -2885556750743978636L; + + /** Type information describing the input type. */ + private final TypeInformation typeInfo; + + /** Object mapper that is used to create output JSON objects. */ + private final ObjectMapper mapper = new ObjectMapper(); + + /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone, without milliseconds). */ + private SimpleDateFormat timeFormat = new SimpleDateFormat("HH:mm:ss'Z'"); + + /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone). */ + private SimpleDateFormat timeFormatWithMillis = new SimpleDateFormat("HH:mm:ss.SSS'Z'"); + + /** Formatter for RFC 3339-compliant string representation of a timestamp value (with UTC timezone). */ + private SimpleDateFormat timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + + /** Reusable object node. */ + private transient ObjectNode node; + + private String updateMode; + + private final String retractKey = "retract"; + + public JsonCRowSerializationSchema(String jsonSchema, String updateMode) { + this(JsonRowSchemaConverter.convert(jsonSchema), updateMode); + } + + /** + * Creates a JSON serialization schema for the given type information. + * + * @param typeInfo The field names of {@link Row} are used to map to JSON properties. + */ + public JsonCRowSerializationSchema(TypeInformation typeInfo, String updateMode) { + Preconditions.checkNotNull(typeInfo, "Type information"); + this.typeInfo = typeInfo; + this.updateMode = updateMode; + } + + + @Override + public byte[] serialize(CRow crow) { + Row row = crow.row(); + boolean change = crow.change(); + if (node == null) { + node = mapper.createObjectNode(); + } + + RowTypeInfo rowTypeInfo = ((CRowTypeInfo) typeInfo).rowType(); + try { + convertRow(node, rowTypeInfo, row); + if (StringUtils.equalsIgnoreCase(updateMode, EUpdateMode.UPSERT.name())) { + node.put(retractKey, change); + } + return mapper.writeValueAsBytes(node); + } catch (Throwable t) { + throw new RuntimeException("Could not serialize row '" + row + "'. " + + "Make sure that the schema matches the input.", t); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final JsonCRowSerializationSchema that = (JsonCRowSerializationSchema) o; + return Objects.equals(typeInfo, that.typeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(typeInfo); + } + + // -------------------------------------------------------------------------------------------- + + private ObjectNode convertRow(ObjectNode reuse, RowTypeInfo info, Row row) { + if (reuse == null) { + reuse = mapper.createObjectNode(); + } + final String[] fieldNames = info.getFieldNames(); + + final TypeInformation[] fieldTypes = info.getFieldTypes(); + + // validate the row + if (row.getArity() != fieldNames.length) { + throw new IllegalStateException(String.format( + "Number of elements in the row '%s' is different from number of field names: %d", row, fieldNames.length)); + } + + for (int i = 0; i < fieldNames.length; i++) { + final String name = fieldNames[i]; + + final JsonNode fieldConverted = convert(reuse, reuse.get(name), fieldTypes[i], row.getField(i)); + reuse.set(name, fieldConverted); + } + + return reuse; + } + + private JsonNode convert(ContainerNode container, JsonNode reuse, TypeInformation info, Object object) { + if (info == Types.VOID || object == null) { + return container.nullNode(); + } else if (info == Types.BOOLEAN) { + return container.booleanNode((Boolean) object); + } else if (info == Types.STRING) { + return container.textNode((String) object); + } else if (info == Types.BIG_DEC) { + // convert decimal if necessary + if (object instanceof BigDecimal) { + return container.numberNode((BigDecimal) object); + } + return container.numberNode(BigDecimal.valueOf(((Number) object).doubleValue())); + } else if (info == Types.BIG_INT) { + // convert integer if necessary + if (object instanceof BigInteger) { + return container.numberNode((BigInteger) object); + } + return container.numberNode(BigInteger.valueOf(((Number) object).longValue())); + } else if (info == Types.SQL_DATE) { + return container.textNode(object.toString()); + } else if (info == Types.SQL_TIME) { + final Time time = (Time) object; + // strip milliseconds if possible + if (time.getTime() % 1000 > 0) { + return container.textNode(timeFormatWithMillis.format(time)); + } + return container.textNode(timeFormat.format(time)); + } else if (info == Types.SQL_TIMESTAMP) { + return container.textNode(timestampFormat.format((Timestamp) object)); + } else if (info instanceof RowTypeInfo) { + if (reuse != null && reuse instanceof ObjectNode) { + return convertRow((ObjectNode) reuse, (RowTypeInfo) info, (Row) object); + } else { + return convertRow(null, (RowTypeInfo) info, (Row) object); + } + } else if (info instanceof ObjectArrayTypeInfo) { + if (reuse != null && reuse instanceof ArrayNode) { + return convertObjectArray((ArrayNode) reuse, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } else { + return convertObjectArray(null, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } + } else if (info instanceof BasicArrayTypeInfo) { + if (reuse != null && reuse instanceof ArrayNode) { + return convertObjectArray((ArrayNode) reuse, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } else { + return convertObjectArray(null, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } + } else if (info instanceof PrimitiveArrayTypeInfo && ((PrimitiveArrayTypeInfo) info).getComponentType() == Types.BYTE) { + return container.binaryNode((byte[]) object); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return mapper.valueToTree(object); + } catch (IllegalArgumentException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for object: " + object, e); + } + } + } + + private ArrayNode convertObjectArray(ArrayNode reuse, TypeInformation info, Object[] array) { + if (reuse == null) { + reuse = mapper.createArrayNode(); + } else { + reuse.removeAll(); + } + + for (Object object : array) { + reuse.add(convert(reuse, null, info, object)); + } + return reuse; + } +} 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 8ba21b953..4ad8947a8 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 @@ -18,9 +18,10 @@ package com.dtstack.flink.sql.sink.kafka.table; +import com.dtstack.flink.sql.enums.EUpdateMode; import com.dtstack.flink.sql.format.FormatType; -import com.dtstack.flink.sql.table.AbsTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; @@ -32,12 +33,13 @@ * @author DocLi * @modifyer maqi */ -public class KafkaSinkParser extends AbsTableParser { +public class KafkaSinkParser extends AbstractTableParser { @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { KafkaSinkTableInfo kafkaSinkTableInfo = new KafkaSinkTableInfo(); kafkaSinkTableInfo.setName(tableName); kafkaSinkTableInfo.setType(MathUtil.getString(props.get(KafkaSinkTableInfo.TYPE_KEY.toLowerCase()))); + parseFieldsInfo(fieldsInfo, kafkaSinkTableInfo); if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { @@ -46,11 +48,14 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map kafkaParam = new HashMap(); @@ -59,6 +70,8 @@ public class KafkaSinkTableInfo extends TargetTableInfo { private String partitionKeys; + private String updateMode; + public void addKafkaParam(String key, String value) { kafkaParam.put(key, value); } @@ -71,7 +84,6 @@ public Set getKafkaParamKeys() { return kafkaParam.keySet(); } - public String getBootstrapServers() { return bootstrapServers; } @@ -104,14 +116,40 @@ public void setFieldDelimiter(String fieldDelimiter) { this.fieldDelimiter = fieldDelimiter; } + public String getUpdateMode() { + return updateMode; + } + + public void setUpdateMode(String updateMode) { + this.updateMode = updateMode; + } + @Override public boolean check() { Preconditions.checkNotNull(getType(), "kafka of type is required"); Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); Preconditions.checkNotNull(topic, "kafka of topic is required"); + + if (StringUtils.equalsIgnoreCase(getSinkDataType(), FormatType.AVRO.name())) { + avroParamCheck(); + } + return false; } + public void avroParamCheck() { + Preconditions.checkNotNull(schemaString, "avro type schemaInfo is required"); + if (StringUtils.equalsIgnoreCase(updateMode, EUpdateMode.UPSERT.name())) { + Schema schema = new Schema.Parser().parse(schemaString); + schema.getFields() + .stream() + .filter(field -> StringUtils.equalsIgnoreCase(field.name(), RETRACT_FIELD_KEY)) + .findFirst() + .orElseThrow(() -> + new NullPointerException(String.valueOf("arvo upsert mode the retract attribute must be contained in schemaInfo field "))); + } + } + public String getEnableKeyPartition() { return enableKeyPartition; } diff --git a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/AbstractKafkaSource.java b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/AbstractKafkaSource.java new file mode 100644 index 000000000..852a381e2 --- /dev/null +++ b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/AbstractKafkaSource.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flink.sql.source.kafka; + +import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.kafka.enums.EKafkaOffset; +import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; +import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.table.api.Table; +import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.ConsumerConfig; + +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * Date: 2020/3/20 + * Company: www.dtstack.com + * @author maqi + */ +public abstract class AbstractKafkaSource implements IStreamSourceGener { + + private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; + + protected Properties getKafkaProperties(KafkaSourceTableInfo kafkaSourceTableInfo) { + Properties props = new Properties(); + props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaSourceTableInfo.getBootstrapServers()); + + if (DtStringUtil.isJson(kafkaSourceTableInfo.getOffsetReset())) { + props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, EKafkaOffset.NONE.name().toLowerCase()); + } else { + props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, kafkaSourceTableInfo.getOffsetReset()); + } + + if (StringUtils.isNotBlank(kafkaSourceTableInfo.getGroupId())) { + props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, kafkaSourceTableInfo.getGroupId()); + } + + for (String key : kafkaSourceTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafkaSourceTableInfo.getKafkaParam(key)); + } + return props; + } + + protected String generateOperatorName(String tabName, String topicName) { + return SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", tabName); + } + + protected TypeInformation getRowTypeInformation(KafkaSourceTableInfo kafkaSourceTableInfo) { + Class[] fieldClasses = kafkaSourceTableInfo.getFieldClasses(); + TypeInformation[] types = IntStream.range(0, fieldClasses.length) + .mapToObj(i -> TypeInformation.of(fieldClasses[i])) + .toArray(TypeInformation[]::new); + + return new RowTypeInfo(types, kafkaSourceTableInfo.getFields()); + } + + protected void setStartPosition(String offset, String topicName, FlinkKafkaConsumerBase kafkaSrc) { + if (StringUtils.equalsIgnoreCase(offset, EKafkaOffset.EARLIEST.name())) { + kafkaSrc.setStartFromEarliest(); + } else if (DtStringUtil.isJson(offset)) { + Map specificStartupOffsets = buildOffsetMap(offset, topicName); + kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); + } else { + kafkaSrc.setStartFromLatest(); + } + } + + /** + * kafka offset,eg.. {"0":12312,"1":12321,"2":12312} + * @param offsetJson + * @param topicName + * @return + */ + protected Map buildOffsetMap(String offsetJson, String topicName) { + try { + Properties properties = PluginUtil.jsonStrToObject(offsetJson, Properties.class); + Map offsetMap = PluginUtil.objectToMap(properties); + Map specificStartupOffsets = offsetMap + .entrySet() + .stream() + .collect(Collectors.toMap( + (Map.Entry entry) -> new KafkaTopicPartition(topicName, Integer.valueOf(entry.getKey())), + (Map.Entry entry) -> Long.valueOf(entry.getValue().toString())) + ); + + return specificStartupOffsets; + } catch (Exception e) { + throw new RuntimeException("not support offsetReset type:" + offsetJson); + } + } + +} diff --git a/cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.java b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/enums/EKafkaOffset.java similarity index 56% rename from cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.java rename to kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/enums/EKafkaOffset.java index 33a0233ac..476ccc8b7 100644 --- a/cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.java +++ b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/enums/EKafkaOffset.java @@ -16,43 +16,16 @@ * limitations under the License. */ - - -package com.dtstack.flinkx; - -import junit.framework.Test; -import junit.framework.TestCase; -import junit.framework.TestSuite; +package com.dtstack.flink.sql.source.kafka.enums; /** - * Unit test for simple App. + * Date: 2020/3/20 + * Company: www.dtstack.com + * @author maqi */ -public class AppTest - extends TestCase -{ - /** - * Create the test case - * - * @param testName name of the test case - */ - public AppTest( String testName ) - { - super( testName ); - } - - /** - * @return the suite of tests being tested - */ - public static Test suite() - { - return new TestSuite( AppTest.class ); - } +public enum EKafkaOffset { - /** - * Rigourous Test :-) - */ - public void testApp() - { - assertTrue( true ); - } + LATEST, + EARLIEST, + NONE } diff --git a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 005bfb4c2..bef86f10d 100644 --- a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -19,11 +19,14 @@ package com.dtstack.flink.sql.source.kafka.table; -import com.dtstack.flink.sql.table.AbsSourceParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.format.FormatType; +import com.dtstack.flink.sql.source.kafka.enums.EKafkaOffset; +import com.dtstack.flink.sql.table.AbstractSourceParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; +import java.util.stream.Collectors; /** * Reason: @@ -32,33 +35,38 @@ * * @author sishu.yss */ -public class KafkaSourceParser extends AbsSourceParser { +public class KafkaSourceParser extends AbstractSourceParser { @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) throws Exception { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) throws Exception { KafkaSourceTableInfo kafkaSourceTableInfo = new KafkaSourceTableInfo(); - kafkaSourceTableInfo.setName(tableName); - kafkaSourceTableInfo.setType(MathUtil.getString(props.get(KafkaSourceTableInfo.TYPE_KEY.toLowerCase()))); parseFieldsInfo(fieldsInfo, kafkaSourceTableInfo); + kafkaSourceTableInfo.setName(tableName); + kafkaSourceTableInfo.setType(MathUtil.getString(props.get(KafkaSourceTableInfo.TYPE_KEY.toLowerCase()))); kafkaSourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); - String bootstrapServer = MathUtil.getString(props.get(KafkaSourceTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase())); - if (bootstrapServer == null || bootstrapServer.trim().equals("")) { - throw new Exception("BootstrapServers can not be empty!"); - } else { - kafkaSourceTableInfo.setBootstrapServers(bootstrapServer); - } + kafkaSourceTableInfo.setBootstrapServers(MathUtil.getString(props.get(KafkaSourceTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); kafkaSourceTableInfo.setGroupId(MathUtil.getString(props.get(KafkaSourceTableInfo.GROUPID_KEY.toLowerCase()))); kafkaSourceTableInfo.setTopic(MathUtil.getString(props.get(KafkaSourceTableInfo.TOPIC_KEY.toLowerCase()))); kafkaSourceTableInfo.setOffsetReset(MathUtil.getString(props.get(KafkaSourceTableInfo.OFFSETRESET_KEY.toLowerCase()))); kafkaSourceTableInfo.setTopicIsPattern(MathUtil.getBoolean(props.get(KafkaSourceTableInfo.TOPICISPATTERN_KEY.toLowerCase()), false)); + kafkaSourceTableInfo.setOffsetReset(MathUtil.getString(props.getOrDefault(KafkaSourceTableInfo.OFFSETRESET_KEY.toLowerCase(), EKafkaOffset.LATEST.name().toLowerCase()))); + kafkaSourceTableInfo.setTopicIsPattern(MathUtil.getBoolean(props.get(KafkaSourceTableInfo.TOPICISPATTERN_KEY.toLowerCase()))); kafkaSourceTableInfo.setTimeZone(MathUtil.getString(props.get(KafkaSourceTableInfo.TIME_ZONE_KEY.toLowerCase()))); - for (String key : props.keySet()) { - if (!key.isEmpty() && key.startsWith("kafka.")) { - kafkaSourceTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); - } - } + + kafkaSourceTableInfo.setSchemaString(MathUtil.getString(props.get(KafkaSourceTableInfo.SCHEMA_STRING_KEY.toLowerCase()))); + kafkaSourceTableInfo.setFieldDelimiter(MathUtil.getString(props.getOrDefault(KafkaSourceTableInfo.CSV_FIELD_DELIMITER_KEY.toLowerCase(), "|"))); + kafkaSourceTableInfo.setSourceDataType(MathUtil.getString(props.getOrDefault(KafkaSourceTableInfo.SOURCE_DATA_TYPE_KEY.toLowerCase(), FormatType.DT_NEST.name()))); + + Map kafkaParams = props.keySet().stream() + .filter(key -> !key.isEmpty() && key.startsWith("kafka.")) + .collect(Collectors.toMap( + key -> key.substring(6), key -> props.get(key).toString()) + ); + + kafkaSourceTableInfo.addKafkaParam(kafkaParams); kafkaSourceTableInfo.check(); + return kafkaSourceTableInfo; } } diff --git a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 4f4851335..c27eee376 100644 --- a/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -17,11 +17,10 @@ */ - package com.dtstack.flink.sql.source.kafka.table; import com.dtstack.flink.sql.format.FormatType; -import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.table.AbstractSourceTableInfo; import com.google.common.base.Preconditions; import java.util.HashMap; @@ -35,136 +34,126 @@ * @author sishu.yss */ -public class KafkaSourceTableInfo extends SourceTableInfo { +public class KafkaSourceTableInfo extends AbstractSourceTableInfo { + + public static final String BOOTSTRAPSERVERS_KEY = "bootstrapServers"; - public static final String BOOTSTRAPSERVERS_KEY = "bootstrapServers"; + public static final String TOPIC_KEY = "topic"; - public static final String TOPIC_KEY = "topic"; + public static final String TYPE_KEY = "type"; - public static final String TYPE_KEY = "type"; + public static final String GROUPID_KEY = "groupId"; - public static final String GROUPID_KEY = "groupId"; + public static final String OFFSETRESET_KEY = "offsetReset"; - public static final String OFFSETRESET_KEY = "offsetReset"; + public static final String TOPICISPATTERN_KEY = "topicIsPattern"; - public static final String TOPICISPATTERN_KEY = "topicIsPattern"; + public static final String SCHEMA_STRING_KEY = "schemaInfo"; - private String bootstrapServers; + public static final String CSV_FIELD_DELIMITER_KEY = "fieldDelimiter"; - private String topic; + public static final String SOURCE_DATA_TYPE_KEY = "sourceDataType"; - private String groupId; + private String bootstrapServers; - //latest, earliest - private String offsetReset = "latest"; + private String topic; - private String offset; + private String groupId; - private Boolean topicIsPattern = false; + private String offsetReset; - private String sourceDataType = FormatType.DT_NEST.name(); + private Boolean topicIsPattern = false; - private String schemaString; + private String sourceDataType; - private String fieldDelimiter; + private String schemaString; - public String getBootstrapServers() { - return bootstrapServers; - } + private String fieldDelimiter; - public void setBootstrapServers(String bootstrapServers) { - this.bootstrapServers = bootstrapServers; - } + public Map kafkaParam = new HashMap<>(); - public String getTopic() { - return topic; - } - public void setTopic(String topic) { - this.topic = topic; - } + public String getBootstrapServers() { + return bootstrapServers; + } - public String getGroupId() { - return groupId; - } + public void setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } - public void setGroupId(String groupId) { - this.groupId = groupId; - } + public String getTopic() { + return topic; + } - public String getOffsetReset() { - return offsetReset; - } + public void setTopic(String topic) { + this.topic = topic; + } - public void setOffsetReset(String offsetReset) { - if(offsetReset == null){ - return; - } - this.offsetReset = offsetReset; - } + public String getGroupId() { + return groupId; + } - public String getOffset() { - return offset; - } + public void setGroupId(String groupId) { + this.groupId = groupId; + } - public void setOffset(String offset) { - if (offsetReset == null) { - return; - } - this.offset = offset; - } + public String getOffsetReset() { + return offsetReset; + } - public Boolean getTopicIsPattern() { - return topicIsPattern; - } + public void setOffsetReset(String offsetReset) { + this.offsetReset = offsetReset; + } - public void setTopicIsPattern(Boolean topicIsPattern) { - this.topicIsPattern = topicIsPattern; - } + public Boolean getTopicIsPattern() { + return topicIsPattern; + } - public Map kafkaParam = new HashMap<>(); + public void setTopicIsPattern(Boolean topicIsPattern) { + this.topicIsPattern = topicIsPattern; + } - public void addKafkaParam(String key, String value) { - kafkaParam.put(key, value); - } + public void addKafkaParam(Map kafkaParam) { + kafkaParam.putAll(kafkaParam); + } - public String getKafkaParam(String key) { - return kafkaParam.get(key); - } + public String getKafkaParam(String key) { + return kafkaParam.get(key); + } - public Set getKafkaParamKeys() { - return kafkaParam.keySet(); - } + public Set getKafkaParamKeys() { + return kafkaParam.keySet(); + } - public String getSourceDataType() { - return sourceDataType; - } + public String getSourceDataType() { + return sourceDataType; + } - public void setSourceDataType(String sourceDataType) { - this.sourceDataType = sourceDataType; - } + public void setSourceDataType(String sourceDataType) { + this.sourceDataType = sourceDataType; + } - public String getSchemaString() { - return schemaString; - } + public String getSchemaString() { + return schemaString; + } - public void setSchemaString(String schemaString) { - this.schemaString = schemaString; - } + public void setSchemaString(String schemaString) { + this.schemaString = schemaString; + } - public String getFieldDelimiter() { - return fieldDelimiter; - } + public String getFieldDelimiter() { + return fieldDelimiter; + } - public void setFieldDelimiter(String fieldDelimiter) { - this.fieldDelimiter = fieldDelimiter; - } + public void setFieldDelimiter(String fieldDelimiter) { + this.fieldDelimiter = fieldDelimiter; + } - @Override - public boolean check() { - Preconditions.checkNotNull(getType(), "kafka of type is required"); - Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); - Preconditions.checkNotNull(topic, "kafka of topic is required"); - return false; - } + @Override + public boolean check() { + Preconditions.checkNotNull(getType(), "kafka of type is required"); + Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(topic, "kafka of topic is required"); + return false; + } } diff --git a/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer.java b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer.java index 71a9cc386..1cbbeafd9 100644 --- a/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer.java +++ b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer.java @@ -23,7 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.types.Row; +import org.apache.flink.table.runtime.types.CRow; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,7 +37,7 @@ * * @author maqi */ -public class KafkaProducer extends FlinkKafkaProducer { +public class KafkaProducer extends FlinkKafkaProducer { private static final Logger LOG = LoggerFactory.getLogger(KafkaProducer.class); @@ -45,7 +45,7 @@ public class KafkaProducer extends FlinkKafkaProducer { private SerializationMetricWrapper serializationMetricWrapper; - public KafkaProducer(String topicId, SerializationSchema serializationSchema, Properties producerConfig, Optional> customPartitioner, String[] parititonKeys) { + public KafkaProducer(String topicId, SerializationSchema serializationSchema, Properties producerConfig, Optional> customPartitioner, String[] parititonKeys) { super(topicId, new CustomerKeyedSerializationSchema((SerializationMetricWrapper)serializationSchema, parititonKeys), producerConfig, customPartitioner); this.serializationMetricWrapper = (SerializationMetricWrapper) serializationSchema; } diff --git a/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducerFactory.java b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducerFactory.java index f3a2f40f5..6bf9014df 100644 --- a/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducerFactory.java +++ b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducerFactory.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.types.Row; +import org.apache.flink.table.runtime.types.CRow; import java.util.Optional; import java.util.Properties; @@ -36,7 +36,7 @@ public class KafkaProducerFactory extends AbstractKafkaProducerFactory { @Override - public RichSinkFunction createKafkaProducer(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation typeInformation, Properties properties, Optional> partitioner, String[] partitionKeys) { + public RichSinkFunction createKafkaProducer(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation typeInformation, Properties properties, Optional> partitioner, String[] partitionKeys) { return new KafkaProducer(kafkaSinkTableInfo.getTopic(), createSerializationMetricWrapper(kafkaSinkTableInfo, typeInformation), properties, partitioner, partitionKeys); } } diff --git a/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index d0d477f4e..632bb720e 100644 --- a/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -18,23 +18,8 @@ package com.dtstack.flink.sql.sink.kafka; -import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; -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; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.sinks.RetractStreamTableSink; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.table.utils.TableConnectorUtils; -import org.apache.flink.types.Row; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import java.util.Optional; import java.util.Properties; @@ -44,104 +29,23 @@ * @create: 2019-11-05 11:45 * @description: **/ -public class KafkaSink implements RetractStreamTableSink, IStreamSinkGener { - - protected String[] fieldNames; - - protected TypeInformation[] fieldTypes; - - protected String topic; - - protected int parallelism; - - protected Properties properties; - - protected FlinkKafkaProducer flinkKafkaProducer; - - /** The schema of the table. */ - private TableSchema schema; - - /** Partitioner to select Kafka partition for each item. */ - protected Optional> partitioner; - - private String[] partitionKeys; - +public class KafkaSink extends AbstractKafkaSink { @Override - public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + public KafkaSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { KafkaSinkTableInfo kafkaSinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; - this.topic = kafkaSinkTableInfo.getTopic(); - - properties = new Properties(); - properties.setProperty("bootstrap.servers", kafkaSinkTableInfo.getBootstrapServers()); - for (String key : kafkaSinkTableInfo.getKafkaParamKeys()) { - properties.setProperty(key, kafkaSinkTableInfo.getKafkaParam(key)); - } + Properties kafkaProperties = getKafkaProperties(kafkaSinkTableInfo); + this.tableName = kafkaSinkTableInfo.getName(); + this.topic = kafkaSinkTableInfo.getTopic(); this.partitioner = Optional.of(new CustomerFlinkPartition<>()); this.partitionKeys = getPartitionKeys(kafkaSinkTableInfo); this.fieldNames = kafkaSinkTableInfo.getFields(); - TypeInformation[] types = new TypeInformation[kafkaSinkTableInfo.getFields().length]; - for (int i = 0; i < kafkaSinkTableInfo.getFieldClasses().length; i++) { - types[i] = TypeInformation.of(kafkaSinkTableInfo.getFieldClasses()[i]); - } - this.fieldTypes = types; - - TableSchema.Builder schemaBuilder = TableSchema.builder(); - for (int i=0;i) new KafkaProducerFactory().createKafkaProducer(kafkaSinkTableInfo, getOutputType().getTypeAt(1), properties, partitioner, partitionKeys); + this.fieldTypes = getTypeInformations(kafkaSinkTableInfo); + this.schema = buildTableSchema(fieldNames, fieldTypes); + this.parallelism = kafkaSinkTableInfo.getParallelism(); + this.sinkOperatorName = SINK_OPERATOR_NAME_TPL.replace("${topic}", topic).replace("${table}", tableName); + this.kafkaProducer = new KafkaProducerFactory().createKafkaProducer(kafkaSinkTableInfo, getRowTypeInfo(), kafkaProperties, partitioner, partitionKeys); return this; } - @Override - public TypeInformation getRecordType() { - return new RowTypeInfo(fieldTypes, fieldNames); - } - - @Override - public void emitDataStream(DataStream> dataStream) { - DataStream mapDataStream = dataStream.filter((Tuple2 record) -> record.f0) - .map((Tuple2 record) -> record.f1) - .returns(getOutputType().getTypeAt(1)) - .setParallelism(parallelism); - - mapDataStream.addSink(flinkKafkaProducer).name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); - } - - @Override - public TupleTypeInfo> getOutputType() { - return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), new RowTypeInfo(fieldTypes, fieldNames)); - } - - @Override - public String[] getFieldNames() { - return fieldNames; - } - - @Override - public TypeInformation[] getFieldTypes() { - return fieldTypes; - } - - @Override - public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { - this.fieldNames = fieldNames; - this.fieldTypes = fieldTypes; - return this; - } - - private String[] getPartitionKeys(KafkaSinkTableInfo kafkaSinkTableInfo){ - if(StringUtils.isNotBlank(kafkaSinkTableInfo.getPartitionKeys())){ - return StringUtils.split(kafkaSinkTableInfo.getPartitionKeys(), ','); - } - return null; - } } diff --git a/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index a445a6529..394ea86ee 100644 --- a/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -18,24 +18,17 @@ package com.dtstack.flink.sql.source.kafka; -import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; -import com.dtstack.flink.sql.table.SourceTableInfo; -import com.dtstack.flink.sql.util.DtStringUtil; -import com.dtstack.flink.sql.util.PluginUtil; +import com.dtstack.flink.sql.table.AbstractSourceTableInfo; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.types.Row; -import java.util.HashMap; -import java.util.Map; import java.util.Properties; /** @@ -43,75 +36,24 @@ * @create: 2019-11-05 10:55 * @description: **/ -public class KafkaSource implements IStreamSourceGener
{ +public class KafkaSource extends AbstractKafkaSource { - private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; - - /** - * Get kafka data source, you need to provide the data field names, data types - * If you do not specify auto.offset.reset, the default use groupoffset - * - * @param sourceTableInfo - * @return - */ - @SuppressWarnings("rawtypes") @Override - public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { - + public Table genStreamSource(AbstractSourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { KafkaSourceTableInfo kafkaSourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; String topicName = kafkaSourceTableInfo.getTopic(); - Properties props = new Properties(); - for (String key : kafkaSourceTableInfo.getKafkaParamKeys()) { - props.setProperty(key, kafkaSourceTableInfo.getKafkaParam(key)); - } - props.setProperty("bootstrap.servers", kafkaSourceTableInfo.getBootstrapServers()); - if (DtStringUtil.isJosn(kafkaSourceTableInfo.getOffsetReset())) { - props.setProperty("auto.offset.reset", "none"); - } else { - props.setProperty("auto.offset.reset", kafkaSourceTableInfo.getOffsetReset()); - } - if (StringUtils.isNotBlank(kafkaSourceTableInfo.getGroupId())) { - props.setProperty("group.id", kafkaSourceTableInfo.getGroupId()); - } - - TypeInformation[] types = new TypeInformation[kafkaSourceTableInfo.getFields().length]; - for (int i = 0; i < kafkaSourceTableInfo.getFieldClasses().length; i++) { - types[i] = TypeInformation.of(kafkaSourceTableInfo.getFieldClasses()[i]); - } - - TypeInformation typeInformation = new RowTypeInfo(types, kafkaSourceTableInfo.getFields()); + Properties kafkaProperties = getKafkaProperties(kafkaSourceTableInfo); + TypeInformation typeInformation = getRowTypeInformation(kafkaSourceTableInfo); + FlinkKafkaConsumer kafkaSrc = (FlinkKafkaConsumer) new KafkaConsumerFactory().createKafkaTableSource(kafkaSourceTableInfo, typeInformation, kafkaProperties); - FlinkKafkaConsumer kafkaSrc = (FlinkKafkaConsumer) new KafkaConsumerFactory().createKafkaTableSource(kafkaSourceTableInfo, typeInformation, props); - - - //earliest,latest - if ("earliest".equalsIgnoreCase(kafkaSourceTableInfo.getOffsetReset())) { - kafkaSrc.setStartFromEarliest(); - } else if (DtStringUtil.isJosn(kafkaSourceTableInfo.getOffsetReset())) {// {"0":12312,"1":12321,"2":12312} - try { - Properties properties = PluginUtil.jsonStrToObject(kafkaSourceTableInfo.getOffsetReset(), Properties.class); - Map offsetMap = PluginUtil.ObjectToMap(properties); - Map specificStartupOffsets = new HashMap<>(); - for (Map.Entry entry : offsetMap.entrySet()) { - specificStartupOffsets.put(new KafkaTopicPartition(topicName, Integer.valueOf(entry.getKey())), Long.valueOf(entry.getValue().toString())); - } - kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); - } catch (Exception e) { - throw new RuntimeException("not support offsetReset type:" + kafkaSourceTableInfo.getOffsetReset()); - } - } else { - kafkaSrc.setStartFromLatest(); - } + String sourceOperatorName = generateOperatorName(sourceTableInfo.getName(), topicName); + DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); + kafkaSource.setParallelism(kafkaSourceTableInfo.getParallelism()); + setStartPosition(kafkaSourceTableInfo.getOffsetReset(), topicName, kafkaSrc); String fields = StringUtils.join(kafkaSourceTableInfo.getFields(), ","); - String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); - DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); - Integer parallelism = kafkaSourceTableInfo.getParallelism(); - if (parallelism != null) { - kafkaSource.setParallelism(parallelism); - } return tableEnv.fromDataStream(kafkaSource, fields); } } diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09.java index c815e134a..bee1865dd 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer09; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,7 +38,7 @@ * * @author maqi */ -public class KafkaProducer09 extends FlinkKafkaProducer09 { +public class KafkaProducer09 extends FlinkKafkaProducer09 { private static final Logger LOG = LoggerFactory.getLogger(KafkaProducer09.class); @@ -45,7 +46,7 @@ public class KafkaProducer09 extends FlinkKafkaProducer09 { private SerializationMetricWrapper serializationMetricWrapper; - public KafkaProducer09(String topicId, SerializationSchema serializationSchema, Properties producerConfig, Optional> customPartitioner,String[] partitionKeys) { + public KafkaProducer09(String topicId, SerializationSchema serializationSchema, Properties producerConfig, Optional> customPartitioner, String[] partitionKeys) { super(topicId, new CustomerKeyedSerializationSchema((SerializationMetricWrapper)serializationSchema, partitionKeys), producerConfig, customPartitioner.orElse(null)); this.serializationMetricWrapper = (SerializationMetricWrapper) serializationSchema; } diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09Factory.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09Factory.java index 7fb3909ee..ee3423b07 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09Factory.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer09Factory.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.types.Row; import java.util.Optional; @@ -36,7 +37,7 @@ public class KafkaProducer09Factory extends AbstractKafkaProducerFactory { @Override - public RichSinkFunction createKafkaProducer(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation typeInformation, Properties properties, Optional> partitioner,String[] partitionKeys) { + public RichSinkFunction createKafkaProducer(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation typeInformation, Properties properties, Optional> partitioner, String[] partitionKeys) { return new KafkaProducer09(kafkaSinkTableInfo.getTopic(), createSerializationMetricWrapper(kafkaSinkTableInfo, typeInformation), properties, partitioner, partitionKeys); } } 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 52caa7bd2..d22be3d59 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 @@ -18,23 +18,8 @@ package com.dtstack.flink.sql.sink.kafka; -import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; -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; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer09; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.sinks.RetractStreamTableSink; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.table.utils.TableConnectorUtils; -import org.apache.flink.types.Row; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import java.util.Optional; import java.util.Properties; @@ -46,109 +31,22 @@ * @author DocLi * @modifyer maqi */ -public class KafkaSink implements RetractStreamTableSink, IStreamSinkGener { - - protected String[] fieldNames; - - protected TypeInformation[] fieldTypes; - - protected String topic; - - protected Properties properties; - - protected FlinkKafkaProducer09 kafkaProducer09; - - /** The schema of the table. */ - private TableSchema schema; - - /** Partitioner to select Kafka partition for each item. */ - protected Optional> partitioner; - - private String[] partitionKeys; - - protected int parallelism; - - - +public class KafkaSink extends AbstractKafkaSink{ @Override - public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + public KafkaSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { KafkaSinkTableInfo kafka09SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; - this.topic = kafka09SinkTableInfo.getTopic(); - - properties = new Properties(); - properties.setProperty("bootstrap.servers", kafka09SinkTableInfo.getBootstrapServers()); - for (String key : kafka09SinkTableInfo.getKafkaParamKeys()) { - properties.setProperty(key, kafka09SinkTableInfo.getKafkaParam(key)); - } + Properties kafkaProperties = getKafkaProperties(kafka09SinkTableInfo); + this.tableName = kafka09SinkTableInfo.getName(); + this.topic = kafka09SinkTableInfo.getTopic(); this.partitioner = Optional.of(new CustomerFlinkPartition<>()); this.partitionKeys = getPartitionKeys(kafka09SinkTableInfo); this.fieldNames = kafka09SinkTableInfo.getFields(); - TypeInformation[] types = new TypeInformation[kafka09SinkTableInfo.getFields().length]; - for (int i = 0; i < kafka09SinkTableInfo.getFieldClasses().length; i++) { - types[i] = TypeInformation.of(kafka09SinkTableInfo.getFieldClasses()[i]); - } - this.fieldTypes = types; - - TableSchema.Builder schemaBuilder = TableSchema.builder(); - for (int i=0;i) new KafkaProducer09Factory() - .createKafkaProducer(kafka09SinkTableInfo, getOutputType().getTypeAt(1), properties, partitioner, partitionKeys); + this.fieldTypes = getTypeInformations(kafka09SinkTableInfo); + this.schema = buildTableSchema(fieldNames, fieldTypes); + this.parallelism = kafka09SinkTableInfo.getParallelism(); + this.sinkOperatorName = SINK_OPERATOR_NAME_TPL.replace("${topic}", topic).replace("${table}", tableName); + this.kafkaProducer = new KafkaProducer09Factory().createKafkaProducer(kafka09SinkTableInfo, getRowTypeInfo(), kafkaProperties, partitioner, partitionKeys); return this; } - - @Override - public TypeInformation getRecordType() { - return new RowTypeInfo(fieldTypes, fieldNames); - } - - @Override - public void emitDataStream(DataStream> dataStream) { - DataStream mapDataStream = dataStream.filter((Tuple2 record) -> record.f0) - .map((Tuple2 record) -> record.f1) - .returns(getOutputType().getTypeAt(1)) - .setParallelism(parallelism); - - mapDataStream.addSink(kafkaProducer09) - .name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); - } - - @Override - public TupleTypeInfo> getOutputType() { - return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), new RowTypeInfo(fieldTypes, fieldNames)); - } - - @Override - public String[] getFieldNames() { - return fieldNames; - } - - @Override - public TypeInformation[] getFieldTypes() { - return fieldTypes; - } - - @Override - public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { - this.fieldNames = fieldNames; - this.fieldTypes = fieldTypes; - return this; - } - - private String[] getPartitionKeys(KafkaSinkTableInfo kafkaSinkTableInfo){ - if(StringUtils.isNotBlank(kafkaSinkTableInfo.getPartitionKeys())){ - return StringUtils.split(kafkaSinkTableInfo.getPartitionKeys(), ','); - } - return null; - } - } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 21ed7c3e3..9f8917761 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -17,17 +17,13 @@ */ - package com.dtstack.flink.sql.source.kafka; -import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; -import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.table.AbstractSourceTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; -import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; @@ -36,7 +32,6 @@ import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.types.Row; -import java.util.HashMap; import java.util.Map; import java.util.Properties; @@ -47,74 +42,23 @@ * @author xuchao */ -public class KafkaSource implements IStreamSourceGener
{ - - private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; - - /** - * Get kafka data source, you need to provide the data field names, data types - * If you do not specify auto.offset.reset, the default use groupoffset - * @param sourceTableInfo - * @return - */ - @SuppressWarnings("rawtypes") +public class KafkaSource extends AbstractKafkaSource { @Override - public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { - + public Table genStreamSource(AbstractSourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { KafkaSourceTableInfo kafkaSourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; String topicName = kafkaSourceTableInfo.getTopic(); - Properties props = new Properties(); - for (String key : kafkaSourceTableInfo.getKafkaParamKeys()) { - props.setProperty(key, kafkaSourceTableInfo.getKafkaParam(key)); - } - props.setProperty("bootstrap.servers", kafkaSourceTableInfo.getBootstrapServers()); - if (DtStringUtil.isJosn(kafkaSourceTableInfo.getOffsetReset())){ - props.setProperty("auto.offset.reset", "none"); - } else { - props.setProperty("auto.offset.reset", kafkaSourceTableInfo.getOffsetReset()); - } - if (StringUtils.isNotBlank(kafkaSourceTableInfo.getGroupId())){ - props.setProperty("group.id", kafkaSourceTableInfo.getGroupId()); - } - // only required for Kafka 0.8 - //TODO props.setProperty("zookeeper.connect", kafkaSourceTableInfo.) + Properties kafkaProperties = getKafkaProperties(kafkaSourceTableInfo); + TypeInformation typeInformation = getRowTypeInformation(kafkaSourceTableInfo); + FlinkKafkaConsumer09 kafkaSrc = (FlinkKafkaConsumer09) new KafkaConsumer09Factory().createKafkaTableSource(kafkaSourceTableInfo, typeInformation, kafkaProperties); - TypeInformation[] types = new TypeInformation[kafkaSourceTableInfo.getFields().length]; - for(int i = 0; i< kafkaSourceTableInfo.getFieldClasses().length; i++){ - types[i] = TypeInformation.of(kafkaSourceTableInfo.getFieldClasses()[i]); - } - - TypeInformation typeInformation = new RowTypeInfo(types, kafkaSourceTableInfo.getFields()); - FlinkKafkaConsumer09 kafkaSrc = (FlinkKafkaConsumer09) new KafkaConsumer09Factory().createKafkaTableSource(kafkaSourceTableInfo, typeInformation, props); - - //earliest,latest - if("earliest".equalsIgnoreCase(kafkaSourceTableInfo.getOffsetReset())){ - kafkaSrc.setStartFromEarliest(); - }else if(DtStringUtil.isJosn(kafkaSourceTableInfo.getOffsetReset())){// {"0":12312,"1":12321,"2":12312} - try { - Properties properties = PluginUtil.jsonStrToObject(kafkaSourceTableInfo.getOffsetReset(), Properties.class); - Map offsetMap = PluginUtil.ObjectToMap(properties); - Map specificStartupOffsets = new HashMap<>(); - for(Map.Entry entry:offsetMap.entrySet()){ - specificStartupOffsets.put(new KafkaTopicPartition(topicName,Integer.valueOf(entry.getKey())),Long.valueOf(entry.getValue().toString())); - } - kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); - } catch (Exception e) { - throw new RuntimeException("not support offsetReset type:" + kafkaSourceTableInfo.getOffsetReset()); - } - }else { - kafkaSrc.setStartFromLatest(); - } + String sourceOperatorName = generateOperatorName(sourceTableInfo.getName(), topicName); + DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); + kafkaSource.setParallelism(kafkaSourceTableInfo.getParallelism()); + setStartPosition(kafkaSourceTableInfo.getOffsetReset(), topicName, kafkaSrc); String fields = StringUtils.join(kafkaSourceTableInfo.getFields(), ","); - String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); - DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); - Integer parallelism = kafkaSourceTableInfo.getParallelism(); - if (parallelism != null) { - kafkaSource.setParallelism(parallelism); - } return tableEnv.fromDataStream(kafkaSource, fields); } } \ No newline at end of file diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer010.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer010.java index 3cdc45dec..3936575ef 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer010.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer010.java @@ -23,7 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.types.Row; +import org.apache.flink.table.runtime.types.CRow; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,7 +37,7 @@ * * @author maqi */ -public class KafkaProducer010 extends FlinkKafkaProducer010 { +public class KafkaProducer010 extends FlinkKafkaProducer010 { private static final Logger LOG = LoggerFactory.getLogger(KafkaProducer010.class); @@ -45,7 +45,7 @@ public class KafkaProducer010 extends FlinkKafkaProducer010 { private SerializationMetricWrapper serializationMetricWrapper; - public KafkaProducer010(String topicId, SerializationSchema serializationSchema, Properties producerConfig, Optional> customPartitioner, String[] partitionKeys) { + public KafkaProducer010(String topicId, SerializationSchema serializationSchema, Properties producerConfig, Optional> customPartitioner, String[] partitionKeys) { super(topicId, new CustomerKeyedSerializationSchema((SerializationMetricWrapper)serializationSchema, partitionKeys), producerConfig, customPartitioner.get()); this.serializationMetricWrapper = (SerializationMetricWrapper) serializationSchema; } diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer010Factory.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer010Factory.java index c44a9fe86..e0e023586 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer010Factory.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer010Factory.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.types.Row; import java.util.Optional; @@ -36,7 +37,7 @@ public class KafkaProducer010Factory extends AbstractKafkaProducerFactory { @Override - public RichSinkFunction createKafkaProducer(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation typeInformation, Properties properties, Optional> partitioner, String[] partitionKeys) { + public RichSinkFunction createKafkaProducer(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation typeInformation, Properties properties, Optional> partitioner, String[] partitionKeys) { return new KafkaProducer010(kafkaSinkTableInfo.getTopic(), createSerializationMetricWrapper(kafkaSinkTableInfo, typeInformation), properties, partitioner, partitionKeys); } } 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 42f1045bf..eea78e121 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 @@ -18,21 +18,8 @@ package com.dtstack.flink.sql.sink.kafka; -import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; -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; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.sinks.RetractStreamTableSink; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.table.utils.TableConnectorUtils; -import org.apache.flink.types.Row; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import java.util.Optional; import java.util.Properties; @@ -46,105 +33,22 @@ * @modifyer maqi * */ -public class KafkaSink implements RetractStreamTableSink, IStreamSinkGener { - - - protected String[] fieldNames; - - protected TypeInformation[] fieldTypes; - - protected String topic; - - protected Properties properties; - - protected int parallelism; - - protected KafkaSinkTableInfo kafka10SinkTableInfo; - - /** The schema of the table. */ - private TableSchema schema; - - private String[] partitionKeys; - +public class KafkaSink extends AbstractKafkaSink { @Override - public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { - this.kafka10SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; - this.topic = kafka10SinkTableInfo.getTopic(); - - properties = new Properties(); - properties.setProperty("bootstrap.servers", kafka10SinkTableInfo.getBootstrapServers()); - - for (String key : kafka10SinkTableInfo.getKafkaParamKeys()) { - properties.setProperty(key, kafka10SinkTableInfo.getKafkaParam(key)); - } + public KafkaSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { + KafkaSinkTableInfo kafka10SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + Properties kafkaProperties = getKafkaProperties(kafka10SinkTableInfo); + this.tableName = kafka10SinkTableInfo.getName(); + this.topic = kafka10SinkTableInfo.getTopic(); + this.partitioner = Optional.of(new CustomerFlinkPartition<>()); this.partitionKeys = getPartitionKeys(kafka10SinkTableInfo); this.fieldNames = kafka10SinkTableInfo.getFields(); - TypeInformation[] types = new TypeInformation[kafka10SinkTableInfo.getFields().length]; - for (int i = 0; i < kafka10SinkTableInfo.getFieldClasses().length; i++) { - types[i] = TypeInformation.of(kafka10SinkTableInfo.getFieldClasses()[i]); - } - this.fieldTypes = types; - - - TableSchema.Builder schemaBuilder = TableSchema.builder(); - for (int i=0;i getRecordType() { - return new RowTypeInfo(fieldTypes, fieldNames); - } - - @Override - public void emitDataStream(DataStream> dataStream) { - - RichSinkFunction kafkaProducer010 = new KafkaProducer010Factory().createKafkaProducer(kafka10SinkTableInfo, getOutputType().getTypeAt(1), properties, - Optional.of(new CustomerFlinkPartition<>()), partitionKeys); - - DataStream mapDataStream = dataStream.filter((Tuple2 record) -> record.f0) - .map((Tuple2 record) -> record.f1) - .returns(getOutputType().getTypeAt(1)) - .setParallelism(parallelism); - - mapDataStream.addSink(kafkaProducer010).name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); - } - - @Override - public TupleTypeInfo> getOutputType() { - return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), new RowTypeInfo(fieldTypes, fieldNames)); - } - - @Override - public String[] getFieldNames() { - return fieldNames; - } - - @Override - public TypeInformation[] getFieldTypes() { - return fieldTypes; - } - - @Override - public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { - this.fieldNames = fieldNames; - this.fieldTypes = fieldTypes; - return this; - } - private String[] getPartitionKeys(KafkaSinkTableInfo kafkaSinkTableInfo){ - if(StringUtils.isNotBlank(kafkaSinkTableInfo.getPartitionKeys())){ - return StringUtils.split(kafkaSinkTableInfo.getPartitionKeys(), ','); - } - return null; - } - } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 55cffbf1e..23989ab7e 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -19,24 +19,17 @@ package com.dtstack.flink.sql.source.kafka; -import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; -import com.dtstack.flink.sql.table.SourceTableInfo; -import com.dtstack.flink.sql.util.DtStringUtil; -import com.dtstack.flink.sql.util.PluginUtil; +import com.dtstack.flink.sql.table.AbstractSourceTableInfo; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.types.Row; -import java.util.HashMap; -import java.util.Map; import java.util.Properties; /** @@ -47,74 +40,24 @@ * @author sishu.yss */ -public class KafkaSource implements IStreamSourceGener
{ +public class KafkaSource extends AbstractKafkaSource { - private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; + @Override + public Table genStreamSource(AbstractSourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { + KafkaSourceTableInfo kafkaSourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; + String topicName = kafkaSourceTableInfo.getTopic(); - /** - * Get kafka data source, you need to provide the data field names, data types - * If you do not specify auto.offset.reset, the default use groupoffset - * - * @param sourceTableInfo - * @return - */ - @SuppressWarnings("rawtypes") - @Override - public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { + Properties kafkaProperties = getKafkaProperties(kafkaSourceTableInfo); + TypeInformation typeInformation = getRowTypeInformation(kafkaSourceTableInfo); + FlinkKafkaConsumer010 kafkaSrc = (FlinkKafkaConsumer010) new KafkaConsumer010Factory().createKafkaTableSource(kafkaSourceTableInfo, typeInformation, kafkaProperties); - KafkaSourceTableInfo kafkaSourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; - String topicName = kafkaSourceTableInfo.getTopic(); + String sourceOperatorName = generateOperatorName(sourceTableInfo.getName(), topicName); + DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); + kafkaSource.setParallelism(kafkaSourceTableInfo.getParallelism()); - Properties props = new Properties(); - for (String key : kafkaSourceTableInfo.getKafkaParamKeys()) { - props.setProperty(key, kafkaSourceTableInfo.getKafkaParam(key)); - } - props.setProperty("bootstrap.servers", kafkaSourceTableInfo.getBootstrapServers()); - if (DtStringUtil.isJosn(kafkaSourceTableInfo.getOffsetReset())){ - props.setProperty("auto.offset.reset", "none"); - } else { - props.setProperty("auto.offset.reset", kafkaSourceTableInfo.getOffsetReset()); - } - if (StringUtils.isNotBlank(kafkaSourceTableInfo.getGroupId())){ - props.setProperty("group.id", kafkaSourceTableInfo.getGroupId()); - } + setStartPosition(kafkaSourceTableInfo.getOffsetReset(), topicName, kafkaSrc); + String fields = StringUtils.join(kafkaSourceTableInfo.getFields(), ","); - TypeInformation[] types = new TypeInformation[kafkaSourceTableInfo.getFields().length]; - for (int i = 0; i < kafkaSourceTableInfo.getFieldClasses().length; i++) { - types[i] = TypeInformation.of(kafkaSourceTableInfo.getFieldClasses()[i]); - } - - TypeInformation typeInformation = new RowTypeInfo(types, kafkaSourceTableInfo.getFields()); - - FlinkKafkaConsumer010 kafkaSrc = (FlinkKafkaConsumer010) new KafkaConsumer010Factory().createKafkaTableSource(kafkaSourceTableInfo, typeInformation, props); - - //earliest,latest - if ("earliest".equalsIgnoreCase(kafkaSourceTableInfo.getOffsetReset())) { - kafkaSrc.setStartFromEarliest(); - } else if (DtStringUtil.isJosn(kafkaSourceTableInfo.getOffsetReset())) {// {"0":12312,"1":12321,"2":12312} - try { - Properties properties = PluginUtil.jsonStrToObject(kafkaSourceTableInfo.getOffsetReset(), Properties.class); - Map offsetMap = PluginUtil.ObjectToMap(properties); - Map specificStartupOffsets = new HashMap<>(); - for (Map.Entry entry : offsetMap.entrySet()) { - specificStartupOffsets.put(new KafkaTopicPartition(topicName, Integer.valueOf(entry.getKey())), Long.valueOf(entry.getValue().toString())); - } - kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); - } catch (Exception e) { - throw new RuntimeException("not support offsetReset type:" + kafkaSourceTableInfo.getOffsetReset()); - } - } else { - kafkaSrc.setStartFromLatest(); - } - - String fields = StringUtils.join(kafkaSourceTableInfo.getFields(), ","); - String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); - - DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); - Integer parallelism = kafkaSourceTableInfo.getParallelism(); - if (parallelism != null) { - kafkaSource.setParallelism(parallelism); - } - return tableEnv.fromDataStream(kafkaSource, fields); - } + return tableEnv.fromDataStream(kafkaSource, fields); + } } \ No newline at end of file diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer011.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer011.java index 7880dd419..429d21a79 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer011.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer011.java @@ -24,7 +24,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.types.Row; +import org.apache.flink.table.runtime.types.CRow; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,7 +38,7 @@ * * @author maqi */ -public class KafkaProducer011 extends FlinkKafkaProducer011 { +public class KafkaProducer011 extends FlinkKafkaProducer011 { private static final Logger LOG = LoggerFactory.getLogger(KafkaProducer011.class); @@ -46,7 +46,7 @@ public class KafkaProducer011 extends FlinkKafkaProducer011 { private SerializationMetricWrapper serializationMetricWrapper; - public KafkaProducer011(String topicId, SerializationSchema serializationSchema, Properties producerConfig, Optional> customPartitioner, String[] partitionKeys) { + public KafkaProducer011(String topicId, SerializationSchema serializationSchema, Properties producerConfig, Optional> customPartitioner, String[] partitionKeys) { super(topicId, new CustomerKeyedSerializationSchema((SerializationMetricWrapper)serializationSchema, partitionKeys), producerConfig, customPartitioner); this.serializationMetricWrapper = (SerializationMetricWrapper) serializationSchema; } diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer011Factory.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer011Factory.java index e2272b16e..0cb11da82 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer011Factory.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaProducer011Factory.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.types.Row; import java.util.Optional; @@ -36,7 +37,7 @@ public class KafkaProducer011Factory extends AbstractKafkaProducerFactory { @Override - public RichSinkFunction createKafkaProducer(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation typeInformation, Properties properties, Optional> partitioner, + public RichSinkFunction createKafkaProducer(KafkaSinkTableInfo kafkaSinkTableInfo, TypeInformation typeInformation, Properties properties, Optional> partitioner, String[] partitionKeys) { return new KafkaProducer011(kafkaSinkTableInfo.getTopic(), createSerializationMetricWrapper(kafkaSinkTableInfo, typeInformation), properties, partitioner, partitionKeys); } 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 c5da782d0..ea45280c7 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 @@ -18,23 +18,8 @@ package com.dtstack.flink.sql.sink.kafka; -import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; -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; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.sinks.RetractStreamTableSink; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.table.utils.TableConnectorUtils; -import org.apache.flink.types.Row; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import java.util.Optional; import java.util.Properties; @@ -49,106 +34,22 @@ * @modifyer maqi * */ -public class KafkaSink implements RetractStreamTableSink, IStreamSinkGener { - - protected String[] fieldNames; - - protected TypeInformation[] fieldTypes; - - protected String topic; - - protected int parallelism; - - protected Properties properties; - - protected FlinkKafkaProducer011 kafkaProducer011; - - /** The schema of the table. */ - private TableSchema schema; - - /** Partitioner to select Kafka partition for each item. */ - protected Optional> partitioner; - private String[] partitionKeys; - - +public class KafkaSink extends AbstractKafkaSink { @Override - public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + public KafkaSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { KafkaSinkTableInfo kafka11SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; - this.topic = kafka11SinkTableInfo.getTopic(); - - properties = new Properties(); - properties.setProperty("bootstrap.servers", kafka11SinkTableInfo.getBootstrapServers()); - for (String key : kafka11SinkTableInfo.getKafkaParamKeys()) { - properties.setProperty(key, kafka11SinkTableInfo.getKafkaParam(key)); - } + Properties kafkaProperties = getKafkaProperties(kafka11SinkTableInfo); + this.tableName = kafka11SinkTableInfo.getName(); + this.topic = kafka11SinkTableInfo.getTopic(); this.partitioner = Optional.of(new CustomerFlinkPartition<>()); this.partitionKeys = getPartitionKeys(kafka11SinkTableInfo); this.fieldNames = kafka11SinkTableInfo.getFields(); - TypeInformation[] types = new TypeInformation[kafka11SinkTableInfo.getFields().length]; - for (int i = 0; i < kafka11SinkTableInfo.getFieldClasses().length; i++) { - types[i] = TypeInformation.of(kafka11SinkTableInfo.getFieldClasses()[i]); - } - this.fieldTypes = types; - - TableSchema.Builder schemaBuilder = TableSchema.builder(); - for (int i=0;i) new KafkaProducer011Factory() - .createKafkaProducer(kafka11SinkTableInfo, getOutputType().getTypeAt(1), properties, partitioner, partitionKeys); + this.fieldTypes = getTypeInformations(kafka11SinkTableInfo); + this.schema = buildTableSchema(fieldNames, fieldTypes); + this.parallelism = kafka11SinkTableInfo.getParallelism(); + this.sinkOperatorName = SINK_OPERATOR_NAME_TPL.replace("${topic}", topic).replace("${table}", tableName); + this.kafkaProducer = new KafkaProducer011Factory().createKafkaProducer(kafka11SinkTableInfo, getRowTypeInfo(), kafkaProperties, partitioner, partitionKeys); return this; } - - @Override - public TypeInformation getRecordType() { - return new RowTypeInfo(fieldTypes, fieldNames); - } - - @Override - public void emitDataStream(DataStream> dataStream) { - DataStream mapDataStream = dataStream.filter((Tuple2 record) -> record.f0) - .map((Tuple2 record) -> record.f1) - .returns(getOutputType().getTypeAt(1)) - .setParallelism(parallelism); - - mapDataStream.addSink(kafkaProducer011).name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); - } - - @Override - public TupleTypeInfo> getOutputType() { - return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), new RowTypeInfo(fieldTypes, fieldNames)); - } - - @Override - public String[] getFieldNames() { - return fieldNames; - } - - @Override - public TypeInformation[] getFieldTypes() { - return fieldTypes; - } - - @Override - public TableSink>configure(String[] fieldNames, TypeInformation[] fieldTypes) { - this.fieldNames = fieldNames; - this.fieldTypes = fieldTypes; - return this; - } - - private String[] getPartitionKeys(KafkaSinkTableInfo kafkaSinkTableInfo){ - if(StringUtils.isNotBlank(kafkaSinkTableInfo.getPartitionKeys())){ - return StringUtils.split(kafkaSinkTableInfo.getPartitionKeys(), ','); - } - return null; - } - } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaConsumer011Factory.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaConsumer011Factory.java index 3f804fc6c..c20f0678b 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaConsumer011Factory.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaConsumer011Factory.java @@ -31,8 +31,8 @@ /** * company: www.dtstack.com - * author: toutian * create: 2019/12/24 + * @author: toutian */ public class KafkaConsumer011Factory extends AbstractKafkaConsumerFactory { diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 1a4eed08a..f58d59d05 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -19,24 +19,16 @@ package com.dtstack.flink.sql.source.kafka; -import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; -import com.dtstack.flink.sql.table.SourceTableInfo; -import com.dtstack.flink.sql.util.DtStringUtil; -import com.dtstack.flink.sql.util.PluginUtil; +import com.dtstack.flink.sql.table.AbstractSourceTableInfo; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.types.Row; - -import java.util.HashMap; -import java.util.Map; import java.util.Properties; /** @@ -47,76 +39,24 @@ * @author sishu.yss */ -public class KafkaSource implements IStreamSourceGener
{ - - private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; - - /** - * Get kafka data source, you need to provide the data field names, data types - * If you do not specify auto.offset.reset, the default use groupoffset - * - * @param sourceTableInfo - * @return - */ - @SuppressWarnings("rawtypes") - @Override - public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { - - KafkaSourceTableInfo kafkaSourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; - String topicName = kafkaSourceTableInfo.getTopic(); - - Properties props = new Properties(); - for (String key : kafkaSourceTableInfo.getKafkaParamKeys()) { - props.setProperty(key, kafkaSourceTableInfo.getKafkaParam(key)); - } - props.setProperty("bootstrap.servers", kafkaSourceTableInfo.getBootstrapServers()); - if (DtStringUtil.isJosn(kafkaSourceTableInfo.getOffsetReset())){ - props.setProperty("auto.offset.reset", "none"); - } else { - props.setProperty("auto.offset.reset", kafkaSourceTableInfo.getOffsetReset()); - } - if (StringUtils.isNotBlank(kafkaSourceTableInfo.getGroupId())){ - props.setProperty("group.id", kafkaSourceTableInfo.getGroupId()); - } - // only required for Kafka 0.8 - //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) - - TypeInformation[] types = new TypeInformation[kafkaSourceTableInfo.getFields().length]; - for (int i = 0; i < kafkaSourceTableInfo.getFieldClasses().length; i++) { - types[i] = TypeInformation.of(kafkaSourceTableInfo.getFieldClasses()[i]); - } +public class KafkaSource extends AbstractKafkaSource { - TypeInformation typeInformation = new RowTypeInfo(types, kafkaSourceTableInfo.getFields()); + @Override + public Table genStreamSource(AbstractSourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { + KafkaSourceTableInfo kafkaSourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; + String topicName = kafkaSourceTableInfo.getTopic(); - FlinkKafkaConsumer011 kafkaSrc = (FlinkKafkaConsumer011) new KafkaConsumer011Factory().createKafkaTableSource(kafkaSourceTableInfo, typeInformation, props); + Properties kafkaProperties = getKafkaProperties(kafkaSourceTableInfo); + TypeInformation typeInformation = getRowTypeInformation(kafkaSourceTableInfo); + FlinkKafkaConsumer011 kafkaSrc = (FlinkKafkaConsumer011) new KafkaConsumer011Factory().createKafkaTableSource(kafkaSourceTableInfo, typeInformation, kafkaProperties); - //earliest,latest - if ("earliest".equalsIgnoreCase(kafkaSourceTableInfo.getOffsetReset())) { - kafkaSrc.setStartFromEarliest(); - } else if (DtStringUtil.isJosn(kafkaSourceTableInfo.getOffsetReset())) {// {"0":12312,"1":12321,"2":12312} - try { - Properties properties = PluginUtil.jsonStrToObject(kafkaSourceTableInfo.getOffsetReset(), Properties.class); - Map offsetMap = PluginUtil.ObjectToMap(properties); - Map specificStartupOffsets = new HashMap<>(); - for (Map.Entry entry : offsetMap.entrySet()) { - specificStartupOffsets.put(new KafkaTopicPartition(topicName, Integer.valueOf(entry.getKey())), Long.valueOf(entry.getValue().toString())); - } - kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); - } catch (Exception e) { - throw new RuntimeException("not support offsetReset type:" + kafkaSourceTableInfo.getOffsetReset()); - } - } else { - kafkaSrc.setStartFromLatest(); - } + String sourceOperatorName = generateOperatorName(sourceTableInfo.getName(), topicName); + DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); + kafkaSource.setParallelism(kafkaSourceTableInfo.getParallelism()); - String fields = StringUtils.join(kafkaSourceTableInfo.getFields(), ","); - String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); + setStartPosition(kafkaSourceTableInfo.getOffsetReset(), topicName, kafkaSrc); + String fields = StringUtils.join(kafkaSourceTableInfo.getFields(), ","); - DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); - Integer parallelism = kafkaSourceTableInfo.getParallelism(); - if (parallelism != null) { - kafkaSource.setParallelism(parallelism); - } - return tableEnv.fromDataStream(kafkaSource, fields); - } + return tableEnv.fromDataStream(kafkaSource, fields); + } } 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 63ff31c50..31f9aa13d 100644 --- a/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllReqRow.java +++ b/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllReqRow.java @@ -1,24 +1,23 @@ package com.dtstack.flink.sql.side.kudu; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.table.runtime.types.CRow; -import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; -import org.apache.flink.types.Row; -import org.apache.flink.util.Collector; -import org.apache.flink.util.Preconditions; - -import com.dtstack.flink.sql.side.AllReqRow; +import com.dtstack.flink.sql.side.BaseAllReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.PredicateInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +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.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.calcite.sql.JoinType; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.runtime.types.CRow; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; import org.apache.kudu.ColumnSchema; import org.apache.kudu.Schema; import org.apache.kudu.client.KuduClient; @@ -41,7 +40,7 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicReference; -public class KuduAllReqRow extends AllReqRow { +public class KuduAllReqRow extends BaseAllReqRow { private static final long serialVersionUID = 6051774809356082219L; @@ -62,7 +61,7 @@ public class KuduAllReqRow extends AllReqRow { private AtomicReference>>> cacheRef = new AtomicReference<>(); - public KuduAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public KuduAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new KuduAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @@ -229,9 +228,6 @@ private KuduScanner getConn(KuduSideTableInfo tableInfo) { if (null != workerCount) { kuduClientBuilder.workerCount(workerCount); } - if (null != defaultSocketReadTimeoutMs) { - kuduClientBuilder.defaultSocketReadTimeoutMs(defaultSocketReadTimeoutMs); - } if (null != defaultOperationTimeoutMs) { kuduClientBuilder.defaultOperationTimeoutMs(defaultOperationTimeoutMs); diff --git a/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllSideInfo.java b/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllSideInfo.java index ddfa73201..1241a6f37 100644 --- a/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllSideInfo.java +++ b/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllSideInfo.java @@ -2,8 +2,8 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.BaseSideInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlNode; import org.apache.commons.collections.CollectionUtils; @@ -12,16 +12,16 @@ import java.util.List; -public class KuduAllSideInfo extends SideInfo { +public class KuduAllSideInfo extends BaseSideInfo { private static final long serialVersionUID = 9005389633060174746L; - public KuduAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public KuduAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + public void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo) { // no use } 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 e1705d796..e534a85f9 100644 --- a/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncReqRow.java +++ b/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncReqRow.java @@ -1,20 +1,12 @@ package com.dtstack.flink.sql.side.kudu; -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.table.typeutils.TimeIndicatorTypeInfo; -import org.apache.flink.types.Row; -import org.apache.flink.util.Preconditions; - import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.AsyncReqRow; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.BaseAsyncReqRow; import com.dtstack.flink.sql.side.CacheMissVal; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.PredicateInfo; -import com.dtstack.flink.sql.side.SideTableInfo; 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; @@ -24,6 +16,13 @@ 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; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.runtime.types.CRow; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; import org.apache.kudu.ColumnSchema; import org.apache.kudu.Schema; import org.apache.kudu.client.AsyncKuduClient; @@ -42,7 +41,7 @@ import java.util.List; import java.util.Map; -public class KuduAsyncReqRow extends AsyncReqRow { +public class KuduAsyncReqRow extends BaseAsyncReqRow { private static final Logger LOG = LoggerFactory.getLogger(KuduAsyncReqRow.class); /** @@ -65,7 +64,7 @@ public class KuduAsyncReqRow extends AsyncReqRow { private AsyncKuduScanner.AsyncKuduScannerBuilder scannerBuilder; - public KuduAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public KuduAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new KuduAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @@ -95,9 +94,6 @@ private void connKuDu() throws KuduException { if (null != workerCount) { asyncKuduClientBuilder.workerCount(workerCount); } - if (null != defaultSocketReadTimeoutMs) { - asyncKuduClientBuilder.defaultSocketReadTimeoutMs(defaultSocketReadTimeoutMs); - } if (null != defaultOperationTimeoutMs) { asyncKuduClientBuilder.defaultOperationTimeoutMs(defaultOperationTimeoutMs); diff --git a/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncSideInfo.java b/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncSideInfo.java index de75f23fb..5ace515f7 100644 --- a/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncSideInfo.java +++ b/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncSideInfo.java @@ -2,8 +2,8 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.BaseSideInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; @@ -12,15 +12,15 @@ import java.util.List; -public class KuduAsyncSideInfo extends SideInfo { +public class KuduAsyncSideInfo extends BaseSideInfo { - public KuduAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public KuduAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + public void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo) { } @Override 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 c03c470a7..62d215d87 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,7 +1,7 @@ package com.dtstack.flink.sql.side.kudu.table; -import com.dtstack.flink.sql.table.AbsSideTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractSideTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.math.BigDecimal; @@ -9,9 +9,9 @@ import java.sql.Timestamp; import java.util.Map; -import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; +import static com.dtstack.flink.sql.table.AbstractTableInfo.PARALLELISM_KEY; -public class KuduSideParser extends AbsSideTableParser { +public class KuduSideParser extends AbstractSideTableParser { public static final String KUDU_MASTERS = "kuduMasters"; @@ -52,7 +52,7 @@ public class KuduSideParser extends AbsSideTableParser { @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { KuduSideTableInfo kuduSideTableInfo = new KuduSideTableInfo(); kuduSideTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, kuduSideTableInfo); @@ -107,6 +107,7 @@ public Class dbTypeConvertToJavaType(String fieldType) { return BigDecimal.class; case "binary": return byte[].class; + default: } throw new RuntimeException("不支持 " + fieldType + " 类型"); diff --git a/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideTableInfo.java b/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideTableInfo.java index c527ec268..eb8df5ad9 100644 --- a/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideTableInfo.java +++ b/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideTableInfo.java @@ -1,9 +1,9 @@ package com.dtstack.flink.sql.side.kudu.table; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.google.common.base.Preconditions; -public class KuduSideTableInfo extends SideTableInfo { +public class KuduSideTableInfo extends AbstractSideTableInfo { private static final String CURR_TYPE = "kudu"; diff --git a/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/utils/KuduUtil.java b/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/utils/KuduUtil.java index b750e2d4a..dfc02dee6 100644 --- a/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/utils/KuduUtil.java +++ b/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/utils/KuduUtil.java @@ -191,6 +191,7 @@ public static KuduPredicate buildKuduPredicate(Schema schema, PredicateInfo info return KuduPredicate.newIsNotNullPredicate(column); case "IS NULL": return KuduPredicate.newIsNullPredicate(column); + default: } return null; diff --git a/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduOutputFormat.java b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduOutputFormat.java index 99d19cdef..4543b9618 100644 --- a/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduOutputFormat.java +++ b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduOutputFormat.java @@ -18,13 +18,11 @@ package com.dtstack.flink.sql.sink.kudu; -import com.dtstack.flink.sql.outputformat.DtRichOutputFormat; +import com.dtstack.flink.sql.outputformat.AbstractDtRichOutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; - -import com.dtstack.flink.sql.outputformat.DtRichOutputFormat; import org.apache.kudu.client.AsyncKuduClient; import org.apache.kudu.client.AsyncKuduSession; import org.apache.kudu.client.KuduClient; @@ -44,7 +42,7 @@ * @author gituser * @modify xiuzhu */ -public class KuduOutputFormat extends DtRichOutputFormat { +public class KuduOutputFormat extends AbstractDtRichOutputFormat { private static final long serialVersionUID = 1L; @@ -125,7 +123,7 @@ public void writeRecord(Tuple2 record) throws IOException { Row row = tupleTrans.getField(1); if (row.getArity() != fieldNames.length) { if(outDirtyRecords.getCount() % DIRTY_PRINT_FREQUENCY == 0) { - LOG.error("record insert failed:{}", row.toString()); + LOG.error("record insert failed ..{}", row.toString()); LOG.error("cause by row.getArity() != fieldNames.length"); } outDirtyRecords.inc(); 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 0c90ffe94..4c7c3bea8 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 @@ -3,7 +3,7 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.kudu.table.KuduTableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -39,7 +39,7 @@ public class KuduSink implements RetractStreamTableSink, Serializable, IStr private int parallelism = -1; @Override - public KuduSink genStreamSink(TargetTableInfo targetTableInfo) { + public KuduSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { KuduTableInfo kuduTableInfo = (KuduTableInfo) targetTableInfo; this.kuduMasters = kuduTableInfo.getKuduMasters(); this.tableName = kuduTableInfo.getTableName(); 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 ea61bd325..20302d44f 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,8 +1,8 @@ package com.dtstack.flink.sql.sink.kudu.table; import com.dtstack.flink.sql.sink.kudu.KuduOutputFormat; -import com.dtstack.flink.sql.table.AbsTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.math.BigDecimal; @@ -10,9 +10,9 @@ import java.sql.Timestamp; import java.util.Map; -import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; +import static com.dtstack.flink.sql.table.AbstractTableInfo.PARALLELISM_KEY; -public class KuduSinkParser extends AbsTableParser { +public class KuduSinkParser extends AbstractTableParser { public static final String KUDU_MASTERS = "kuduMasters"; @@ -27,7 +27,7 @@ public class KuduSinkParser extends AbsTableParser { public static final String SOCKET_READ_TIMEOUT_MS = "defaultSocketReadTimeoutMs"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { KuduTableInfo kuduTableInfo = new KuduTableInfo(); kuduTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, kuduTableInfo); @@ -86,6 +86,7 @@ public Class dbTypeConvertToJavaType(String fieldType) { return BigDecimal.class; case "binary": return byte[].class; + default: } throw new RuntimeException("不支持 " + fieldType + " 类型"); diff --git a/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduTableInfo.java b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduTableInfo.java index c086c7298..80e47761b 100644 --- a/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduTableInfo.java +++ b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduTableInfo.java @@ -1,10 +1,10 @@ package com.dtstack.flink.sql.sink.kudu.table; import com.dtstack.flink.sql.sink.kudu.KuduOutputFormat; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import com.google.common.base.Preconditions; -public class KuduTableInfo extends TargetTableInfo { +public class KuduTableInfo extends AbstractTargetTableInfo { private static final String CURR_TYPE = "kudu"; diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 14cd847b5..8f0090db5 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -21,11 +21,11 @@ import com.dtstack.flink.sql.enums.ClusterMode; import com.dtstack.flink.sql.option.Options; import com.dtstack.flink.sql.util.PluginUtil; +import com.esotericsoftware.minlog.Log; import org.apache.commons.io.Charsets; import org.apache.commons.lang.StringUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.MiniClusterClient; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.JobManagerOptions; @@ -89,7 +89,7 @@ public static ClusterClient createYarnSessionClient(Options launcherOptions) { if (StringUtils.isNotBlank(yarnConfDir)) { try { - config.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnConfDir); + config.setString("fs.hdfs.hadoopconf", yarnConfDir); FileSystem.initialize(config); YarnConfiguration yarnConf = YarnConfLoader.getYarnConf(yarnConfDir); @@ -109,7 +109,7 @@ public static ClusterClient createYarnSessionClient(Options launcherOptions) { applicationId = getYarnClusterApplicationId(yarnClient); } - System.out.println("applicationId=" + applicationId.toString()); + Log.info("applicationId={}", applicationId.toString()); if (StringUtils.isEmpty(applicationId.toString())) { throw new RuntimeException("No flink session found on yarn cluster."); @@ -166,7 +166,7 @@ private static ApplicationId getYarnClusterApplicationId(YarnClient yarnClient) private static ApplicationId toApplicationId(String appIdStr) { Iterator it = StringHelper._split(appIdStr).iterator(); - if (!(it.next()).equals("application")) { + if (!"application".equals(it.next())) { throw new IllegalArgumentException("Invalid ApplicationId prefix: " + appIdStr + ". The valid ApplicationId should start with prefix " + "application"); } else { try { diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java index 3de1cdbc5..0e5089bc9 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -21,6 +21,7 @@ import com.dtstack.flink.sql.enums.EPluginLoadMode; import com.dtstack.flink.sql.launcher.YarnConfLoader; import com.dtstack.flink.sql.option.Options; +import com.esotericsoftware.minlog.Log; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.configuration.Configuration; @@ -79,7 +80,7 @@ public void init(String yarnConfDir, Configuration flinkConfig, Properties userC yarnClient.init(yarnConf); yarnClient.start(); - System.out.println("----init yarn success ----"); + Log.info("----init yarn success ----"); } public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(String flinkJarPath, Options launcherOptions, JobGraph jobGraph) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java index 29cc4890a..ca2d5bdd7 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java @@ -73,7 +73,6 @@ public static String submit(Options launcherOptions, JobGraph jobGraph, Configur String flinkJobId = jobGraph.getJobID().toString(); String tips = String.format("deploy per_job with appId: %s, jobId: %s", applicationId, flinkJobId); - System.out.println(tips); LOG.info(tips); return applicationId; diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java index f90cfd973..3373f3667 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java @@ -18,25 +18,17 @@ package com.dtstack.flink.sql.side.mongo; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.table.runtime.types.CRow; -import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; -import org.apache.flink.types.Row; -import org.apache.flink.util.Collector; - -import com.dtstack.flink.sql.side.AllReqRow; +import com.dtstack.flink.sql.side.BaseAllReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; import com.dtstack.flink.sql.side.mongo.utils.MongoUtil; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.mongodb.BasicDBObject; import com.mongodb.MongoClient; -import com.mongodb.MongoClientOptions; -import com.mongodb.MongoCredential; -import com.mongodb.ServerAddress; +import com.mongodb.MongoClientURI; import com.mongodb.client.FindIterable; import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoCursor; @@ -44,13 +36,19 @@ import org.apache.calcite.sql.JoinType; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.flink.table.runtime.types.CRow; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; import org.bson.Document; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.sql.SQLException; import java.sql.Timestamp; -import java.util.ArrayList; import java.util.Calendar; import java.util.List; import java.util.Map; @@ -62,7 +60,7 @@ * * @author xuqianjin */ -public class MongoAllReqRow extends AllReqRow { +public class MongoAllReqRow extends BaseAllReqRow { private static final long serialVersionUID = -675332795591842778L; @@ -78,7 +76,7 @@ public class MongoAllReqRow extends AllReqRow { private AtomicReference>>> cacheRef = new AtomicReference<>(); - public MongoAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public MongoAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new MongoAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @@ -181,34 +179,13 @@ private String buildKey(Map val, List equalFieldList) { return sb.toString(); } - private MongoCollection getConn(String address, String userName, String password, String database, String tableName) { + private MongoCollection getConn(String address, String database, String tableName) { MongoCollection dbCollection; - try { - MongoCredential credential; - String[] servers = StringUtils.split(address, ","); - String host; - Integer port; - String[] hostAndPort; - List lists = new ArrayList<>(); - for (String server : servers) { - hostAndPort = StringUtils.split(server, ":"); - host = hostAndPort[0]; - port = Integer.parseInt(hostAndPort[1]); - lists.add(new ServerAddress(host, port)); - } - if (!StringUtils.isEmpty(userName) || !StringUtils.isEmpty(password)) { - credential = MongoCredential.createCredential(userName, database, password.toCharArray()); - // To connect to mongodb server - mongoClient = new MongoClient(lists, credential, new MongoClientOptions.Builder().build()); - } else { - mongoClient = new MongoClient(lists); - } - db = mongoClient.getDatabase(database); - dbCollection = db.getCollection(tableName, Document.class); - return dbCollection; - } catch (Exception e) { - throw new RuntimeException("[connMongoDB]:" + e.getMessage()); - } + mongoClient = new MongoClient(new MongoClientURI(address)); + db = mongoClient.getDatabase(database); + dbCollection = db.getCollection(tableName, Document.class); + return dbCollection; + } private void loadData(Map>> tmpCache) throws SQLException { @@ -218,8 +195,7 @@ private void loadData(Map>> tmpCache) throws SQ try { for (int i = 0; i < CONN_RETRY_NUM; i++) { try { - dbCollection = getConn(tableInfo.getAddress(), tableInfo.getUserName(), tableInfo.getPassword(), - tableInfo.getDatabase(), tableInfo.getTableName()); + dbCollection = getConn(tableInfo.getAddress(), tableInfo.getDatabase(), tableInfo.getTableName()); break; } catch (Exception e) { if (i == CONN_RETRY_NUM - 1) { diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java index b2d5c2686..4a33f1a69 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java @@ -20,8 +20,8 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.BaseSideInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlNode; import org.apache.commons.collections.CollectionUtils; @@ -36,16 +36,16 @@ * * @author xuqianjin */ -public class MongoAllSideInfo extends SideInfo{ +public class MongoAllSideInfo extends BaseSideInfo { private static final long serialVersionUID = -1696860430075523841L; - public MongoAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public MongoAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + public void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo) { } @Override diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index 42749e935..d85a9990d 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -27,33 +27,35 @@ import org.apache.flink.types.Row; import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.AsyncReqRow; +import com.dtstack.flink.sql.side.BaseAsyncReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; import com.dtstack.flink.sql.side.mongo.utils.MongoUtil; import com.google.common.collect.Lists; import com.mongodb.BasicDBObject; import com.mongodb.Block; -import com.mongodb.MongoCredential; -import com.mongodb.ServerAddress; +import com.mongodb.ConnectionString; import com.mongodb.async.SingleResultCallback; import com.mongodb.async.client.MongoClient; -import com.mongodb.async.client.MongoClientSettings; +import com.mongodb.MongoClientSettings; import com.mongodb.async.client.MongoClients; import com.mongodb.async.client.MongoCollection; import com.mongodb.async.client.MongoDatabase; -import com.mongodb.connection.ClusterSettings; -import com.mongodb.connection.ConnectionPoolSettings; -import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import com.google.common.collect.Lists; +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.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; import org.bson.Document; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.sql.Timestamp; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -66,68 +68,43 @@ * * @author xuqianjin */ -public class MongoAsyncReqRow extends AsyncReqRow { +public class MongoAsyncReqRow extends BaseAsyncReqRow { private static final long serialVersionUID = -1183158242862673706L; private static final Logger LOG = LoggerFactory.getLogger(MongoAsyncReqRow.class); - private final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = 20; - private transient MongoClient mongoClient; private MongoDatabase db; - private MongoSideTableInfo MongoSideTableInfo; + private MongoSideTableInfo mongoSideTableInfo; - public MongoAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public MongoAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new MongoAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @Override public void open(Configuration parameters) throws Exception { super.open(parameters); - MongoSideTableInfo = (MongoSideTableInfo) sideInfo.getSideTableInfo(); - connMongoDB(); + mongoSideTableInfo = (MongoSideTableInfo) sideInfo.getSideTableInfo(); + connMongoDb(); } - public void connMongoDB() throws Exception { - MongoCredential mongoCredential; - String[] servers = StringUtils.split(MongoSideTableInfo.getAddress(), ","); - String host; - Integer port; - String[] hostAndPort; - List lists = new ArrayList<>(); - for (String server : servers) { - hostAndPort = StringUtils.split(server, ":"); - host = hostAndPort[0]; - port = Integer.parseInt(hostAndPort[1]); - lists.add(new ServerAddress(host, port)); - } - ClusterSettings clusterSettings = ClusterSettings.builder().hosts(lists).build(); - ConnectionPoolSettings connectionPoolSettings = ConnectionPoolSettings.builder() - .maxSize(DEFAULT_MAX_DB_CONN_POOL_SIZE) + public void connMongoDb() throws Exception { + String address = mongoSideTableInfo.getAddress(); + ConnectionString connectionString = new ConnectionString(address); + + MongoClientSettings settings = MongoClientSettings.builder() + .applyConnectionString(connectionString) .build(); - if (!StringUtils.isEmpty(MongoSideTableInfo.getUserName()) || !StringUtils.isEmpty(MongoSideTableInfo.getPassword())) { - mongoCredential = MongoCredential.createCredential(MongoSideTableInfo.getUserName(), MongoSideTableInfo.getDatabase(), - MongoSideTableInfo.getPassword().toCharArray()); - MongoClientSettings settings = MongoClientSettings.builder().credential(mongoCredential) - .clusterSettings(clusterSettings) - .connectionPoolSettings(connectionPoolSettings) - .build(); - mongoClient = MongoClients.create(settings); - } else { - MongoClientSettings settings = MongoClientSettings.builder().clusterSettings(clusterSettings) - .connectionPoolSettings(connectionPoolSettings) - .build(); - mongoClient = MongoClients.create(settings); - } - db = mongoClient.getDatabase(MongoSideTableInfo.getDatabase()); + mongoClient = MongoClients.create(settings); + db = mongoClient.getDatabase(mongoSideTableInfo.getDatabase()); } @Override public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exception { CRow inputCopy = new CRow(input.row(), input.change()); - BasicDBObject basicDBObject = new BasicDBObject(); + BasicDBObject basicDbObject = new BasicDBObject(); for (int i = 0; i < sideInfo.getEqualFieldList().size(); i++) { Integer conValIndex = sideInfo.getEqualValIndex().get(i); Object equalObj = inputCopy.row().getField(conValIndex); @@ -135,14 +112,14 @@ public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exce dealMissKey(inputCopy, resultFuture); return; } - basicDBObject.put(sideInfo.getEqualFieldList().get(i), equalObj); + basicDbObject.put(sideInfo.getEqualFieldList().get(i), equalObj); } try { // 填充谓词 sideInfo.getSideTableInfo().getPredicateInfoes().stream().map(info -> { BasicDBObject filterCondition = MongoUtil.buildFilterObject(info); if (null != filterCondition) { - basicDBObject.append(info.getFieldName(), filterCondition); + basicDbObject.append(info.getFieldName(), filterCondition); } return info; }).count(); @@ -150,7 +127,7 @@ public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exce LOG.info("add predicate infoes error ", e); } - String key = buildCacheKey(basicDBObject.values()); + String key = buildCacheKey(basicDbObject.values()); if (openCache()) { CacheObj val = getFromCache(key); if (val != null) { @@ -172,7 +149,7 @@ public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exce } } AtomicInteger atomicInteger = new AtomicInteger(0); - MongoCollection dbCollection = db.getCollection(MongoSideTableInfo.getTableName(), Document.class); + MongoCollection dbCollection = db.getCollection(mongoSideTableInfo.getTableName(), Document.class); List cacheContent = Lists.newArrayList(); Block printDocumentBlock = new Block() { @Override @@ -198,7 +175,7 @@ public void onResult(final Void result, final Throwable t) { } } }; - dbCollection.find(basicDBObject).forEach(printDocumentBlock, callbackWhenFinished); + dbCollection.find(basicDbObject).forEach(printDocumentBlock, callbackWhenFinished); } @Override diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java index de81ed1f1..19534d4e1 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java @@ -20,16 +20,13 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; -import com.dtstack.flink.sql.util.ParseUtils; +import com.dtstack.flink.sql.side.BaseSideInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import com.google.common.collect.Lists; import java.util.List; @@ -39,16 +36,16 @@ * * @author xuqianjin */ -public class MongoAsyncSideInfo extends SideInfo { +public class MongoAsyncSideInfo extends BaseSideInfo { private static final long serialVersionUID = -3694857194254465989L; - public MongoAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public MongoAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + public void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo) { } diff --git a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideParser.java b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideParser.java index 5edc12419..4fe1ebee1 100644 --- a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideParser.java +++ b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideParser.java @@ -19,15 +19,13 @@ package com.dtstack.flink.sql.side.mongo.table; -import com.dtstack.flink.sql.table.AbsSideTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractSideTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; +import static com.dtstack.flink.sql.table.AbstractTableInfo.PARALLELISM_KEY; /** * Reason: @@ -35,7 +33,7 @@ * * @author xuqianjin */ -public class MongoSideParser extends AbsSideTableParser { +public class MongoSideParser extends AbstractSideTableParser { public static final String ADDRESS_KEY = "address"; @@ -48,7 +46,7 @@ public class MongoSideParser extends AbsSideTableParser { public static final String DATABASE_KEY = "database"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { MongoSideTableInfo mongoSideTableInfo = new MongoSideTableInfo(); mongoSideTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, mongoSideTableInfo); diff --git a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java index a5c834469..c83de7bbc 100644 --- a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java +++ b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.side.mongo.table; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.google.common.base.Preconditions; /** @@ -28,7 +28,7 @@ * * @author xuqianjin */ -public class MongoSideTableInfo extends SideTableInfo { +public class MongoSideTableInfo extends AbstractSideTableInfo { private static final long serialVersionUID = -1L; diff --git a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/utils/MongoUtil.java b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/utils/MongoUtil.java index 2f1a2092d..cbcba8f97 100644 --- a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/utils/MongoUtil.java +++ b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/utils/MongoUtil.java @@ -55,6 +55,7 @@ public static BasicDBObject buildFilterObject(PredicateInfo info) { return new BasicDBObject("$exists", true); case "IS NULL": return new BasicDBObject("$exists", false); + default: } return null; } diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java index 337fbad59..3f1f9f7e2 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java @@ -19,20 +19,16 @@ package com.dtstack.flink.sql.sink.mongo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.types.Row; - -import com.dtstack.flink.sql.outputformat.DtRichOutputFormat; +import com.dtstack.flink.sql.outputformat.AbstractDtRichOutputFormat; import com.mongodb.MongoClient; -import com.mongodb.MongoClientOptions; -import com.mongodb.MongoCredential; -import com.mongodb.ServerAddress; +import com.mongodb.MongoClientURI; import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoDatabase; import com.mongodb.client.result.UpdateResult; -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.configuration.Configuration; +import org.apache.flink.types.Row; import org.bson.Document; import org.bson.types.ObjectId; import org.slf4j.Logger; @@ -48,7 +44,7 @@ * * @author xuqianjin */ -public class MongoOutputFormat extends DtRichOutputFormat { +public class MongoOutputFormat extends AbstractDtRichOutputFormat { private static final Logger LOG = LoggerFactory.getLogger(MongoOutputFormat.class); private String address; @@ -125,30 +121,8 @@ public void close() { } private void establishConnection() { - try { - MongoCredential credential; - String[] servers = StringUtils.split(address, ","); - String host; - Integer port; - String[] hostAndPort; - List lists = new ArrayList<>(); - for (String server : servers) { - hostAndPort = StringUtils.split(server, ":"); - host = hostAndPort[0]; - port = Integer.parseInt(hostAndPort[1]); - lists.add(new ServerAddress(host, port)); - } - if (!StringUtils.isEmpty(userName) || !StringUtils.isEmpty(password)) { - credential = MongoCredential.createCredential(userName, database, password.toCharArray()); - // To connect to mongodb server - mongoClient = new MongoClient(lists, credential, new MongoClientOptions.Builder().build()); - } else { - mongoClient = new MongoClient(lists); - } - db = mongoClient.getDatabase(database); - } catch (Exception e) { - throw new IllegalArgumentException("[connMongoDB]:" + e.getMessage()); - } + mongoClient = new MongoClient(new MongoClientURI(address)); + db = mongoClient.getDatabase(database); } private MongoOutputFormat() { diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoSink.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoSink.java index 4e28d8fd2..3f34055ec 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoSink.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoSink.java @@ -21,7 +21,7 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.mongo.table.MongoTableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -54,7 +54,7 @@ public MongoSink() { } @Override - public MongoSink genStreamSink(TargetTableInfo targetTableInfo) { + public MongoSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { MongoTableInfo mongoTableInfo = (MongoTableInfo) targetTableInfo; this.address = mongoTableInfo.getAddress(); this.tableName = mongoTableInfo.getTableName(); diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java index d4810fb6d..f093b70aa 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java @@ -19,13 +19,13 @@ package com.dtstack.flink.sql.sink.mongo.table; -import com.dtstack.flink.sql.table.AbsTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; -import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; +import static com.dtstack.flink.sql.table.AbstractTableInfo.PARALLELISM_KEY; /** * Reason: @@ -34,7 +34,7 @@ * @author xuqianjin */ -public class MongoSinkParser extends AbsTableParser { +public class MongoSinkParser extends AbstractTableParser { private static final String CURR_TYPE = "mongo"; @@ -49,7 +49,7 @@ public class MongoSinkParser extends AbsTableParser { public static final String DATABASE_KEY = "database"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { MongoTableInfo mongoTableInfo = new MongoTableInfo(); mongoTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, mongoTableInfo); diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java index 02a96d6bb..c9d94dfe6 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.sink.mongo.table; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import com.google.common.base.Preconditions; /** @@ -30,7 +30,7 @@ */ -public class MongoTableInfo extends TargetTableInfo { +public class MongoTableInfo extends AbstractTargetTableInfo { private static final String CURR_TYPE = "mongo"; diff --git a/mongo/mongo-sink/src/test/java/com/dtstack/flinkx/AppTest.java b/mongo/mongo-sink/src/test/java/com/dtstack/flinkx/AppTest.java deleted file mode 100644 index 33a0233ac..000000000 --- a/mongo/mongo-sink/src/test/java/com/dtstack/flinkx/AppTest.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - - -package com.dtstack.flinkx; - -import junit.framework.Test; -import junit.framework.TestCase; -import junit.framework.TestSuite; - -/** - * Unit test for simple App. - */ -public class AppTest - extends TestCase -{ - /** - * Create the test case - * - * @param testName name of the test case - */ - public AppTest( String testName ) - { - super( testName ); - } - - /** - * @return the suite of tests being tested - */ - public static Test suite() - { - return new TestSuite( AppTest.class ); - } - - /** - * Rigourous Test :-) - */ - public void testApp() - { - assertTrue( true ); - } -} diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java index b6b7f45cb..eb294e64e 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -20,8 +20,8 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.AbstractRdbAllReqRow; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.flink.api.java.typeutils.RowTypeInfo; import com.google.common.collect.Maps; @@ -41,7 +41,7 @@ * @author xuchao */ -public class MysqlAllReqRow extends RdbAllReqRow { +public class MysqlAllReqRow extends AbstractRdbAllReqRow { private static final long serialVersionUID = 2098635140857937717L; @@ -49,18 +49,18 @@ public class MysqlAllReqRow extends RdbAllReqRow { private static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; - public MysqlAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public MysqlAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new MysqlAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @Override - public Connection getConn(String dbURL, String userName, String password) { + public Connection getConn(String dbUrl, String userName, String password) { try { Class.forName(MYSQL_DRIVER); //add param useCursorFetch=true Map addParams = Maps.newHashMap(); addParams.put("useCursorFetch", "true"); - String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); + String targetDbUrl = DtStringUtil.addJdbcParam(dbUrl, addParams, true); return DriverManager.getConnection(targetDbUrl, userName, password); } catch (Exception e) { LOG.error("", e); diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java index 07b06ba38..503dbf9a3 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java @@ -20,7 +20,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -35,7 +35,7 @@ */ public class MysqlAllSideInfo extends RdbAllSideInfo { - public MysqlAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public MysqlAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index de673aea7..d52f1918f 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -21,7 +21,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncReqRow; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import io.vertx.core.Vertx; @@ -30,8 +30,6 @@ import io.vertx.ext.jdbc.JDBCClient; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.List; @@ -44,12 +42,9 @@ */ public class MysqlAsyncReqRow extends RdbAsyncReqRow { - - private static final Logger LOG = LoggerFactory.getLogger(MysqlAsyncReqRow.class); - private final static String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; - public MysqlAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public MysqlAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new MysqlAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @@ -76,7 +71,7 @@ public void open(Configuration parameters) throws Exception { vo.setWorkerPoolSize(rdbSideTableInfo.getAsyncPoolSize()); vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); - setRdbSQLClient(JDBCClient.createNonShared(vertx, mysqlClientConfig)); + setRdbSqlClient(JDBCClient.createNonShared(vertx, mysqlClientConfig)); } } diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java index 5d6ed370d..9555897aa 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java @@ -20,7 +20,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -36,7 +36,7 @@ public class MysqlAsyncSideInfo extends RdbAsyncSideInfo { - public MysqlAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public MysqlAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java b/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java index 40f68e7e4..b0a38e344 100644 --- a/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java +++ b/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.side.mysql.table; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import java.util.Map; @@ -37,8 +37,8 @@ public class MysqlSideParser extends RdbSideParser { private static final String CURR_TYPE = "mysql"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo mysqlTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + AbstractTableInfo mysqlTableInfo = super.getTableInfo(tableName, fieldsInfo, props); mysqlTableInfo.setType(CURR_TYPE); return mysqlTableInfo; } diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java index 8c1146ede..0a1749f04 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java @@ -21,7 +21,7 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; -import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.AbstractRdbSink; import com.dtstack.flink.sql.sink.rdb.JDBCOptions; import com.dtstack.flink.sql.sink.rdb.format.JDBCUpsertOutputFormat; @@ -33,7 +33,7 @@ * @author xuchao */ -public class MysqlSink extends RdbSink implements IStreamSinkGener { +public class MysqlSink extends AbstractRdbSink implements IStreamSinkGener { public MysqlSink() { super(new MySQLDialect()); @@ -42,7 +42,7 @@ public MysqlSink() { @Override public JDBCUpsertOutputFormat getOutputFormat() { JDBCOptions jdbcOptions = JDBCOptions.builder() - .setDBUrl(dbURL) + .setDbUrl(dbUrl) .setDialect(jdbcDialect) .setUsername(userName) .setPassword(password) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java index 2247eb8cc..49105a7a8 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.sink.mysql.table; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import java.util.Map; @@ -36,8 +36,8 @@ public class MysqlSinkParser extends RdbSinkParser { private static final String CURR_TYPE = "mysql"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo mysqlTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + AbstractTableInfo mysqlTableInfo = super.getTableInfo(tableName, fieldsInfo, props); mysqlTableInfo.setType(CURR_TYPE); return mysqlTableInfo; } diff --git a/mysql/mysql-sink/src/test/java/com/dtstack/flinkx/AppTest.java b/mysql/mysql-sink/src/test/java/com/dtstack/flinkx/AppTest.java deleted file mode 100644 index 33a0233ac..000000000 --- a/mysql/mysql-sink/src/test/java/com/dtstack/flinkx/AppTest.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - - -package com.dtstack.flinkx; - -import junit.framework.Test; -import junit.framework.TestCase; -import junit.framework.TestSuite; - -/** - * Unit test for simple App. - */ -public class AppTest - extends TestCase -{ - /** - * Create the test case - * - * @param testName name of the test case - */ - public AppTest( String testName ) - { - super( testName ); - } - - /** - * @return the suite of tests being tested - */ - public static Test suite() - { - return new TestSuite( AppTest.class ); - } - - /** - * Rigourous Test :-) - */ - public void testApp() - { - assertTrue( true ); - } -} diff --git a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java index 18d9ba045..3be6687ec 100644 --- a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java +++ b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java @@ -20,8 +20,8 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.AbstractRdbAllReqRow; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.flink.api.java.typeutils.RowTypeInfo; import com.google.common.collect.Maps; @@ -36,22 +36,22 @@ /** * side operator with cache for all(period reload) */ -public class OracleAllReqRow extends RdbAllReqRow { +public class OracleAllReqRow extends AbstractRdbAllReqRow { private static final Logger LOG = LoggerFactory.getLogger(OracleAllReqRow.class); private static final String ORACLE_DRIVER = "oracle.jdbc.driver.OracleDriver"; - public OracleAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public OracleAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new OracleAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @Override - public Connection getConn(String dbURL, String userName, String password) { + public Connection getConn(String dbUrl, String userName, String password) { try { Class.forName(ORACLE_DRIVER); Map addParams = Maps.newHashMap(); - String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); + String targetDbUrl = DtStringUtil.addJdbcParam(dbUrl, addParams, true); return DriverManager.getConnection(targetDbUrl, userName, password); } catch (Exception e) { LOG.error("", e); diff --git a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java index 361366929..342533681 100644 --- a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java +++ b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java @@ -19,18 +19,17 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; -import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; import java.util.List; public class OracleAllSideInfo extends RdbAllSideInfo { - public OracleAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public OracleAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java index 20904a061..938fd870c 100644 --- a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java @@ -21,7 +21,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncReqRow; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import io.vertx.core.Vertx; @@ -30,19 +30,14 @@ import io.vertx.ext.jdbc.JDBCClient; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.List; public class OracleAsyncReqRow extends RdbAsyncReqRow { - - private static final Logger LOG = LoggerFactory.getLogger(OracleAsyncReqRow.class); - private static final String ORACLE_DRIVER = "oracle.jdbc.driver.OracleDriver"; - public OracleAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public OracleAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new OracleAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @@ -68,6 +63,6 @@ public void open(Configuration parameters) throws Exception { vo.setWorkerPoolSize(rdbSideTableInfo.getAsyncPoolSize()); vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); - setRdbSQLClient(JDBCClient.createNonShared(vertx, oracleClientConfig)); + setRdbSqlClient(JDBCClient.createNonShared(vertx, oracleClientConfig)); } } diff --git a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java index 8e801970f..658020b1c 100644 --- a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java @@ -20,19 +20,20 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; + import java.util.List; public class OracleAsyncSideInfo extends RdbAsyncSideInfo { - public OracleAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public OracleAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @@ -55,7 +56,7 @@ public String wrapperPlaceholder(String fieldName) { String rpadFormat = "rpad(?, %d, ' ')"; if (StringUtils.contains(type.toLowerCase(), "char")) { - TableInfo.FieldExtraInfo fieldExtraInfo = sideTableInfo.getFieldExtraInfoList().get(pos); + AbstractTableInfo.FieldExtraInfo fieldExtraInfo = sideTableInfo.getFieldExtraInfoList().get(pos); int charLength = fieldExtraInfo == null ? 0 : fieldExtraInfo.getLength(); if (charLength > 0) { return String.format(rpadFormat, charLength); diff --git a/oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java b/oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java index f9124bd10..897b77d30 100644 --- a/oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java +++ b/oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.side.oracle.table; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import java.util.Map; @@ -27,8 +27,8 @@ public class OracleSideParser extends RdbSideParser { private static final String CURR_TYPE = "oracle"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo oracleTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + AbstractTableInfo oracleTableInfo = super.getTableInfo(tableName, fieldsInfo, props); oracleTableInfo.setType(CURR_TYPE); return oracleTableInfo; } diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleDialect.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleDialect.java index a687aa012..3a320658c 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 @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.sink.oracle; import com.dtstack.flink.sql.sink.rdb.dialect.JDBCDialect; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.commons.lang3.StringUtils; @@ -41,7 +41,7 @@ public class OracleDialect implements JDBCDialect { private List fieldList; private List fieldTypeList; - private List fieldExtraInfoList; + private List fieldExtraInfoList; @Override public boolean canHandle(String url) { @@ -133,7 +133,7 @@ public String wrapperPlaceholder(String fieldName) { String type = fieldTypeList.get(pos); if (StringUtils.contains(type.toLowerCase(), DEAL_CHAR_KEY)) { - TableInfo.FieldExtraInfo fieldExtraInfo = fieldExtraInfoList.get(pos); + AbstractTableInfo.FieldExtraInfo fieldExtraInfo = fieldExtraInfoList.get(pos); int charLength = fieldExtraInfo == null ? 0 : fieldExtraInfo.getLength(); if (charLength > 0) { return String.format(RPAD_FORMAT, charLength); @@ -151,7 +151,7 @@ public void setFieldTypeList(List fieldTypeList) { this.fieldTypeList = fieldTypeList; } - public void setFieldExtraInfoList(List fieldExtraInfoList) { + public void setFieldExtraInfoList(List fieldExtraInfoList) { this.fieldExtraInfoList = fieldExtraInfoList; } } diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index 7a37aa6cd..f916b55f2 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -19,7 +19,7 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.JDBCOptions; -import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.AbstractRdbSink; import com.dtstack.flink.sql.sink.rdb.format.JDBCUpsertOutputFormat; /** @@ -29,7 +29,7 @@ * * @author maqi */ -public class OracleSink extends RdbSink implements IStreamSinkGener { +public class OracleSink extends AbstractRdbSink implements IStreamSinkGener { public OracleSink() { super(new OracleDialect()); @@ -42,7 +42,7 @@ public JDBCUpsertOutputFormat getOutputFormat() { ((OracleDialect) jdbcDialect).setFieldExtraInfoList(fieldExtraInfoList); JDBCOptions jdbcOptions = JDBCOptions.builder() - .setDBUrl(dbURL) + .setDbUrl(dbUrl) .setDialect(jdbcDialect) .setUsername(userName) .setPassword(password) diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java index aff096bd3..1732c8cc2 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.sink.oracle.table; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import java.util.Map; @@ -33,8 +33,8 @@ public class OracleSinkParser extends RdbSinkParser { private static final String CURR_TYPE = "oracle"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo oracleTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + AbstractTableInfo oracleTableInfo = super.getTableInfo(tableName, fieldsInfo, props); oracleTableInfo.setType(CURR_TYPE); return oracleTableInfo; } diff --git a/polardb/polardb-side/polardb-all-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAllReqRow.java b/polardb/polardb-side/polardb-all-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAllReqRow.java index f9e0950dd..5fbf0824b 100644 --- a/polardb/polardb-side/polardb-all-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAllReqRow.java +++ b/polardb/polardb-side/polardb-all-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAllReqRow.java @@ -1,9 +1,26 @@ +/* + * 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.polardb; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.AbstractRdbAllReqRow; import com.dtstack.flink.sql.util.DtStringUtil; import com.google.common.collect.Maps; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -15,7 +32,12 @@ import java.util.List; import java.util.Map; -public class PolardbAllReqRow extends RdbAllReqRow { +/** + * Date: 2019/12/20 + * Company: www.dtstack.com + * @author yinxi + */ +public class PolardbAllReqRow extends AbstractRdbAllReqRow { private static final long serialVersionUID = 2098635140857937717L; @@ -23,18 +45,18 @@ public class PolardbAllReqRow extends RdbAllReqRow { private static final String POLARDB_DRIVER = "com.mysql.cj.jdbc.Driver"; - public PolardbAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public PolardbAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new PolardbAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @Override - public Connection getConn(String dbURL, String userName, String password) { + public Connection getConn(String dbUrl, String userName, String password) { try { Class.forName(POLARDB_DRIVER); //add param useCursorFetch=true Map addParams = Maps.newHashMap(); addParams.put("useCursorFetch", "true"); - String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); + String targetDbUrl = DtStringUtil.addJdbcParam(dbUrl, addParams, true); return DriverManager.getConnection(targetDbUrl, userName, password); } catch (Exception e) { LOG.error("", e); diff --git a/polardb/polardb-side/polardb-all-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAllSideInfo.java b/polardb/polardb-side/polardb-all-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAllSideInfo.java index 822cbaa70..c46c9dfd6 100644 --- a/polardb/polardb-side/polardb-all-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAllSideInfo.java +++ b/polardb/polardb-side/polardb-all-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAllSideInfo.java @@ -1,15 +1,37 @@ +/* + * 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.polardb; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import java.util.List; +/** + * Date: 2019/12/20 + * Company: www.dtstack.com + * @author yinxi + */ public class PolardbAllSideInfo extends RdbAllSideInfo { - public PolardbAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public PolardbAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/polardb/polardb-side/polardb-async-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAsyncReqRow.java b/polardb/polardb-side/polardb-async-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAsyncReqRow.java index 52e57e262..13d25b111 100644 --- a/polardb/polardb-side/polardb-async-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAsyncReqRow.java +++ b/polardb/polardb-side/polardb-async-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAsyncReqRow.java @@ -1,8 +1,25 @@ +/* + * 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.polardb; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncReqRow; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import io.vertx.core.Vertx; @@ -16,13 +33,18 @@ import java.util.List; +/** + * Date: 2019/12/20 + * Company: www.dtstack.com + * @author yinxi + */ public class PolardbAsyncReqRow extends RdbAsyncReqRow { private static final Logger LOG = LoggerFactory.getLogger(PolardbAsyncReqRow.class); private final static String POLARDB_DRIVER = "com.mysql.cj.jdbc.Driver"; - public PolardbAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public PolardbAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new PolardbAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @@ -48,6 +70,6 @@ public void open(Configuration parameters) throws Exception { vo.setWorkerPoolSize(rdbSideTableInfo.getAsyncPoolSize()); vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); - setRdbSQLClient(JDBCClient.createNonShared(vertx, mysqlClientConfig)); + setRdbSqlClient(JDBCClient.createNonShared(vertx, mysqlClientConfig)); } } diff --git a/polardb/polardb-side/polardb-async-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAsyncSideInfo.java b/polardb/polardb-side/polardb-async-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAsyncSideInfo.java index bad71375f..15237956b 100644 --- a/polardb/polardb-side/polardb-async-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAsyncSideInfo.java +++ b/polardb/polardb-side/polardb-async-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAsyncSideInfo.java @@ -1,16 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flink.sql.side.polardb; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import java.util.List; +/** + * Date: 2019/12/20 + * Company: www.dtstack.com + * @author yinxi + */ public class PolardbAsyncSideInfo extends RdbAsyncSideInfo { - public PolardbAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public PolardbAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/polardb/polardb-side/polardb-side-core/src/main/java/com/dtstack/flink/sql/side/polardb/table/PolardbSideParser.java b/polardb/polardb-side/polardb-side-core/src/main/java/com/dtstack/flink/sql/side/polardb/table/PolardbSideParser.java index 348b3aed9..71aad7cbb 100644 --- a/polardb/polardb-side/polardb-side-core/src/main/java/com/dtstack/flink/sql/side/polardb/table/PolardbSideParser.java +++ b/polardb/polardb-side/polardb-side-core/src/main/java/com/dtstack/flink/sql/side/polardb/table/PolardbSideParser.java @@ -1,16 +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.side.polardb.table; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import java.util.Map; +/** + * Date: 2019/12/20 + * Company: www.dtstack.com + * @author yinxi + */ public class PolardbSideParser extends RdbSideParser { private static final String CURR_TYPE = "polardb"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo mysqlTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + AbstractTableInfo mysqlTableInfo = super.getTableInfo(tableName, fieldsInfo, props); mysqlTableInfo.setType(CURR_TYPE); return mysqlTableInfo; } diff --git a/polardb/polardb-sink/src/main/java/com/dtstack/flink/sql/sink/polardb/PolardbSink.java b/polardb/polardb-sink/src/main/java/com/dtstack/flink/sql/sink/polardb/PolardbSink.java index 44d558140..c69f31e8f 100644 --- a/polardb/polardb-sink/src/main/java/com/dtstack/flink/sql/sink/polardb/PolardbSink.java +++ b/polardb/polardb-sink/src/main/java/com/dtstack/flink/sql/sink/polardb/PolardbSink.java @@ -1,11 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package com.dtstack.flink.sql.sink.polardb; import com.dtstack.flink.sql.sink.rdb.JDBCOptions; -import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.AbstractRdbSink; import com.dtstack.flink.sql.sink.rdb.format.JDBCUpsertOutputFormat; - -public class PolardbSink extends RdbSink { +/** + * Date: 2019/12/20 + * Company: www.dtstack.com + * @author yinxi + */ +public class PolardbSink extends AbstractRdbSink { public PolardbSink() { super(new PolardbDialect()); } @@ -13,7 +35,7 @@ public PolardbSink() { @Override public JDBCUpsertOutputFormat getOutputFormat() { JDBCOptions jdbcOptions = JDBCOptions.builder() - .setDBUrl(dbURL) + .setDbUrl(dbUrl) .setDialect(jdbcDialect) .setUsername(userName) .setPassword(password) diff --git a/polardb/polardb-sink/src/main/java/com/dtstack/flink/sql/sink/polardb/table/PolardbSinkParser.java b/polardb/polardb-sink/src/main/java/com/dtstack/flink/sql/sink/polardb/table/PolardbSinkParser.java index 68eead938..b4f02665e 100644 --- a/polardb/polardb-sink/src/main/java/com/dtstack/flink/sql/sink/polardb/table/PolardbSinkParser.java +++ b/polardb/polardb-sink/src/main/java/com/dtstack/flink/sql/sink/polardb/table/PolardbSinkParser.java @@ -1,16 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flink.sql.sink.polardb.table; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import java.util.Map; +/** + * Date: 2019/12/20 + * Company: www.dtstack.com + * @author yinxi + */ public class PolardbSinkParser extends RdbSinkParser { private static final String CURR_TYPE = "polardb"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo polardbTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + AbstractTableInfo polardbTableInfo = super.getTableInfo(tableName, fieldsInfo, props); polardbTableInfo.setType(CURR_TYPE); return polardbTableInfo; } diff --git a/pom.xml b/pom.xml index 2794f9004..de963240c 100644 --- a/pom.xml +++ b/pom.xml @@ -18,6 +18,7 @@ mysql hbase elasticsearch5 + elasticsearch6 mongo redis5 launcher @@ -33,7 +34,6 @@ impala db2 polardb - elasticsearch6 @@ -54,6 +54,20 @@ + + org.jacoco + jacoco-maven-plugin + 0.7.8 + + + + prepare-agent + report + + + + + pl.project13.maven git-commit-id-plugin diff --git a/postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllReqRow.java b/postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllReqRow.java index 6d68cfdca..0ec8f96ce 100644 --- a/postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllReqRow.java +++ b/postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllReqRow.java @@ -20,8 +20,8 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.AbstractRdbAllReqRow; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; @@ -41,7 +41,7 @@ * @author tcm */ -public class PostgresqlAllReqRow extends RdbAllReqRow { +public class PostgresqlAllReqRow extends AbstractRdbAllReqRow { private static final long serialVersionUID = 2098635140857937717L; @@ -49,18 +49,18 @@ public class PostgresqlAllReqRow extends RdbAllReqRow { private static final String POSTGRESQL_DRIVER = "org.postgresql.Driver"; - public PostgresqlAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public PostgresqlAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new PostgresqlAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @Override - public Connection getConn(String dbURL, String userName, String password) { + public Connection getConn(String dbUrl, String userName, String password) { try { Class.forName(POSTGRESQL_DRIVER); //add param useCursorFetch=true Map addParams = Maps.newHashMap(); addParams.put("useCursorFetch", "true"); - String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); + String targetDbUrl = DtStringUtil.addJdbcParam(dbUrl, addParams, true); return DriverManager.getConnection(targetDbUrl, userName, password); } catch (Exception e) { LOG.error("", e); diff --git a/postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllSideInfo.java b/postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllSideInfo.java index d383ee46d..a4a52eabc 100644 --- a/postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllSideInfo.java +++ b/postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllSideInfo.java @@ -20,7 +20,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -35,7 +35,7 @@ */ public class PostgresqlAllSideInfo extends RdbAllSideInfo { - public PostgresqlAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public PostgresqlAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } } diff --git a/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncReqRow.java b/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncReqRow.java index bed10f206..42aa39a51 100644 --- a/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncReqRow.java +++ b/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncReqRow.java @@ -21,7 +21,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncReqRow; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import io.vertx.core.Vertx; @@ -48,7 +48,7 @@ public class PostgresqlAsyncReqRow extends RdbAsyncReqRow { private final static String POSTGRESQL_DRIVER = "org.postgresql.Driver"; - public PostgresqlAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public PostgresqlAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new PostgresqlAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @@ -71,7 +71,7 @@ public void open(Configuration parameters) throws Exception { vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); vo.setWorkerPoolSize(rdbSideTableInfo.getAsyncPoolSize()); Vertx vertx = Vertx.vertx(vo); - setRdbSQLClient(JDBCClient.createNonShared(vertx, pgClientConfig)); + setRdbSqlClient(JDBCClient.createNonShared(vertx, pgClientConfig)); } } diff --git a/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncSideInfo.java b/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncSideInfo.java index 1d89f4894..f19a488bc 100644 --- a/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncSideInfo.java +++ b/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncSideInfo.java @@ -20,7 +20,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -36,7 +36,7 @@ public class PostgresqlAsyncSideInfo extends RdbAsyncSideInfo { - public PostgresqlAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public PostgresqlAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } } diff --git a/postgresql/postgresql-side/postgresql-side-core/src/main/java/com/dtstack/flink/sql/side/postgresql/table/PostgresqlSideParser.java b/postgresql/postgresql-side/postgresql-side-core/src/main/java/com/dtstack/flink/sql/side/postgresql/table/PostgresqlSideParser.java index faee2c704..be0c5ea8b 100644 --- a/postgresql/postgresql-side/postgresql-side-core/src/main/java/com/dtstack/flink/sql/side/postgresql/table/PostgresqlSideParser.java +++ b/postgresql/postgresql-side/postgresql-side-core/src/main/java/com/dtstack/flink/sql/side/postgresql/table/PostgresqlSideParser.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.side.postgresql.table; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import java.util.Map; @@ -37,8 +37,8 @@ public class PostgresqlSideParser extends RdbSideParser { private static final String CURR_TYPE = "postgresql"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo pgTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + AbstractTableInfo pgTableInfo = super.getTableInfo(tableName, fieldsInfo, props); pgTableInfo.setType(CURR_TYPE); return pgTableInfo; } diff --git a/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlSink.java b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlSink.java index cf45a6bd9..bb731152c 100644 --- a/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlSink.java +++ b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlSink.java @@ -22,13 +22,13 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.JDBCOptions; -import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.AbstractRdbSink; import com.dtstack.flink.sql.sink.rdb.format.JDBCUpsertOutputFormat; /** * @author maqi */ -public class PostgresqlSink extends RdbSink implements IStreamSinkGener { +public class PostgresqlSink extends AbstractRdbSink implements IStreamSinkGener { public PostgresqlSink() { super(new PostgresqlDialect()); } @@ -36,7 +36,7 @@ public PostgresqlSink() { @Override public JDBCUpsertOutputFormat getOutputFormat() { JDBCOptions jdbcOptions = JDBCOptions.builder() - .setDBUrl(dbURL) + .setDbUrl(dbUrl) .setDialect(jdbcDialect) .setUsername(userName) .setPassword(password) diff --git a/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlSinkParser.java b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlSinkParser.java index 5943b5e52..e94cfc8a0 100644 --- a/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlSinkParser.java +++ b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlSinkParser.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.sink.postgresql.table; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import java.util.Map; @@ -35,8 +35,8 @@ public class PostgresqlSinkParser extends RdbSinkParser { private static final String CURR_TYPE = "postgresql"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo pgTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + AbstractTableInfo pgTableInfo = super.getTableInfo(tableName, fieldsInfo, props); pgTableInfo.setType(CURR_TYPE); return pgTableInfo; } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/AbstractRdbAllReqRow.java similarity index 96% rename from rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java rename to rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/AbstractRdbAllReqRow.java index c3220968d..bb88c7787 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/AbstractRdbAllReqRow.java @@ -25,8 +25,9 @@ import org.apache.flink.types.Row; import org.apache.flink.util.Collector; -import com.dtstack.flink.sql.side.AllReqRow; -import com.dtstack.flink.sql.side.SideInfo; + +import com.dtstack.flink.sql.side.BaseAllReqRow; +import com.dtstack.flink.sql.side.BaseSideInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.side.rdb.util.SwitchUtil; import com.google.common.collect.Lists; @@ -57,11 +58,11 @@ * @author maqi */ -public abstract class RdbAllReqRow extends AllReqRow { +public abstract class AbstractRdbAllReqRow extends BaseAllReqRow { private static final long serialVersionUID = 2098635140857937718L; - private static final Logger LOG = LoggerFactory.getLogger(RdbAllReqRow.class); + private static final Logger LOG = LoggerFactory.getLogger(AbstractRdbAllReqRow.class); private static final int CONN_RETRY_NUM = 3; @@ -69,7 +70,7 @@ public abstract class RdbAllReqRow extends AllReqRow { private AtomicReference>>> cacheRef = new AtomicReference<>(); - public RdbAllReqRow(SideInfo sideInfo) { + public AbstractRdbAllReqRow(BaseSideInfo sideInfo) { super(sideInfo); } @@ -91,6 +92,7 @@ protected void initCache() throws SQLException { protected void reloadCache() { //reload cacheRef and replace to old cacheRef Map>> newCache = Maps.newConcurrentMap(); + cacheRef.set(newCache); try { loadData(newCache); } catch (SQLException e) { 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 3b7444b77..143017133 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 @@ -23,14 +23,16 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.PredicateInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.BaseSideInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; import com.google.common.collect.Lists; import org.apache.calcite.sql.SqlNode; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Arrays; import java.util.List; @@ -44,20 +46,21 @@ * @author maqi */ -public class RdbAllSideInfo extends SideInfo { +public class RdbAllSideInfo extends BaseSideInfo { private static final long serialVersionUID = -5858335638589472159L; + private static final Logger LOG = LoggerFactory.getLogger(RdbAllSideInfo.class.getSimpleName()); - public RdbAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public RdbAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + public void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo) { RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideTableInfo; sqlCondition = getSelectFromStatement(getTableName(rdbSideTableInfo), Arrays.asList(StringUtils.split(sideSelectFields, ",")), sideTableInfo.getPredicateInfoes()); - System.out.println("--------dimension sql query-------\n" + sqlCondition); + LOG.info("--------dimension sql query-------\n{}" + sqlCondition); } public String getAdditionalWhereClause() { 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 971285e52..9af6d02fd 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 @@ -20,7 +20,9 @@ package com.dtstack.flink.sql.side.rdb.async; import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.BaseAsyncReqRow; +import com.dtstack.flink.sql.side.CacheMissVal; +import com.dtstack.flink.sql.side.BaseSideInfo; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.side.rdb.util.SwitchUtil; @@ -30,6 +32,7 @@ import io.vertx.ext.sql.SQLClient; import io.vertx.ext.sql.SQLConnection; import com.google.common.collect.Lists; +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.table.typeutils.TimeIndicatorTypeInfo; @@ -50,7 +53,7 @@ * @author maqi */ -public class RdbAsyncReqRow extends AsyncReqRow { +public class RdbAsyncReqRow extends BaseAsyncReqRow { private static final long serialVersionUID = 2098635244857937720L; @@ -72,20 +75,27 @@ public class RdbAsyncReqRow extends AsyncReqRow { public final static String PREFERRED_TEST_QUERY_SQL = "select 1 from dual"; - private transient SQLClient rdbSQLClient; + private transient SQLClient rdbSqlClient; - public RdbAsyncReqRow(SideInfo sideInfo) { + public RdbAsyncReqRow(BaseSideInfo sideInfo) { super(sideInfo); init(sideInfo); } - protected void init(SideInfo sideInfo) { + protected void init(BaseSideInfo sideInfo) { RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); int defaultAsyncPoolSize = Math.min(MAX_DB_CONN_POOL_SIZE_LIMIT, DEFAULT_DB_CONN_POOL_SIZE); int rdbPoolSize = rdbSideTableInfo.getAsyncPoolSize() > 0 ? rdbSideTableInfo.getAsyncPoolSize() : defaultAsyncPoolSize; rdbSideTableInfo.setAsyncPoolSize(rdbPoolSize); } + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); + LOG.info("rdb dim table config info: {} ", rdbSideTableInfo.toString()); + } + @Override public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exception { CRow copyCrow = new CRow(input.row(), input.change()); @@ -238,8 +248,8 @@ public Row fillData(Row input, Object line) { @Override public void close() throws Exception { super.close(); - if (rdbSQLClient != null) { - rdbSQLClient.close(); + if (rdbSqlClient != null) { + rdbSqlClient.close(); } } @@ -254,8 +264,8 @@ public String buildCacheKey(JsonArray jsonArray) { return sb.toString(); } - public void setRdbSQLClient(SQLClient rdbSQLClient) { - this.rdbSQLClient = rdbSQLClient; + public void setRdbSqlClient(SQLClient rdbSqlClient) { + this.rdbSqlClient = rdbSqlClient; } } 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 f3830793e..4b6782494 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 @@ -23,8 +23,8 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.PredicateInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.BaseSideInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; import com.google.common.collect.Lists; @@ -33,6 +33,8 @@ import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Arrays; import java.util.List; @@ -48,16 +50,18 @@ * @author maqi */ -public class RdbAsyncSideInfo extends SideInfo { +public class RdbAsyncSideInfo extends BaseSideInfo { private static final long serialVersionUID = 1942629132469918611L; + private static final Logger LOG = LoggerFactory.getLogger(RdbAsyncSideInfo.class); - public RdbAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + + public RdbAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + public void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo) { RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideTableInfo; String sideTableName = joinInfo.getSideTableName(); @@ -75,7 +79,7 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { sqlCondition = getSelectFromStatement(getTableName(rdbSideTableInfo), Arrays.asList(StringUtils.split(sideSelectFields, ",")), equalFieldList, sqlJoinCompareOperate, sideTableInfo.getPredicateInfoes()); - System.out.println("----------dimension sql query-----------\n" + sqlCondition); + LOG.info("----------dimension sql query-----------\n{}", sqlCondition); } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/provider/DTC3P0DataSourceProvider.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/provider/DTC3P0DataSourceProvider.java index fcca4f0ef..542258257 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/provider/DTC3P0DataSourceProvider.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/provider/DTC3P0DataSourceProvider.java @@ -25,13 +25,19 @@ import javax.sql.DataSource; import java.beans.PropertyVetoException; import java.sql.SQLException; - +/** + * Date: 2019/9/17 + * Company: www.dtstack.com + * @author maqi + */ public class DTC3P0DataSourceProvider extends C3P0DataSourceProvider { @Override public DataSource getDataSource(JsonObject config) throws SQLException { String url = config.getString("url"); - if (url == null) throw new NullPointerException("url cannot be null"); + if (url == null) { + throw new NullPointerException("url cannot be null"); + } String driverClass = config.getString("driver_class"); String user = config.getString("user"); String password = config.getString("password"); diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java index b9811b0ee..79942ae11 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java @@ -19,8 +19,8 @@ package com.dtstack.flink.sql.side.rdb.table; -import com.dtstack.flink.sql.table.AbsSideTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractSideTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; @@ -33,10 +33,10 @@ * @author maqi */ -public class RdbSideParser extends AbsSideTableParser { +public class RdbSideParser extends AbstractSideTableParser { @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { RdbSideTableInfo rdbTableInfo = new RdbSideTableInfo(); rdbTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, rdbTableInfo); diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java index fdb024b1d..46a60a10b 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java @@ -17,7 +17,7 @@ */ package com.dtstack.flink.sql.side.rdb.table; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.google.common.base.Preconditions; /** @@ -27,7 +27,7 @@ * * @author maqi */ -public class RdbSideTableInfo extends SideTableInfo { +public class RdbSideTableInfo extends AbstractSideTableInfo { private static final long serialVersionUID = -1L; 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 1205c5521..49d163d4d 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,7 @@ public static Object getTarget(Object obj, String targetType) { case "timestamp": case "datetime": return MathUtil.getTimestamp(obj); + default: } return obj; } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/AbstractRdbSink.java similarity index 90% rename from rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java rename to rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/AbstractRdbSink.java index 84eea68ff..ee51c22d1 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/AbstractRdbSink.java @@ -20,8 +20,8 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.format.JDBCUpsertOutputFormat; import com.dtstack.flink.sql.sink.rdb.table.RdbTableInfo; -import com.dtstack.flink.sql.table.TableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -44,11 +44,8 @@ * * @author maqi */ -public abstract class RdbSink implements RetractStreamTableSink, Serializable, IStreamSinkGener { - - protected String driverName; - - protected String dbURL; +public abstract class AbstractRdbSink implements RetractStreamTableSink, Serializable, IStreamSinkGener { + protected String dbUrl; protected String userName; @@ -86,20 +83,20 @@ public abstract class RdbSink implements RetractStreamTableSink, Serializab public List fieldList; public List fieldTypeList; - public List fieldExtraInfoList; + public List fieldExtraInfoList; - public RdbSink(JDBCDialect jdbcDialect) { + public AbstractRdbSink(JDBCDialect jdbcDialect) { this.jdbcDialect = jdbcDialect; } @Override - public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { + public AbstractRdbSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { RdbTableInfo rdbTableInfo = (RdbTableInfo) targetTableInfo; this.batchNum = rdbTableInfo.getBatchSize() == null ? batchNum : rdbTableInfo.getBatchSize(); this.batchWaitInterval = rdbTableInfo.getBatchWaitInterval() == null ? batchWaitInterval : rdbTableInfo.getBatchWaitInterval(); this.parallelism = rdbTableInfo.getParallelism() == null ? parallelism : rdbTableInfo.getParallelism(); - this.dbURL = rdbTableInfo.getUrl(); + this.dbUrl = rdbTableInfo.getUrl(); this.userName = rdbTableInfo.getUserName(); this.password = rdbTableInfo.getPassword(); this.tableName = rdbTableInfo.getTableName(); diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/JDBCOptions.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/JDBCOptions.java index e9a4f391f..f476bfadf 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/JDBCOptions.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/JDBCOptions.java @@ -29,7 +29,7 @@ public class JDBCOptions { - private String dbURL; + private String dbUrl; private String tableName; private String driverName; private String username; @@ -37,9 +37,9 @@ public class JDBCOptions { private String schema; private JDBCDialect dialect; - private JDBCOptions(String dbURL, String tableName, String driverName, String username, + private JDBCOptions(String dbUrl, String tableName, String driverName, String username, String password, String schema, JDBCDialect dialect) { - this.dbURL = dbURL; + this.dbUrl = dbUrl; this.tableName = tableName; this.driverName = driverName; this.username = username; @@ -48,8 +48,8 @@ private JDBCOptions(String dbURL, String tableName, String driverName, String us this.dialect = dialect; } - public String getDbURL() { - return dbURL; + public String getDbUrl() { + return dbUrl; } public String getTableName() { @@ -84,7 +84,7 @@ public static Builder builder() { public boolean equals(Object o) { if (o instanceof JDBCOptions) { JDBCOptions options = (JDBCOptions) o; - return Objects.equals(dbURL, options.dbURL) && + return Objects.equals(dbUrl, options.dbUrl) && Objects.equals(tableName, options.tableName) && Objects.equals(driverName, options.driverName) && Objects.equals(username, options.username) && @@ -100,7 +100,7 @@ public boolean equals(Object o) { * Builder of {@link JDBCOptions}. */ public static class Builder { - private String dbURL; + private String dbUrl; private String tableName; private String driverName; private String username; @@ -152,8 +152,8 @@ public Builder setSchema(String schema) { /** * required, JDBC DB url. */ - public Builder setDBUrl(String dbURL) { - this.dbURL = dbURL; + public Builder setDbUrl(String dbUrl) { + this.dbUrl = dbUrl; return this; } @@ -163,7 +163,7 @@ public Builder setDialect(JDBCDialect dialect) { } public JDBCOptions build() { - checkNotNull(dbURL, "No dbURL supplied."); + checkNotNull(dbUrl, "No dbURL supplied."); checkNotNull(tableName, "No tableName supplied."); if (this.driverName == null) { @@ -173,7 +173,7 @@ public JDBCOptions build() { }); } - return new JDBCOptions(dbURL, tableName, driverName, username, password, schema, dialect); + return new JDBCOptions(dbUrl, tableName, driverName, username, password, schema, dialect); } } } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/AbstractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/AbstractJDBCOutputFormat.java index 73f17d8fd..2a3ce5e90 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/AbstractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/AbstractJDBCOutputFormat.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.sink.rdb.format; -import com.dtstack.flink.sql.outputformat.DtRichOutputFormat; +import com.dtstack.flink.sql.outputformat.AbstractDtRichOutputFormat; import com.dtstack.flink.sql.util.JDBCUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; @@ -36,7 +36,7 @@ * @see Row * @see DriverManager */ -public abstract class AbstractJDBCOutputFormat extends DtRichOutputFormat { +public abstract class AbstractJDBCOutputFormat extends AbstractDtRichOutputFormat { private static final long serialVersionUID = 1L; public static final int DEFAULT_FLUSH_MAX_SIZE = 100; @@ -47,16 +47,16 @@ public abstract class AbstractJDBCOutputFormat extends DtRichOutputFormat protected final String username; protected final String password; - private final String drivername; + private final String driverName; protected final String dbURL; protected transient Connection connection; - public AbstractJDBCOutputFormat(String username, String password, String drivername, String dbURL) { + public AbstractJDBCOutputFormat(String username, String password, String driverName, String dbUrl) { this.username = username; this.password = password; - this.drivername = drivername; - this.dbURL = dbURL; + this.driverName = driverName; + this.dbURL = dbUrl; } @Override @@ -64,7 +64,7 @@ public void configure(Configuration parameters) { } protected void establishConnection() throws SQLException, ClassNotFoundException, IOException { - JDBCUtils.forName(drivername, getClass().getClassLoader()); + JDBCUtils.forName(driverName, getClass().getClassLoader()); if (username == null) { connection = DriverManager.getConnection(dbURL); } else { diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/JDBCUpsertOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/JDBCUpsertOutputFormat.java index e1b2cc6f4..e8b6dc8a4 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 @@ -25,7 +25,7 @@ import com.dtstack.flink.sql.sink.rdb.dialect.JDBCDialect; import com.dtstack.flink.sql.sink.rdb.writer.AppendOnlyWriter; import com.dtstack.flink.sql.sink.rdb.writer.JDBCWriter; -import com.dtstack.flink.sql.sink.rdb.writer.UpsertWriter; +import com.dtstack.flink.sql.sink.rdb.writer.AbstractUpsertWriter; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.Row; @@ -44,6 +44,7 @@ /** * An upsert OutputFormat for JDBC. + * @author maqi */ public class JDBCUpsertOutputFormat extends AbstractJDBCOutputFormat> { @@ -83,7 +84,7 @@ public JDBCUpsertOutputFormat( long flushIntervalMills, boolean allReplace, String updateMode) { - super(options.getUsername(), options.getPassword(), options.getDriverName(), options.getDbURL()); + super(options.getUsername(), options.getPassword(), options.getDriverName(), options.getDbUrl()); this.schema = options.getSchema(); this.tableName = options.getTableName(); this.dialect = options.getDialect(); @@ -111,12 +112,11 @@ public void open(int taskNumber, int numTasks) throws IOException { 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); - System.out.println("execute insert sql :" + insertSQL); - jdbcWriter = new AppendOnlyWriter(insertSQL, fieldTypes, this); + String insertSql = dialect.getInsertIntoStatement(schema, tableName, fieldNames, partitionFields); + LOG.info("execute insert sql: {}", insertSql); + jdbcWriter = new AppendOnlyWriter(insertSql, fieldTypes, this); } else { - jdbcWriter = UpsertWriter.create( + jdbcWriter = AbstractUpsertWriter.create( dialect, schema, tableName, fieldNames, fieldTypes, keyFields, partitionFields, getRuntimeContext().getExecutionConfig().isObjectReuseEnabled(), allReplace, this); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java new file mode 100644 index 000000000..e69de29bb diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java index 9bbff3b4d..aca7acb6c 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java @@ -17,9 +17,8 @@ */ package com.dtstack.flink.sql.sink.rdb.table; -import com.dtstack.flink.sql.constrant.ConfigConstrant; -import com.dtstack.flink.sql.table.AbsTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; @@ -31,9 +30,9 @@ * * @author maqi */ -public class RdbSinkParser extends AbsTableParser { +public class RdbSinkParser extends AbstractTableParser { @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { RdbTableInfo rdbTableInfo = new RdbTableInfo(); rdbTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, rdbTableInfo); diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java index e490978e4..ccdf28e48 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java @@ -18,12 +18,10 @@ package com.dtstack.flink.sql.sink.rdb.table; import com.dtstack.flink.sql.enums.EUpdateMode; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import com.google.common.base.Preconditions; import org.apache.commons.lang3.StringUtils; -import java.util.Arrays; - /** * Reason: * Date: 2018/11/27 @@ -31,7 +29,7 @@ * * @author maqi */ -public class RdbTableInfo extends TargetTableInfo { +public class RdbTableInfo extends AbstractTargetTableInfo { public static final String URL_KEY = "url"; diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/UpsertWriter.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/AbstractUpsertWriter.java similarity index 84% rename from rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/UpsertWriter.java rename to rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/AbstractUpsertWriter.java index ce3a1c680..2c070b680 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/UpsertWriter.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/writer/AbstractUpsertWriter.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.sink.rdb.writer; -import com.dtstack.flink.sql.outputformat.DtRichOutputFormat; +import com.dtstack.flink.sql.outputformat.AbstractDtRichOutputFormat; import com.dtstack.flink.sql.sink.rdb.dialect.JDBCDialect; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.Row; @@ -41,14 +41,15 @@ /** * Upsert writer to deal with upsert, delete message.dd + * @author maqi */ -public abstract class UpsertWriter implements JDBCWriter { +public abstract class AbstractUpsertWriter implements JDBCWriter { private static final long serialVersionUID = 1L; - private static final Logger LOG = LoggerFactory.getLogger(UpsertWriter.class); + private static final Logger LOG = LoggerFactory.getLogger(AbstractUpsertWriter.class); - public static UpsertWriter create( + public static AbstractUpsertWriter create( JDBCDialect dialect, String schema, String tableName, @@ -58,7 +59,7 @@ public static UpsertWriter create( String[] partitionFields, boolean objectReuse, boolean allReplace, - DtRichOutputFormat metricOutputFormat) { + AbstractDtRichOutputFormat metricOutputFormat) { checkNotNull(keyFields); @@ -67,20 +68,18 @@ public static UpsertWriter create( int[] pkTypes = fieldTypes == null ? null : Arrays.stream(pkFields).map(f -> fieldTypes[f]).toArray(); - String deleteSQL = dialect.getDeleteStatement(schema, tableName, keyFields); - LOG.info("deleteSQL is :{}", deleteSQL); - System.out.println("deleteSQL is :" + deleteSQL); + String deleteSql = dialect.getDeleteStatement(schema, tableName, keyFields); + LOG.info("deleteSQL is :{}", deleteSql); - Optional upsertSQL = dialect.getUpsertStatement(schema, tableName, fieldNames, keyFields, allReplace); - LOG.info("execute UpsertStatement: {}", upsertSQL.orElse("use UsingInsertUpdateStatement")); - System.out.println("execute UpsertStatement: " + upsertSQL.orElse("use UsingInsertUpdateStatement")); + Optional upsertSql = dialect.getUpsertStatement(schema, tableName, fieldNames, keyFields, allReplace); + LOG.info("execute UpsertStatement: {}", upsertSql.orElse("use UsingInsertUpdateStatement")); - return upsertSQL.map((Function) sql -> + return upsertSql.map((Function) sql -> new UpsertWriterUsingUpsertStatement( - fieldTypes, pkFields, pkTypes, objectReuse, deleteSQL, sql, metricOutputFormat)) + fieldTypes, pkFields, pkTypes, objectReuse, deleteSql, sql, metricOutputFormat)) .orElseGet(() -> new UpsertWriterUsingInsertUpdateStatement( - fieldTypes, pkFields, pkTypes, objectReuse, deleteSQL, + fieldTypes, pkFields, pkTypes, objectReuse, deleteSql, dialect.getRowExistsStatement(tableName, keyFields), dialect.getInsertIntoStatement(schema, tableName, fieldNames, partitionFields), dialect.getUpdateStatement(tableName, fieldNames, keyFields), @@ -90,19 +89,19 @@ public static UpsertWriter create( final int[] fieldTypes; final int[] pkTypes; private final int[] pkFields; - private final String deleteSQL; + private final String deleteSql; private final boolean objectReuse; private transient Map> keyToRows; private transient PreparedStatement deleteStatement; // only use metric - private transient DtRichOutputFormat metricOutputFormat; + private transient AbstractDtRichOutputFormat metricOutputFormat; - private UpsertWriter(int[] fieldTypes, int[] pkFields, int[] pkTypes, String deleteSQL, boolean objectReuse, DtRichOutputFormat metricOutputFormat) { + private AbstractUpsertWriter(int[] fieldTypes, int[] pkFields, int[] pkTypes, String deleteSql, boolean objectReuse, AbstractDtRichOutputFormat metricOutputFormat) { this.fieldTypes = fieldTypes; this.pkFields = pkFields; this.pkTypes = pkTypes; - this.deleteSQL = deleteSQL; + this.deleteSql = deleteSql; this.objectReuse = objectReuse; this.metricOutputFormat = metricOutputFormat; } @@ -115,7 +114,7 @@ public void open(Connection connection) throws SQLException { @Override public void prepareStatement(Connection connection) throws SQLException { - this.deleteStatement = connection.prepareStatement(deleteSQL); + this.deleteStatement = connection.prepareStatement(deleteSql); } @Override @@ -170,7 +169,6 @@ public void executeUpdate(Connection connection) throws SQLException { } connection.commit(); } catch (Exception e) { - System.out.println(e.getCause()); // deal pg error: current transaction is aborted, commands ignored until end of transaction block connection.rollback(); connection.commit(); @@ -207,10 +205,10 @@ private Row getPrimaryKey(Row row) { // ---------------------------------------------------------------------------------------- - private static final class UpsertWriterUsingUpsertStatement extends UpsertWriter { + private static final class UpsertWriterUsingUpsertStatement extends AbstractUpsertWriter { private static final long serialVersionUID = 1L; - private final String upsertSQL; + private final String upsertSql; private transient PreparedStatement upsertStatement; @@ -219,11 +217,11 @@ private UpsertWriterUsingUpsertStatement( int[] pkFields, int[] pkTypes, boolean objectReuse, - String deleteSQL, - String upsertSQL, - DtRichOutputFormat metricOutputFormat) { - super(fieldTypes, pkFields, pkTypes, deleteSQL, objectReuse, metricOutputFormat); - this.upsertSQL = upsertSQL; + String deleteSql, + String upsertSql, + AbstractDtRichOutputFormat metricOutputFormat) { + super(fieldTypes, pkFields, pkTypes, deleteSql, objectReuse, metricOutputFormat); + this.upsertSql = upsertSql; } @Override @@ -234,7 +232,7 @@ public void open(Connection connection) throws SQLException { @Override public void prepareStatement(Connection connection) throws SQLException { super.prepareStatement(connection); - upsertStatement = connection.prepareStatement(upsertSQL); + upsertStatement = connection.prepareStatement(upsertSql); } @Override @@ -264,12 +262,12 @@ public void close() throws SQLException { } } - private static final class UpsertWriterUsingInsertUpdateStatement extends UpsertWriter { + private static final class UpsertWriterUsingInsertUpdateStatement extends AbstractUpsertWriter { private static final long serialVersionUID = 1L; - private final String existSQL; - private final String insertSQL; - private final String updateSQL; + private final String existSql; + private final String insertSql; + private final String updateSql; private transient PreparedStatement existStatement; private transient PreparedStatement insertStatement; @@ -280,15 +278,15 @@ private UpsertWriterUsingInsertUpdateStatement( int[] pkFields, int[] pkTypes, boolean objectReuse, - String deleteSQL, - String existSQL, - String insertSQL, - String updateSQL, - DtRichOutputFormat metricOutputFormat) { - super(fieldTypes, pkFields, pkTypes, deleteSQL, objectReuse, metricOutputFormat); - this.existSQL = existSQL; - this.insertSQL = insertSQL; - this.updateSQL = updateSQL; + String deleteSql, + String existSql, + String insertSql, + String updateSql, + AbstractDtRichOutputFormat metricOutputFormat) { + super(fieldTypes, pkFields, pkTypes, deleteSql, objectReuse, metricOutputFormat); + this.existSql = existSql; + this.insertSql = insertSql; + this.updateSql = updateSql; } @Override @@ -299,9 +297,9 @@ public void open(Connection connection) throws SQLException { @Override public void prepareStatement(Connection connection) throws SQLException { super.prepareStatement(connection); - existStatement = connection.prepareStatement(existSQL); - insertStatement = connection.prepareStatement(insertSQL); - updateStatement = connection.prepareStatement(updateSQL); + existStatement = connection.prepareStatement(existSql); + insertStatement = connection.prepareStatement(insertSql); + updateStatement = connection.prepareStatement(updateSql); } @Override 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 ca2bd4c66..3559d4376 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 @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.sink.rdb.writer; -import com.dtstack.flink.sql.outputformat.DtRichOutputFormat; +import com.dtstack.flink.sql.outputformat.AbstractDtRichOutputFormat; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.Row; import org.slf4j.Logger; @@ -34,6 +34,7 @@ /** * Just append record to jdbc, can not receive retract/delete message. + * @author maqi */ public class AppendOnlyWriter implements JDBCWriter { @@ -41,16 +42,16 @@ public class AppendOnlyWriter implements JDBCWriter { private static final Logger LOG = LoggerFactory.getLogger(AppendOnlyWriter.class); - private final String insertSQL; + private final String insertSql; private final int[] fieldTypes; private transient PreparedStatement statement; private transient List rows; // only use metric - private transient DtRichOutputFormat metricOutputFormat; + private transient AbstractDtRichOutputFormat metricOutputFormat; - public AppendOnlyWriter(String insertSQL, int[] fieldTypes, DtRichOutputFormat metricOutputFormat) { - this.insertSQL = insertSQL; + public AppendOnlyWriter(String insertSql, int[] fieldTypes, AbstractDtRichOutputFormat metricOutputFormat) { + this.insertSql = insertSql; this.fieldTypes = fieldTypes; this.metricOutputFormat = metricOutputFormat; } @@ -63,7 +64,7 @@ public void open(Connection connection) throws SQLException { @Override public void prepareStatement(Connection connection) throws SQLException { - this.statement = connection.prepareStatement(insertSQL); + this.statement = connection.prepareStatement(insertSql); } /** 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 0d257c5b9..5d09c8f3d 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 @@ -18,6 +18,10 @@ package com.dtstack.flink.sql.side.redis; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.BaseAllReqRow; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.*; import com.dtstack.flink.sql.side.redis.enums.RedisType; import com.dtstack.flink.sql.side.redis.table.RedisSideReqRow; @@ -29,22 +33,34 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.pool2.impl.GenericObjectPoolConfig; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import com.google.common.collect.Maps; 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.*; +import redis.clients.jedis.HostAndPort; +import redis.clients.jedis.Jedis; +import redis.clients.jedis.JedisCluster; +import redis.clients.jedis.JedisCommands; +import redis.clients.jedis.JedisPool; +import redis.clients.jedis.JedisSentinelPool; import java.io.Closeable; import java.io.IOException; import java.sql.SQLException; -import java.util.*; -import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; -public class RedisAllReqRow extends AllReqRow{ +import java.util.Calendar; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.atomic.AtomicReference; +/** + * @author yanxi + */ +public class RedisAllReqRow extends BaseAllReqRow { private static final long serialVersionUID = 7578879189085344807L; @@ -62,7 +78,7 @@ public class RedisAllReqRow extends AllReqRow{ private RedisSideReqRow redisSideReqRow; - public RedisAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public RedisAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new RedisAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); this.redisSideReqRow = new RedisSideReqRow(super.sideInfo); } @@ -90,7 +106,7 @@ protected void reloadCache() { } cacheRef.set(newCache); - LOG.info("----- Redis all cacheRef reload end:{}", newCache.size()); + LOG.info("----- Redis all cacheRef reload end:{}", Calendar.getInstance()); } @Override diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java index 81e7b1f4f..ec0fa0d15 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java @@ -20,25 +20,27 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.BaseSideInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; import com.google.common.collect.Lists; import java.util.List; - -public class RedisAllSideInfo extends SideInfo { +/** + * @author yanxi + */ +public class RedisAllSideInfo extends BaseSideInfo { private static final long serialVersionUID = 1998703966487857613L; - public RedisAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public RedisAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + public void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo) { String sideTableName = joinInfo.getSideTableName(); SqlNode conditionNode = joinInfo.getCondition(); 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 f2e164922..d53bbed8c 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,8 +18,18 @@ package com.dtstack.flink.sql.side.redis; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.BaseAsyncReqRow; +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.*; +import com.dtstack.flink.sql.side.CacheMissVal; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.redis.enums.RedisType; import com.dtstack.flink.sql.side.redis.table.RedisSideReqRow; @@ -29,24 +39,20 @@ import io.lettuce.core.api.StatefulRedisConnection; import io.lettuce.core.api.async.RedisHashAsyncCommands; import io.lettuce.core.api.async.RedisKeyAsyncCommands; -import io.lettuce.core.api.async.RedisStringAsyncCommands; import io.lettuce.core.cluster.RedisClusterClient; import io.lettuce.core.cluster.api.StatefulRedisClusterConnection; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang.StringUtils; -import org.apache.flink.api.java.typeutils.RowTypeInfo; import com.google.common.collect.Maps; -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.Collections; import java.util.List; import java.util.Map; import java.util.function.Consumer; - -public class RedisAsyncReqRow extends AsyncReqRow { +/** + * @author yanxi + */ +public class RedisAsyncReqRow extends BaseAsyncReqRow { private static final long serialVersionUID = -2079908694523987738L; @@ -64,7 +70,7 @@ public class RedisAsyncReqRow extends AsyncReqRow { private RedisSideReqRow redisSideReqRow; - public RedisAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public RedisAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new RedisAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); redisSideReqRow = new RedisSideReqRow(super.sideInfo); } diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java index 9bdefe8c5..c23035ec7 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java @@ -20,24 +20,26 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.BaseSideInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; import com.google.common.collect.Lists; import java.util.List; - -public class RedisAsyncSideInfo extends SideInfo { +/** + * @author yanxi + */ +public class RedisAsyncSideInfo extends BaseSideInfo { private static final long serialVersionUID = -4851348392924455039L; - public RedisAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public RedisAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + public void buildEqualInfo(JoinInfo joinInfo, AbstractSideTableInfo sideTableInfo) { String sideTableName = joinInfo.getSideTableName(); SqlNode conditionNode = joinInfo.getCondition(); diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java index f8a83b868..514984f0a 100644 --- a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java @@ -18,16 +18,19 @@ package com.dtstack.flink.sql.side.redis.table; -import com.dtstack.flink.sql.table.AbsSideTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractSideTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; -public class RedisSideParser extends AbsSideTableParser { +/** + * @author yanxi + */ +public class RedisSideParser extends AbstractSideTableParser { @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { RedisSideTableInfo redisSideTableInfo = new RedisSideTableInfo(); redisSideTableInfo.setName(tableName); 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 5007868e9..17557f916 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 @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.side.redis.table; import com.dtstack.flink.sql.side.ISideReqRow; -import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.BaseSideInfo; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; @@ -40,9 +40,9 @@ public class RedisSideReqRow implements ISideReqRow, Serializable { private static final long serialVersionUID = 3751171828444748982L; - private SideInfo sideInfo; + private BaseSideInfo sideInfo; - public RedisSideReqRow(SideInfo sideInfo){ + public RedisSideReqRow(BaseSideInfo sideInfo){ this.sideInfo = sideInfo; } @@ -72,7 +72,7 @@ public Row fillData(Row input, Object sideInput) { return row; } - public void setRowField(Row row, Integer index, SideInfo sideInfo, String value) { + public void setRowField(Row row, Integer index, BaseSideInfo sideInfo, String value) { Integer keyIndex = sideInfo.getSideFieldIndex().get(index); String classType = sideInfo.getSideTableInfo().getFieldClassList().get(keyIndex).getName(); switch (classType){ diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java index 88cfcb8a2..5224134c8 100644 --- a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java @@ -18,10 +18,12 @@ package com.dtstack.flink.sql.side.redis.table; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.google.common.base.Preconditions; - -public class RedisSideTableInfo extends SideTableInfo { +/** + * @author yanxi + */ +public class RedisSideTableInfo extends AbstractSideTableInfo { private static final long serialVersionUID = -1L; 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 0bd053473..ae4fe5a4b 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 @@ -18,14 +18,13 @@ package com.dtstack.flink.sql.sink.redis; +import com.dtstack.flink.sql.outputformat.AbstractDtRichOutputFormat; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.pool2.impl.GenericObjectPoolConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; - -import com.dtstack.flink.sql.outputformat.DtRichOutputFormat; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.pool2.impl.GenericObjectPoolConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import redis.clients.jedis.HostAndPort; @@ -42,7 +41,10 @@ import java.util.List; import java.util.Set; -public class RedisOutputFormat extends DtRichOutputFormat { +/** + * @author yanxi + */ +public class RedisOutputFormat extends AbstractDtRichOutputFormat { private static final Logger LOG = LoggerFactory.getLogger(RedisOutputFormat.class); private String url; @@ -140,7 +142,8 @@ private void establishConnection() { break; //集群 case 3: - jedis = new JedisCluster(addresses, timeout, timeout,10, password, poolConfig); + jedis = new JedisCluster(addresses, timeout, timeout, 10, password, poolConfig); + default: } } @@ -156,10 +159,10 @@ public void writeRecord(Tuple2 record) throws IOException { return; } - HashMap map = new HashMap<>(); - for (String primaryKey : primaryKeys){ - for (int i=0; i map = new HashMap<>(8); + for (String primaryKey : primaryKeys) { + for (int i = 0; i < fieldNames.length; i++) { + if (fieldNames[i].equals(primaryKey)) { map.put(primaryKey, i); } } @@ -167,10 +170,10 @@ public void writeRecord(Tuple2 record) throws IOException { List kvList = new LinkedList<>(); for (String primaryKey : primaryKeys){ - StringBuilder primaryKV = new StringBuilder(); + StringBuilder primaryKv = new StringBuilder(); int index = map.get(primaryKey).intValue(); - primaryKV.append(primaryKey).append(":").append(row.getField(index)); - kvList.add(primaryKV.toString()); + primaryKv.append(primaryKey).append(":").append(row.getField(index)); + kvList.add(primaryKv.toString()); } String perKey = String.join(":", kvList); 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 d2e28c01f..cc49a3ba8 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 @@ -20,7 +20,7 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.redis.table.RedisTableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -33,7 +33,9 @@ import org.apache.flink.types.Row; import java.util.List; - +/** + * @author yanxi + */ public class RedisSink implements RetractStreamTableSink, IStreamSinkGener { protected String[] fieldNames; @@ -67,7 +69,7 @@ public RedisSink(){ } @Override - public RedisSink genStreamSink(TargetTableInfo targetTableInfo) { + public RedisSink genStreamSink(AbstractTargetTableInfo targetTableInfo) { RedisTableInfo redisTableInfo = (RedisTableInfo) targetTableInfo; this.url = redisTableInfo.getUrl(); this.database = redisTableInfo.getDatabase(); 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 0c9b0a64a..e965eeecb 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 @@ -18,8 +18,8 @@ package com.dtstack.flink.sql.sink.redis.table; -import com.dtstack.flink.sql.table.AbsTableParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import com.google.common.collect.Lists; import org.apache.commons.lang3.StringUtils; @@ -27,10 +27,12 @@ import java.util.Arrays; import java.util.List; import java.util.Map; - -public class RedisSinkParser extends AbsTableParser { +/** + * @author yanxi + */ +public class RedisSinkParser extends AbstractTableParser { @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { RedisTableInfo redisTableInfo = new RedisTableInfo(); redisTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, redisTableInfo); diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java index 60d6dd12f..2425f8c38 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java @@ -18,10 +18,12 @@ package com.dtstack.flink.sql.sink.redis.table; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.table.AbstractTargetTableInfo; import com.google.common.base.Preconditions; - -public class RedisTableInfo extends TargetTableInfo { +/** + * @author yanxi + */ +public class RedisTableInfo extends AbstractTargetTableInfo { private static final String CURR_TYPE = "redis"; diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java index e46e0a76f..b032fdce6 100644 --- a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java @@ -20,7 +20,7 @@ import com.dtstack.flink.sql.format.DeserializationMetricWrapper; import com.dtstack.flink.sql.format.dtnest.DtNestRowDeserializationSchema; import com.dtstack.flink.sql.source.serversocket.table.ServersocketSourceTableInfo; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.types.Row; @@ -67,7 +67,7 @@ public class CustomerSocketTextStreamFunction implements SourceFunction { ServersocketSourceTableInfo tableInfo; public CustomerSocketTextStreamFunction(ServersocketSourceTableInfo tableInfo, TypeInformation typeInfo, - Map rowAndFieldMapping, List fieldExtraInfos) { + Map rowAndFieldMapping, List fieldExtraInfos) { this.tableInfo = tableInfo; this.deserializationSchema = new DtNestRowDeserializationSchema(typeInfo, rowAndFieldMapping, fieldExtraInfos); this.deserializationMetricWrapper = new DeserializationMetricWrapper(typeInfo, deserializationSchema); @@ -94,7 +94,7 @@ public void run(SourceContext ctx) throws Exception { while (buffer.length() >= delimiter.length() && (delimPos = buffer.indexOf(delimiter)) != -1) { String record = buffer.substring(0, delimPos); // truncate trailing carriage return - if (delimiter.equals("\n") && record.endsWith("\r")) { + if ("\n".equals(delimiter) && "\r".endsWith(record)) { record = record.substring(0, record.length() - 1); } try { diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/ServersocketSource.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/ServersocketSource.java index 3ab01fb42..b4dda5f3b 100644 --- a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/ServersocketSource.java +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/ServersocketSource.java @@ -19,7 +19,7 @@ import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.serversocket.table.ServersocketSourceTableInfo; -import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.table.AbstractSourceTableInfo; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -37,7 +37,7 @@ */ public class ServersocketSource implements IStreamSourceGener
{ @Override - public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { + public Table genStreamSource(AbstractSourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { ServersocketSourceTableInfo serversocketSourceTableInfo = (ServersocketSourceTableInfo) sourceTableInfo; String tableName = serversocketSourceTableInfo.getName(); diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceParser.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceParser.java index b52e38e59..f2d861b17 100644 --- a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceParser.java +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceParser.java @@ -17,8 +17,8 @@ */ package com.dtstack.flink.sql.source.serversocket.table; -import com.dtstack.flink.sql.table.AbsSourceParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractSourceParser; +import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; @@ -30,9 +30,9 @@ * * @author maqi */ -public class ServersocketSourceParser extends AbsSourceParser { +public class ServersocketSourceParser extends AbstractSourceParser { @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { ServersocketSourceTableInfo serversocketSourceTableInfo = new ServersocketSourceTableInfo(); serversocketSourceTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, serversocketSourceTableInfo); diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java index 796728eb3..3123b477f 100644 --- a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java @@ -17,7 +17,7 @@ */ package com.dtstack.flink.sql.source.serversocket.table; -import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.table.AbstractSourceTableInfo; import com.google.common.base.Preconditions; /** @@ -27,8 +27,8 @@ * * @author maqi */ -public class ServersocketSourceTableInfo extends SourceTableInfo { - //version +public class ServersocketSourceTableInfo extends AbstractSourceTableInfo { + private static final String CURR_TYPE = "serversocket"; public static final String HOSTNAME_KEY = "host"; diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java index 961539fde..dbae271e4 100644 --- a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java @@ -20,39 +20,43 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.AbstractRdbAllReqRow; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.flink.api.java.typeutils.RowTypeInfo; import com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import java.sql.Connection; import java.sql.DriverManager; import java.util.List; import java.util.Map; + /** - * side operator with cache for all(period reload) + * side operator with cache for all(period reload) + * Date: 2019/11/26 + * Company: www.dtstack.com + * @author maqi */ -public class SqlserverAllReqRow extends RdbAllReqRow { +public class SqlserverAllReqRow extends AbstractRdbAllReqRow { private static final Logger LOG = LoggerFactory.getLogger(SqlserverAllReqRow.class); private static final String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; - public SqlserverAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public SqlserverAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new SqlserverAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @Override - public Connection getConn(String dbURL, String userName, String password) { + public Connection getConn(String dbUrl, String userName, String password) { try { Class.forName(SQLSERVER_DRIVER); //add param useCursorFetch=true Map addParams = Maps.newHashMap(); - //addParams.put("useCursorFetch", "true"); - String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); + String targetDbUrl = DtStringUtil.addJdbcParam(dbUrl, addParams, true); return DriverManager.getConnection(targetDbUrl, userName, password); } catch (Exception e) { LOG.error("", e); diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java index 2b9913e7e..a40af5697 100644 --- a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java +++ b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java @@ -20,14 +20,18 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import java.util.List; - +/** + * Date: 2019/11/26 + * Company: www.dtstack.com + * @author maqi + */ public class SqlserverAllSideInfo extends RdbAllSideInfo { - public SqlserverAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public SqlserverAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } } diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java index 9fc2c35cd..a63d28607 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java @@ -33,14 +33,18 @@ import java.util.List; - +/** + * Date: 2019/11/26 + * Company: www.dtstack.com + * @author maqi + */ public class SqlserverAsyncReqRow extends RdbAsyncReqRow { private static final Logger LOG = LoggerFactory.getLogger(SqlserverAsyncReqRow.class); private final static String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; - public SqlserverAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public SqlserverAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(new SqlserverAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @@ -67,6 +71,6 @@ public void open(Configuration parameters) throws Exception { vo.setWorkerPoolSize(rdbSideTableInfo.getAsyncPoolSize()); vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); - setRdbSQLClient(JDBCClient.createNonShared(vertx, sqlserverClientConfig)); + setRdbSqlClient(JDBCClient.createNonShared(vertx, sqlserverClientConfig)); } } diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncSideInfo.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncSideInfo.java index 347ca9bf4..e8a487721 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncSideInfo.java +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncSideInfo.java @@ -20,16 +20,21 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.AbstractSideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import java.util.List; +import java.util.List; +/** + * Date: 2019/11/26 + * Company: www.dtstack.com + * @author maqi + */ public class SqlserverAsyncSideInfo extends RdbAsyncSideInfo { - public SqlserverAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public SqlserverAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, AbstractSideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } } diff --git a/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java b/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java index 3631dcd8a..e06c13898 100644 --- a/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java +++ b/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java @@ -18,16 +18,20 @@ package com.dtstack.flink.sql.side.sqlserver.table; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import java.util.Map; - +/** + * Date: 2019/11/26 + * Company: www.dtstack.com + * @author maqi + */ public class SqlserverSideParser extends RdbSideParser { private static final String CURR_TYPE = "sqlserver"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo sqlServerTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + AbstractTableInfo sqlServerTableInfo = super.getTableInfo(tableName, fieldsInfo, props); sqlServerTableInfo.setType(CURR_TYPE); return sqlServerTableInfo; } diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java index dbbeb820e..3c331cdd0 100644 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java @@ -19,7 +19,7 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.JDBCOptions; -import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.AbstractRdbSink; import com.dtstack.flink.sql.sink.rdb.format.JDBCUpsertOutputFormat; /** @@ -29,7 +29,7 @@ * * @author maqi */ -public class SqlserverSink extends RdbSink implements IStreamSinkGener { +public class SqlserverSink extends AbstractRdbSink implements IStreamSinkGener { public SqlserverSink() { super(new SqlserverDialect()); @@ -38,7 +38,7 @@ public SqlserverSink() { @Override public JDBCUpsertOutputFormat getOutputFormat() { JDBCOptions jdbcOptions = JDBCOptions.builder() - .setDBUrl(dbURL) + .setDbUrl(dbUrl) .setDialect(jdbcDialect) .setUsername(userName) .setPassword(password) diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java index a695d6c3b..5300884bc 100644 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.sink.sqlserver.table; import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; -import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.AbstractTableInfo; import java.util.Map; @@ -33,8 +33,8 @@ public class SqlserverSinkParser extends RdbSinkParser { private static final String CURR_TYPE = "sqlserver"; @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo sqlserverTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + AbstractTableInfo sqlserverTableInfo = super.getTableInfo(tableName, fieldsInfo, props); sqlserverTableInfo.setType(CURR_TYPE); return sqlserverTableInfo; } From c5859aae1a45bbc93d86ed56959480ee8fd4aaa3 Mon Sep 17 00:00:00 2001 From: dapeng Date: Mon, 11 May 2020 16:04:27 +0800 Subject: [PATCH 084/109] =?UTF-8?q?hbase=E7=BB=93=E6=9E=9C=E8=A1=A8?= =?UTF-8?q?=E6=96=87=E6=A1=A3=E6=9B=B4=E6=96=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../sql/side/cassandra/table/CassandraSideParser.java | 1 + docs/plugin/hbaseSink.md | 10 +++++++--- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java b/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java index 62dd753b9..38b1d082a 100644 --- a/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java +++ b/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java @@ -99,6 +99,7 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Mon, 11 May 2020 16:05:00 +0800 Subject: [PATCH 085/109] =?UTF-8?q?hbase=E7=BB=93=E6=9E=9C=E8=A1=A8?= =?UTF-8?q?=E6=96=87=E6=A1=A3=E6=9B=B4=E6=96=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/hbaseSink.md | 52 ++++++++++++++++++++++++++++++++-------- 1 file changed, 42 insertions(+), 10 deletions(-) diff --git a/docs/plugin/hbaseSink.md b/docs/plugin/hbaseSink.md index d151fc139..00c340b3d 100644 --- a/docs/plugin/hbaseSink.md +++ b/docs/plugin/hbaseSink.md @@ -41,19 +41,51 @@ hbase2.0 ## 5.样例: ``` - CREATE TABLE MyResult( - cf:info VARCHAR, - cf:name VARCHAR, - cf:channel varchar +CREATE TABLE MyTable( + name varchar, + channel varchar, + age int )WITH( - type ='hbase', - zookeeperQuorum ='xx:2181', - zookeeperParent ='/hbase', - tableName ='workerinfo01', - rowKey ='channel' + type ='kafka10', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest01', + timezone='Asia/Shanghai', + updateMode ='append', + enableKeyPartitions ='false', + topicIsPattern ='false', + parallelism ='1' + ); + +CREATE TABLE MyResult( + cf:name varchar , + cf:channel varchar + )WITH( + type ='hbase', + zookeeperQuorum ='172.16.10.104:2181,172.16.10.224:2181,172.16.10.252:2181', + zookeeperParent ='/hbase', + tableName ='myresult', + partitionedJoin ='false', + parallelism ='1', + rowKey='name,channel' ); + +insert +into + MyResult + select + channel, + name + from + MyTable a ``` ## 6.hbase数据 ### 数据内容说明 -### 数据内容示例 \ No newline at end of file +hbase的rowkey 构建规则:以描述的rowkey字段值作为key,多个字段以'-'连接 +### 数据内容示例 +hbase(main):007:0> scan 'myresult' + ROW COLUMN+CELL + roc-daishu column=cf:channel, timestamp=1589183971724, value=daishu + roc-daishu column=cf:name, timestamp=1589183971724, value=roc \ No newline at end of file From 960660be24c86a1c44edab0a92e43bbabd197b53 Mon Sep 17 00:00:00 2001 From: xuchao Date: Mon, 11 May 2020 16:20:22 +0800 Subject: [PATCH 086/109] =?UTF-8?q?=E5=9B=9E=E6=BB=9A=205f30a74b=20?= =?UTF-8?q?=EF=BC=8C=E8=A7=A3=E5=86=B3=20'=E5=9B=9E=E6=BB=9A=E5=AF=BC?= =?UTF-8?q?=E8=87=B4=E7=9A=84test=E5=88=86=E6=94=AF=E5=8F=98=E6=9B=B4?= =?UTF-8?q?=E5=86=85=E5=AE=B9'=E6=97=A0=E6=B3=95=E6=8F=90=E4=BA=A4?= =?UTF-8?q?=E7=9A=84=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java | 1 + .../main/java/com/dtstack/flink/sql/util/ParseUtils.java | 2 ++ .../dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java | 6 +++--- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java index b4240dbf3..8854ff4ec 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java @@ -27,6 +27,7 @@ import com.google.common.base.Strings; import java.io.Serializable; +import java.util.Map; /** * Join信息 diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java index fa06a6cc3..363d8bca0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java @@ -38,6 +38,7 @@ import com.google.common.collect.HashBasedTable; +import com.google.common.collect.HashBiMap; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlJoin; @@ -48,6 +49,7 @@ import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Set; import static org.apache.calcite.sql.SqlKind.AS; 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 9af6d02fd..6005146e2 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 @@ -130,7 +130,7 @@ public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exce } } - rdbSQLClient.getConnection(conn -> { + rdbSqlClient.getConnection(conn -> { if (conn.failed()) { //Treatment failures resultFuture.completeExceptionally(conn.cause()); @@ -197,9 +197,9 @@ private Object convertDataType(Object val) { } else if (val instanceof Instant) { } else if (val instanceof Timestamp) { - val = DateUtil.getStringFromTimestamp((Timestamp) val); + val = DateUtil.timestampToString((Timestamp) val); } else if (val instanceof java.util.Date) { - val = DateUtil.getStringFromDate((java.sql.Date) val); + val = DateUtil.dateToString((java.sql.Date) val); } else { val = val.toString(); } From ac7bc46d95b7c70e6a893a3643b85c5b017c8dec Mon Sep 17 00:00:00 2001 From: xuchao Date: Mon, 11 May 2020 16:59:50 +0800 Subject: [PATCH 087/109] =?UTF-8?q?=E5=8E=BB=E6=8E=89=E6=9C=AC=E5=9C=B0log?= =?UTF-8?q?=E6=8F=92=E4=BB=B6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/pom.xml | 5 ----- .../dtstack/flink/sql/exec/ExecuteProcessHelper.java | 10 ---------- 2 files changed, 15 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index eb470a3c0..be1020b04 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -122,11 +122,6 @@ junit 4.12 - - com.aiweiergou - tools-logger - ${logger.tool.version} - 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 2831eabd5..60e3c8cd9 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 @@ -18,7 +18,6 @@ package com.dtstack.flink.sql.exec; -import com.aiweiergou.tool.logger.api.ChangeLogLevelProcess; import com.dtstack.flink.sql.parser.CreateFuncParser; import com.dtstack.flink.sql.parser.CreateTmpTableParser; import com.dtstack.flink.sql.parser.FlinkPlanner; @@ -354,13 +353,4 @@ public static StreamExecutionEnvironment getStreamExeEnv(Properties confProperti return env; } - - public static void setLogLevel(ParamsInfo paramsInfo){ - String logLevel = paramsInfo.getConfProp().getProperty(ConfigConstrant.LOG_LEVEL_KEY); - if(org.apache.commons.lang3.StringUtils.isBlank(logLevel)){ - return; - } - ChangeLogLevelProcess logLevelProcess = new ChangeLogLevelProcess(); - logLevelProcess.process(logLevel); - } } \ No newline at end of file From 425bdc23c5748040a176d2b168bce2eed9192a1c Mon Sep 17 00:00:00 2001 From: xuchao Date: Mon, 11 May 2020 17:00:36 +0800 Subject: [PATCH 088/109] =?UTF-8?q?=E5=8E=BB=E6=8E=89=E6=9C=AC=E5=9C=B0log?= =?UTF-8?q?=E6=8F=92=E4=BB=B6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/java/com/dtstack/flink/sql/Main.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index fe925f9a6..5d7528869 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -38,7 +38,6 @@ public class Main { public static void main(String[] args) throws Exception { ParamsInfo paramsInfo = ExecuteProcessHelper.parseParams(args); - ExecuteProcessHelper.setLogLevel(paramsInfo); StreamExecutionEnvironment env = ExecuteProcessHelper.getStreamExecution(paramsInfo); env.execute(paramsInfo.getName()); LOG.info("program {} execution success", paramsInfo.getName()); From 3012c91c6e542c6dd1ab11049b004f96b09e8a5d Mon Sep 17 00:00:00 2001 From: dapeng Date: Mon, 11 May 2020 21:29:52 +0800 Subject: [PATCH 089/109] =?UTF-8?q?mongo=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/mongoSink.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/plugin/mongoSink.md b/docs/plugin/mongoSink.md index 8c32dc95e..bf4bbb233 100644 --- a/docs/plugin/mongoSink.md +++ b/docs/plugin/mongoSink.md @@ -102,7 +102,7 @@ on ``` - 数据结果: + ## 数据结果示例 向Topic mqTest03 发送数据 {"name":"maqi","id":1001} 插入 (1001,"maqi",null) From 0e21db667b3acbf630bea5ff56f5932086cda31f Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 12 May 2020 11:48:03 +0800 Subject: [PATCH 090/109] fix merge comflict bug --- .../sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java b/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java index a01e0f71a..502f23f0a 100644 --- a/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java +++ b/elasticsearch6/elasticsearch6-side/elasticsearch6-async-side/src/main/java/com/dtstack/flink/sql/side/elasticsearch6/Elasticsearch6AsyncReqRow.java @@ -128,7 +128,7 @@ public void onResponse(SearchResponse searchResponse) { dealCacheData(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); resultFuture.complete(rowList); } catch (Exception e) { - dealFillDataError(resultFuture, e, input); + dealFillDataError(input, resultFuture, e); } finally { if (tmpRhlClient != null) { try { From 42e83e7eb13f8353376456203d383bd0e018a600 Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 12 May 2020 14:15:59 +0800 Subject: [PATCH 091/109] =?UTF-8?q?=E7=BC=96=E8=AF=91=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 71652042c..fbb0aa291 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 @@ -144,7 +144,7 @@ public void accept(Map values) { dealCacheData(key,CacheObj.buildCacheObj(ECacheContentType.MultiLine, values)); resultFuture.complete(Collections.singleton(new CRow(row, input.change()))); } catch (Exception e) { - dealFillDataError(resultFuture, e, input); + dealFillDataError(input, resultFuture, e); } } else { dealMissKey(input, resultFuture); From 811797dc454bd5504ec2caedf358bb8d8f11c818 Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 12 May 2020 15:09:49 +0800 Subject: [PATCH 092/109] =?UTF-8?q?oracle=20=E6=96=87=E6=A1=A3=E6=9B=B4?= =?UTF-8?q?=E6=96=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/oracleSink.md | 70 +++++++++++++++++++++++++++++++++------ 1 file changed, 59 insertions(+), 11 deletions(-) diff --git a/docs/plugin/oracleSink.md b/docs/plugin/oracleSink.md index c15a6109a..93771f3d6 100644 --- a/docs/plugin/oracleSink.md +++ b/docs/plugin/oracleSink.md @@ -37,19 +37,67 @@ CREATE TABLE tableName( | tableName | oracle表名称|是|| | schema | oracle 的schema|否|当前登录用户| | parallelism | 并行度设置|否|1| +| batchSize | flush的大小|否|100| +| batchWaitInterval | flush的时间间隔,单位ms|否|1000| +| allReplace | true:新值替换旧值|否|false| +| updateMode | APPEND:不回撤数据,只下发增量数据,UPSERT:先删除回撤数据,然后更新|否|结果表设置主键则为UPSERT| + ## 5.样例: ``` -CREATE TABLE MyResult( - channel VARCHAR, - pv VARCHAR +CREATE TABLE MyTable( + name varchar, + channel varchar, + id int )WITH( - type ='oracle', - url ='jdbc:oracle:thin:@xx.xx.xx.xx:1521:orcl', - userName ='dtstack', - password ='abc123', - tableName ='pv2', - schema = 'MQTEST', + type ='kafka10', + bootstrapServers ='172.16.8.107:9092', + zookeeperQuorum ='172.16.8.107:2181/kafka', + offsetReset ='latest', + topic ='mqTest01', + timezone='Asia/Shanghai', + updateMode ='append', + enableKeyPartitions ='false', + topicIsPattern ='false', parallelism ='1' - ) - ``` \ No newline at end of file + ); + + CREATE TABLE MyResult( + primarykey_id int , + name VARCHAR, + address VARCHAR + )WITH( + type ='oracle', + url ='jdbc:oracle:thin:@172.16.8.178', + userName ='system', + password ='oracle', + tableName ='YELUO_TEST_ORACLE_01', + updateMode ='append', + parallelism ='1', + batchSize ='100', + batchWaitInterval ='1000' + ); + +insert +into + MyResult + select + id as primarykey_id, + channel as address, + name + from + MyTable a + ``` + +## 6.数据示例 +### 输入数据 +``` +{"name":"roc","id":11,"channel":"daishuyun"} +``` +### 结果数据 +``` ++---------+------+------+-----------+ +| primarykey_id | name | address | ++---------+------+------+----------+ +| 11 | roc | daishuyun | +``` \ No newline at end of file From 2051383ef4ff48ff84512bb8ca5b81147f5b4a92 Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 12 May 2020 16:19:48 +0800 Subject: [PATCH 093/109] =?UTF-8?q?rdb=20=E8=BF=9E=E6=8E=A5=E9=87=8D?= =?UTF-8?q?=E8=AF=95=E9=85=8D=E7=BD=AE=E9=A1=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/side/AbstractSideTableInfo.java | 15 ++++++++++++++- .../flink/sql/table/AbstractSideTableParser.java | 8 ++++---- .../flink/sql/side/rdb/async/RdbAsyncReqRow.java | 2 +- 3 files changed, 19 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java index 75f507684..994674546 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AbstractSideTableInfo.java @@ -55,7 +55,9 @@ public abstract class AbstractSideTableInfo extends AbstractTableInfo implements public static final String ASYNC_TIMEOUT_KEY = "asyncTimeout"; public static final String ASYNC_FAIL_MAX_NUM_KEY = "asyncFailMaxNum"; - + + public static final String CONNECT_RETRY_MAX_NUM_KEY = "connectRetryMaxNum"; + public static final String ASYNC_REQ_POOL_KEY = "asyncPoolSize"; private String cacheType = "none"; @@ -79,6 +81,8 @@ public abstract class AbstractSideTableInfo extends AbstractTableInfo implements private Long asyncFailMaxNum; + private Integer connectRetryMaxNum; + private List predicateInfoes = Lists.newArrayList(); public RowTypeInfo getRowTypeInfo(){ @@ -164,6 +168,7 @@ public void setAsyncFailMaxNum(Long asyncFailMaxNum) { this.asyncFailMaxNum = asyncFailMaxNum; } + public int getAsyncPoolSize() { return asyncPoolSize; } @@ -172,6 +177,14 @@ public void setAsyncPoolSize(int asyncPoolSize) { this.asyncPoolSize = asyncPoolSize; } + + public Integer getConnectRetryMaxNum(Integer defaultValue) { + return Objects.isNull(connectRetryMaxNum) ? defaultValue : connectRetryMaxNum; + } + + public void setConnectRetryMaxNum(Integer connectRetryMaxNum) { + this.connectRetryMaxNum = connectRetryMaxNum; + } @Override public String toString() { return "Cache Info{" + diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java index 3aff6e2e6..276575c7e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java @@ -110,10 +110,10 @@ protected void parseCacheProp(AbstractSideTableInfo sideTableInfo, Map 0){ - sideTableInfo.setAsyncFailMaxNum(asyncFailNum); + if(props.containsKey(AbstractSideTableInfo.CONNECT_RETRY_MAX_NUM_KEY.toLowerCase())){ + Integer connectRetryMaxNum = MathUtil.getIntegerVal(props.get(AbstractSideTableInfo.CONNECT_RETRY_MAX_NUM_KEY.toLowerCase())); + if (connectRetryMaxNum > 0){ + sideTableInfo.setConnectRetryMaxNum(connectRetryMaxNum); } } } 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 b7667ca0e..6b639bed2 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 @@ -137,7 +137,7 @@ private void connectWithRetry(Map inputParams, CRow input, Resul if(failCounter.getAndIncrement() % 1000 == 0){ LOG.error("getConnection error", conn.cause()); } - if(failCounter.get() >= sideInfo.getSideTableInfo().getAsyncFailMaxNum(3L)){ + if(failCounter.get() >= sideInfo.getSideTableInfo().getConnectRetryMaxNum(3)){ dealFillDataError(input, resultFuture, conn.cause()); finishFlag.set(true); } From 95e7eaa84f55a976568b895c192248a7256ef173 Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 12 May 2020 18:00:10 +0800 Subject: [PATCH 094/109] =?UTF-8?q?=E8=A7=A3=E6=9E=90columnName=20error?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/side/BaseAsyncReqRow.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 f57956096..7e9937ad1 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 @@ -182,7 +182,7 @@ private Map parseInputParam(CRow input){ if(equalObj == null){ return inputParams; } - String columnName = sideInfo.getEqualFieldList().get(conValIndex); + String columnName = sideInfo.getEqualFieldList().get(i); inputParams.put(columnName, equalObj); } return inputParams; From c80c4d38a06965690d100b0eda63e984a558ef55 Mon Sep 17 00:00:00 2001 From: dapeng Date: Wed, 13 May 2020 11:30:07 +0800 Subject: [PATCH 095/109] fix impalaDialect npe --- .../com/dtstack/flink/sql/sink/impala/ImpalaDialect.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/ImpalaDialect.java b/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/ImpalaDialect.java index 7bb24481a..3eaffc0ad 100644 --- a/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/ImpalaDialect.java +++ b/impala/impala-sink/src/main/java/com/dtstack/flink/sql/sink/impala/ImpalaDialect.java @@ -19,10 +19,12 @@ package com.dtstack.flink.sql.sink.impala; import com.dtstack.flink.sql.sink.rdb.dialect.JDBCDialect; +import com.google.common.collect.Lists; import org.apache.commons.lang3.StringUtils; import java.util.Arrays; import java.util.List; +import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; @@ -61,7 +63,7 @@ public String getInsertIntoStatement(String schema, String tableName, String[] f String schemaInfo = StringUtils.isEmpty(schema) ? "" : quoteIdentifier(schema) + "."; - List partitionFieldsList = Arrays.asList(partitionFields); + List partitionFieldsList = Objects.isNull(partitionFields) ? Lists.newArrayList() : Arrays.asList(partitionFields); String columns = Arrays.stream(fieldNames) .filter(f -> !partitionFieldsList.contains(f)) @@ -72,7 +74,7 @@ public String getInsertIntoStatement(String schema, String tableName, String[] f .map(f -> "?") .collect(Collectors.joining(", ")); - String partitionFieldStr = Arrays.stream(partitionFields) + String partitionFieldStr = partitionFieldsList.stream() .map(field -> field.replaceAll("\"", "'")) .collect(Collectors.joining(", ")); From ae6784f040bad10971d028cd129b255eda3d2707 Mon Sep 17 00:00:00 2001 From: dapeng Date: Wed, 13 May 2020 14:46:03 +0800 Subject: [PATCH 096/109] =?UTF-8?q?fix=20=E4=BB=A3=E7=A0=81merge=E7=9A=84?= =?UTF-8?q?=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 4 ---- 1 file changed, 4 deletions(-) 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 8fc1fbb02..6e7bb3408 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 @@ -787,10 +787,6 @@ private void joinFun(Object pollObj, Map localTableCache, throw new RuntimeException("can't not find side table:" + joinInfo.getRightTableName()); } -// if(!checkJoinCondition(joinInfo.getCondition(), joinInfo.getRightTableAlias(), sideTableInfo)){ -// throw new RuntimeException("ON condition must contain all equal fields!!!"); -// } - rightScopeChild.setRowTypeInfo(sideTableInfo.getRowTypeInfo()); joinScope.addScope(leftScopeChild); From 323c6183dc9cd3fe4904c4cedaa4baba8f470040 Mon Sep 17 00:00:00 2001 From: dapeng Date: Wed, 13 May 2020 16:43:52 +0800 Subject: [PATCH 097/109] =?UTF-8?q?elasticsearch=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/elasticsearch6Sink.md | 90 ++++++++++++++++++++++++++----- docs/plugin/elasticsearchSink.md | 60 +++++++++++++++++---- 2 files changed, 128 insertions(+), 22 deletions(-) diff --git a/docs/plugin/elasticsearch6Sink.md b/docs/plugin/elasticsearch6Sink.md index 0bab12cc0..0f7a04280 100644 --- a/docs/plugin/elasticsearch6Sink.md +++ b/docs/plugin/elasticsearch6Sink.md @@ -42,21 +42,85 @@ CREATE TABLE tableName( |password | 密码 | 否,authMesh='true'时为必填 || |parallelism | 并行度设置|否|1| -## 5.样例: +## 5.完整样例: ``` +CREATE TABLE MyTable( + channel varchar, + pv INT, + xctime bigint + )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( - aa INT, - bb INT + pv varchar, + channel varchar )WITH( type ='elasticsearch6', - address ='172.16.10.47:9500', - cluster='es_47_menghan', - esType ='type1', - index ='xc_es_test', - authMesh = 'true', - userName = 'elastic', - password = 'abc123', - id ='0,1', + address ='172.16.8.193:9200', + authMesh='true', + username='elastic', + password='abc123', + estype ='external', + cluster ='docker-cluster', + index ='myresult', + id ='1', + updateMode ='append', parallelism ='1' - ) - ``` \ No newline at end of file + ); + +CREATE TABLE sideTable( + a varchar, + b varchar, + PRIMARY KEY(a) , + PERIOD FOR SYSTEM_TIME + )WITH( + type ='elasticsearch6', + address ='172.16.8.193:9200', + estype ='external', + cluster ='docker-cluster', + index ='sidetest', + authMesh='true', + username='elastic', + password='abc123', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + partitionedJoin ='false', + parallelism ='1' + ); + +insert +into + MyResult + select + w.b as pv, + s.channel as channel + from + MyTable s + join + sideTable w + on s.pv = w.a + where + w.a = '10' + and s.channel='xc'; + ``` +## 6.结果数据 +### 输入数据 +``` +{"channel":"xc26","pv":10,"xctime":1232312} +``` +### 输出数据 +``` +http://172.16.8.193:9200/myresult/_search +{"_index":"myresult","_type":"external","_id":"8aX_DHIBn3B7OBuqFl-i","_score":1.0,"_source":{"pv":"10","channel":"xc26"}} +``` \ No newline at end of file diff --git a/docs/plugin/elasticsearchSink.md b/docs/plugin/elasticsearchSink.md index 4908f8a05..9a2a32b6f 100644 --- a/docs/plugin/elasticsearchSink.md +++ b/docs/plugin/elasticsearchSink.md @@ -40,16 +40,58 @@ CREATE TABLE tableName( ## 5.样例: ``` +CREATE TABLE MyTable( + channel varchar, + pv 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( - aa INT, - bb INT + pv varchar, + channel varchar )WITH( type ='elasticsearch', - address ='172.16.10.47:9500', - cluster='es_47_menghan', - estype ='type1', - index ='xc_es_test', - id ='0,1', + address ='172.16.8.193:9200', + authMesh='true', + username='elastic', + password='abc123', + estype ='external', + cluster ='docker-cluster', + index ='myresult', + id ='1', + updateMode ='append', parallelism ='1' - ) - ``` \ No newline at end of file + ); + + +insert +into + MyResult + select + s.pv as pv, + s.channel as channel + from + MyTable s + + ``` + +## 6.结果示例 +### 输入数据示例 +``` +{"channel":"xc26","pv":"10","xctime":1232312} +``` +### 输出数据示例 +``` +http://172.16.8.193:9200/myresult/_search +{"_index":"myresult","_type":"external","_id":"8aX_DHIBn3B7OBuqFl-i","_score":1.0,"_source":{"pv":"10","channel":"xc26"}} +``` \ No newline at end of file From 3ba019dfd7453242eb9d97537c43f57c35ddd90b Mon Sep 17 00:00:00 2001 From: dapeng Date: Wed, 13 May 2020 17:33:48 +0800 Subject: [PATCH 098/109] fix --- .../dtstack/flink/sql/table/AbstractSideTableParser.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java index 276575c7e..0f0612c09 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbstractSideTableParser.java @@ -110,6 +110,14 @@ protected void parseCacheProp(AbstractSideTableInfo sideTableInfo, Map 0."); + } + sideTableInfo.setAsyncFailMaxNum(asyncFailMaxNum); + } + if(props.containsKey(AbstractSideTableInfo.CONNECT_RETRY_MAX_NUM_KEY.toLowerCase())){ Integer connectRetryMaxNum = MathUtil.getIntegerVal(props.get(AbstractSideTableInfo.CONNECT_RETRY_MAX_NUM_KEY.toLowerCase())); if (connectRetryMaxNum > 0){ From b282ce6988c912a737797edba02902c7cb293301 Mon Sep 17 00:00:00 2001 From: dapeng Date: Thu, 14 May 2020 11:46:06 +0800 Subject: [PATCH 099/109] =?UTF-8?q?=E4=BF=AE=E5=A4=8Dmongo=E8=BF=9E?= =?UTF-8?q?=E6=8E=A5url=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/mongoSide.md | 3 +-- .../flink/sql/side/mongo/MongoAllReqRow.java | 18 +++++++++++++++--- .../flink/sql/side/mongo/MongoAsyncReqRow.java | 16 +++++++++++++--- .../sql/sink/mongo/MongoOutputFormat.java | 13 ++++++++++++- 4 files changed, 41 insertions(+), 9 deletions(-) diff --git a/docs/plugin/mongoSide.md b/docs/plugin/mongoSide.md index 62d2d8698..002743256 100644 --- a/docs/plugin/mongoSide.md +++ b/docs/plugin/mongoSide.md @@ -62,8 +62,7 @@ create table sideTable( PERIOD FOR SYSTEM_TIME )WITH( type ='mongo', - //mongodb://[username:password@]host1[:port1][,host2[:port2],...[,hostN[:portN]]]/[?options]] - address ='mongodb://172.21.32.1:27017,172.21.32.1:27017', + address ='172.21.32.1:27017,172.21.32.1:27017', database ='test', tableName ='sidetest', cache ='LRU', diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java index c1a32cb64..1857dbede 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java @@ -177,9 +177,10 @@ private String buildKey(Map val, List equalFieldList) { return sb.toString(); } - private MongoCollection getConn(String address, String database, String tableName) { + private MongoCollection getConn(String host, String userName, String password, String database, String tableName) { + MongoCollection dbCollection; - mongoClient = new MongoClient(new MongoClientURI(address)); + mongoClient = new MongoClient(new MongoClientURI(getConnectionUrl(host, userName, password))); db = mongoClient.getDatabase(database); dbCollection = db.getCollection(tableName, Document.class); return dbCollection; @@ -193,7 +194,8 @@ private void loadData(Map>> tmpCache) throws SQ try { for (int i = 0; i < CONN_RETRY_NUM; i++) { try { - dbCollection = getConn(tableInfo.getAddress(), tableInfo.getDatabase(), tableInfo.getTableName()); + dbCollection = getConn(tableInfo.getAddress(), tableInfo.getUserName(), tableInfo.getPassword(), + tableInfo.getDatabase(), tableInfo.getTableName()); break; } catch (Exception e) { if (i == CONN_RETRY_NUM - 1) { @@ -251,4 +253,14 @@ private void loadData(Map>> tmpCache) throws SQ } } } + private String getConnectionUrl(String address, String userName, String password){ + if(address.startsWith("mongodb://") || address.startsWith("mongodb+srv://")){ + return address; + } + if (StringUtils.isNotBlank(userName) && StringUtils.isNotBlank(password)) { + return String.format("mongodb://%s:%s@%s", userName, password, address); + } + return String.format("mongodb://%s", address); + } + } diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index 6bea9e024..04f2ad9af 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.side.mongo; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.ResultFuture; @@ -84,9 +85,8 @@ public void open(Configuration parameters) throws Exception { } public void connMongoDb() throws Exception { - String address = mongoSideTableInfo.getAddress(); - ConnectionString connectionString = new ConnectionString(address); - + ConnectionString connectionString = new ConnectionString(getConnectionUrl(mongoSideTableInfo.getAddress(), + mongoSideTableInfo.getUserName(), mongoSideTableInfo.getPassword())); MongoClientSettings settings = MongoClientSettings.builder() .applyConnectionString(connectionString) .build(); @@ -193,4 +193,14 @@ public void close() throws Exception { } } + private String getConnectionUrl(String address, String userName, String password){ + if(address.startsWith("mongodb://") || address.startsWith("mongodb+srv://")){ + return address; + } + if (StringUtils.isNotBlank(userName) && StringUtils.isNotBlank(password)) { + return String.format("mongodb://%s:%s@%s", userName, password, address); + } + return String.format("mongodb://%s", address); + } + } diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java index 37f7f5952..df8293522 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java @@ -25,6 +25,7 @@ import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoDatabase; import com.mongodb.client.result.UpdateResult; +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.configuration.Configuration; @@ -119,7 +120,8 @@ public void close() { } private void establishConnection() { - mongoClient = new MongoClient(new MongoClientURI(address)); + + mongoClient = new MongoClient(new MongoClientURI(getConnectionUrl())); db = mongoClient.getDatabase(database); } @@ -197,5 +199,14 @@ public MongoOutputFormat finish() { } } + private String getConnectionUrl(){ + if(address.startsWith("mongodb://") || address.startsWith("mongodb+srv://")){ + return address; + } + if (StringUtils.isNotBlank(userName) && StringUtils.isNotBlank(password)) { + return String.format("mongodb://%s:%s@%s", userName, password, address); + } + return String.format("mongodb://%s", address); + } } From abfb0839a87f874eeb796f35ad6c9564f1e9c02a Mon Sep 17 00:00:00 2001 From: dapeng Date: Thu, 14 May 2020 12:56:10 +0800 Subject: [PATCH 100/109] fix mongo url bug --- docs/mongoSide.md | 5 ++--- .../flink/sql/side/mongo/MongoAllReqRow.java | 18 +++++++++++++++--- .../flink/sql/side/mongo/MongoAsyncReqRow.java | 16 +++++++++++++--- .../sql/sink/mongo/MongoOutputFormat.java | 13 ++++++++++++- 4 files changed, 42 insertions(+), 10 deletions(-) diff --git a/docs/mongoSide.md b/docs/mongoSide.md index 73ce9644f..002743256 100644 --- a/docs/mongoSide.md +++ b/docs/mongoSide.md @@ -30,7 +30,7 @@ |----|---| | tableName | 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同)| | colName | 列名称| - | colType | 列类型 [colType支持的类型](colType.md)| + | colType | 列类型 [colType支持的类型](docs/colType.md)| | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| @@ -62,8 +62,7 @@ create table sideTable( PERIOD FOR SYSTEM_TIME )WITH( type ='mongo', - //mongodb://[username:password@]host1[:port1][,host2[:port2],...[,hostN[:portN]]]/[?options]] - address ='mongodb://172.21.32.1:27017,172.21.32.1:27017', + address ='172.21.32.1:27017,172.21.32.1:27017', database ='test', tableName ='sidetest', cache ='LRU', diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java index c1a32cb64..1857dbede 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java @@ -177,9 +177,10 @@ private String buildKey(Map val, List equalFieldList) { return sb.toString(); } - private MongoCollection getConn(String address, String database, String tableName) { + private MongoCollection getConn(String host, String userName, String password, String database, String tableName) { + MongoCollection dbCollection; - mongoClient = new MongoClient(new MongoClientURI(address)); + mongoClient = new MongoClient(new MongoClientURI(getConnectionUrl(host, userName, password))); db = mongoClient.getDatabase(database); dbCollection = db.getCollection(tableName, Document.class); return dbCollection; @@ -193,7 +194,8 @@ private void loadData(Map>> tmpCache) throws SQ try { for (int i = 0; i < CONN_RETRY_NUM; i++) { try { - dbCollection = getConn(tableInfo.getAddress(), tableInfo.getDatabase(), tableInfo.getTableName()); + dbCollection = getConn(tableInfo.getAddress(), tableInfo.getUserName(), tableInfo.getPassword(), + tableInfo.getDatabase(), tableInfo.getTableName()); break; } catch (Exception e) { if (i == CONN_RETRY_NUM - 1) { @@ -251,4 +253,14 @@ private void loadData(Map>> tmpCache) throws SQ } } } + private String getConnectionUrl(String address, String userName, String password){ + if(address.startsWith("mongodb://") || address.startsWith("mongodb+srv://")){ + return address; + } + if (StringUtils.isNotBlank(userName) && StringUtils.isNotBlank(password)) { + return String.format("mongodb://%s:%s@%s", userName, password, address); + } + return String.format("mongodb://%s", address); + } + } diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index 6bea9e024..04f2ad9af 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.side.mongo; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.ResultFuture; @@ -84,9 +85,8 @@ public void open(Configuration parameters) throws Exception { } public void connMongoDb() throws Exception { - String address = mongoSideTableInfo.getAddress(); - ConnectionString connectionString = new ConnectionString(address); - + ConnectionString connectionString = new ConnectionString(getConnectionUrl(mongoSideTableInfo.getAddress(), + mongoSideTableInfo.getUserName(), mongoSideTableInfo.getPassword())); MongoClientSettings settings = MongoClientSettings.builder() .applyConnectionString(connectionString) .build(); @@ -193,4 +193,14 @@ public void close() throws Exception { } } + private String getConnectionUrl(String address, String userName, String password){ + if(address.startsWith("mongodb://") || address.startsWith("mongodb+srv://")){ + return address; + } + if (StringUtils.isNotBlank(userName) && StringUtils.isNotBlank(password)) { + return String.format("mongodb://%s:%s@%s", userName, password, address); + } + return String.format("mongodb://%s", address); + } + } diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java index 37f7f5952..df8293522 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java @@ -25,6 +25,7 @@ import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoDatabase; import com.mongodb.client.result.UpdateResult; +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.configuration.Configuration; @@ -119,7 +120,8 @@ public void close() { } private void establishConnection() { - mongoClient = new MongoClient(new MongoClientURI(address)); + + mongoClient = new MongoClient(new MongoClientURI(getConnectionUrl())); db = mongoClient.getDatabase(database); } @@ -197,5 +199,14 @@ public MongoOutputFormat finish() { } } + private String getConnectionUrl(){ + if(address.startsWith("mongodb://") || address.startsWith("mongodb+srv://")){ + return address; + } + if (StringUtils.isNotBlank(userName) && StringUtils.isNotBlank(password)) { + return String.format("mongodb://%s:%s@%s", userName, password, address); + } + return String.format("mongodb://%s", address); + } } From c52ffa9a164540f3afba772e0ec6666e318d71ec Mon Sep 17 00:00:00 2001 From: dapeng Date: Thu, 14 May 2020 15:18:56 +0800 Subject: [PATCH 101/109] =?UTF-8?q?hbase=20=E8=A7=A3=E6=9E=90=E5=88=97?= =?UTF-8?q?=E6=97=8F=E7=9A=84=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../sql/sink/hbase/table/HbaseSinkParser.java | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java index bd1b7cac3..001443daa 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java @@ -26,6 +26,7 @@ import com.dtstack.flink.sql.table.AbstractTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.MathUtil; +import com.google.common.collect.Maps; import org.apache.commons.lang3.StringUtils; import java.util.LinkedHashMap; @@ -98,14 +99,23 @@ public void parseFieldsInfo(String fieldsInfo, HbaseTableInfo tableInfo){ String fieldType = filedInfoArr[filedInfoArr.length - 1 ].trim(); Class fieldClass = dbTypeConvertToJavaType(fieldType); String[] columnFamily = StringUtils.split(fieldName.trim(), ":"); - columnFamilies.put(fieldName.trim(),columnFamily[1]); tableInfo.addPhysicalMappings(filedInfoArr[0],filedInfoArr[0]); tableInfo.addField(columnFamily[1]); tableInfo.addFieldClass(fieldClass); tableInfo.addFieldType(fieldType); tableInfo.addFieldExtraInfo(null); } - tableInfo.setColumnNameFamily(columnFamilies); + tableInfo.setColumnNameFamily(parseColumnFamily(tableInfo.getPhysicalFields())); tableInfo.finish(); } + + private Map parseColumnFamily(Map physicalFieldMap){ + Map columnFamiles = Maps.newHashMap(); + physicalFieldMap.values().forEach(x -> { + String[] columnFamily = StringUtils.split(x.trim(), ":"); + columnFamiles.put(x, columnFamily[1]); + }); + + return columnFamiles; + } } From 15c86353dcf954fb48d0f11eb05703204e18ec02 Mon Sep 17 00:00:00 2001 From: dapeng Date: Thu, 14 May 2020 16:20:03 +0800 Subject: [PATCH 102/109] =?UTF-8?q?fix=20mongo=20=E5=BC=82=E6=AD=A5?= =?UTF-8?q?=E7=BB=B4=E8=A1=A8=E6=B2=A1=E6=9C=89=E8=A7=A3=E6=9E=90equalFiel?= =?UTF-8?q?d=20bug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/side/mongo/MongoAsyncReqRow.java | 2 +- .../flink/sql/side/mongo/MongoAsyncSideInfo.java | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index 04f2ad9af..0cf8158c7 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -96,7 +96,7 @@ public void connMongoDb() throws Exception { @Override public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { - CRow inputCopy = new CRow(input.row(), input.change()); + CRow inputCopy = new CRow(Row.copy(input.row()), input.change()); BasicDBObject basicDbObject = new BasicDBObject(); try { basicDbObject.putAll(inputParams); diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java index 19534d4e1..37ec8d8c6 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java @@ -22,6 +22,8 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.BaseSideInfo; import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.util.ParseUtils; +import com.google.common.collect.Lists; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; @@ -46,6 +48,16 @@ public MongoAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List sqlNodeList = Lists.newArrayList(); + ParseUtils.parseAnd(conditionNode, sqlNodeList); + + for(SqlNode sqlNode : sqlNodeList){ + dealOneEqualCon(sqlNode, sideTableName); + } } From 818c3203b5fb4d77a550c8ed9a4c1516dc2999f8 Mon Sep 17 00:00:00 2001 From: dapeng Date: Thu, 14 May 2020 21:51:16 +0800 Subject: [PATCH 103/109] fix mongo and async block bug --- .../flink/sql/side/mongo/utils/MongoUtil.java | 19 +++++++++++-------- .../sql/side/rdb/async/RdbAsyncReqRow.java | 12 +++++++++--- 2 files changed, 20 insertions(+), 11 deletions(-) diff --git a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/utils/MongoUtil.java b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/utils/MongoUtil.java index cbcba8f97..1b3ce7194 100644 --- a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/utils/MongoUtil.java +++ b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/utils/MongoUtil.java @@ -21,6 +21,7 @@ import com.dtstack.flink.sql.side.PredicateInfo; import com.mongodb.BasicDBObject; import org.apache.commons.lang3.StringUtils; +import org.bson.Document; import java.util.Arrays; import java.util.stream.Collectors; @@ -32,25 +33,27 @@ */ public class MongoUtil { public static BasicDBObject buildFilterObject(PredicateInfo info) { + + String value = info.getCondition().replaceAll("'", ""); switch (info.getOperatorName()) { case "=": - return new BasicDBObject("$eq", info.getCondition()); + return new BasicDBObject("$eq", value); case ">": - return new BasicDBObject("$gt", info.getCondition()); + return new BasicDBObject("$gt", value); case ">=": - return new BasicDBObject("$gte", info.getCondition()); + return new BasicDBObject("$gte", value); case "<": - return new BasicDBObject("$lt", info.getCondition()); + return new BasicDBObject("$lt", value); case "<=": - return new BasicDBObject("$lte", info.getCondition()); + return new BasicDBObject("$lte", value); case "<>": - return new BasicDBObject("$ne", info.getCondition()); + return new BasicDBObject("$ne", value); case "IN": - Object[] values = Arrays.stream(StringUtils.split(info.getCondition(), ",")).map(String::trim) + Object[] values = Arrays.stream(StringUtils.split(value, ",")).map(String::trim) .collect(Collectors.toList()).toArray(); return new BasicDBObject("$in", values); case "NOT IN": - return new BasicDBObject("$nin", StringUtils.split(info.getCondition(), ",")); + return new BasicDBObject("$nin", StringUtils.split(value, ",")); case "IS NOT NULL": return new BasicDBObject("$exists", true); case "IS NULL": 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 6b639bed2..e9d534df2 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 @@ -137,8 +137,8 @@ private void connectWithRetry(Map inputParams, CRow input, Resul if(failCounter.getAndIncrement() % 1000 == 0){ LOG.error("getConnection error", conn.cause()); } - if(failCounter.get() >= sideInfo.getSideTableInfo().getConnectRetryMaxNum(3)){ - dealFillDataError(input, resultFuture, conn.cause()); + if(failCounter.get() >= sideInfo.getSideTableInfo().getConnectRetryMaxNum(100)){ + resultFuture.completeExceptionally(conn.cause()); finishFlag.set(true); } conn.result().close(); @@ -155,12 +155,18 @@ private void connectWithRetry(Map inputParams, CRow input, Resul latch.countDown(); } }); - //主线程阻塞 try { latch.wait(); } catch (InterruptedException e) { LOG.error("", e); } + if(!finishFlag.get()){ + try { + Thread.sleep(100); + } catch (Exception e){ + LOG.error("", e); + } + } } } From 9435152893df7a8ac1a99b3282119592ee12f484 Mon Sep 17 00:00:00 2001 From: dapeng Date: Fri, 15 May 2020 10:10:48 +0800 Subject: [PATCH 104/109] =?UTF-8?q?kudu=E6=96=87=E6=A1=A3=E6=9B=B4?= =?UTF-8?q?=E6=96=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/plugin/kafkaSource.md | 5 +++ docs/plugin/kuduSink.md | 80 ++++++++++++++++++++++++++++++++++---- 2 files changed, 77 insertions(+), 8 deletions(-) diff --git a/docs/plugin/kafkaSource.md b/docs/plugin/kafkaSource.md index 035e39ffb..1ad247398 100644 --- a/docs/plugin/kafkaSource.md +++ b/docs/plugin/kafkaSource.md @@ -78,6 +78,11 @@ kafka.dual.commit.enabled kafka.partition.assignment.strategy kafka.socket.receive.buffer.bytes kafka.fetch.min.bytes + +###kerberos认证相关参数 +kafka.security.protocal +kafka.sasl.mechanism +kafka.sasl.kerberos.service.name ``` ## 5.样例: diff --git a/docs/plugin/kuduSink.md b/docs/plugin/kuduSink.md index 25f44ede1..b39ba1d67 100644 --- a/docs/plugin/kuduSink.md +++ b/docs/plugin/kuduSink.md @@ -46,17 +46,81 @@ kudu 1.9.0+cdh6.2.0 ## 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( - id int, - title VARCHAR, - amount decimal, - tablename1 VARCHAR + a string, + b string, + c string, + d string )WITH( type ='kudu', - kuduMasters ='localhost1,localhost2,localhost3', - tableName ='impala::default.test', - writeMode='upsert', + kuduMasters ='cdh03.cdhsite:7051', + tableName ='myresult', + writeMode='insert', parallelism ='1' ); - ``` \ No newline at end of file +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"} +``` \ No newline at end of file From bbd8cf935b12d218569bc9174c86418c2ec160d0 Mon Sep 17 00:00:00 2001 From: dapeng Date: Fri, 15 May 2020 11:52:56 +0800 Subject: [PATCH 105/109] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E5=85=A8=E5=B1=80?= =?UTF-8?q?=E7=9A=84=E8=BF=9E=E6=8E=A5=E7=8A=B6=E6=80=81=E5=8F=98=E9=87=8F?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/side/rdb/async/RdbAsyncReqRow.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) 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 e9d534df2..7340357a9 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 @@ -82,7 +82,7 @@ public class RdbAsyncReqRow extends BaseAsyncReqRow { private transient SQLClient rdbSqlClient; - private final static AtomicBoolean CONN_STATUS = new AtomicBoolean(true); + private AtomicBoolean connectionStatus = new AtomicBoolean(true); public RdbAsyncReqRow(BaseSideInfo sideInfo) { super(sideInfo); @@ -105,7 +105,7 @@ protected void preInvoke(CRow input, ResultFuture resultFuture){ public void handleAsyncInvoke(Map inputParams, CRow input, ResultFuture resultFuture) throws Exception { AtomicLong networkLogCounter = new AtomicLong(0L); - while (!CONN_STATUS.get()){//network is unhealth + while (!connectionStatus.get()){//network is unhealth if(networkLogCounter.getAndIncrement() % 1000 == 0){ LOG.info("network unhealth to block task"); } @@ -114,11 +114,11 @@ public void handleAsyncInvoke(Map inputParams, CRow input, Resul Map params = formatInputParam(inputParams); rdbSqlClient.getConnection(conn -> { if(conn.failed()){ - CONN_STATUS.set(false); + connectionStatus.set(false); connectWithRetry(params, input, resultFuture, rdbSqlClient); return; } - CONN_STATUS.set(true); + connectionStatus.set(true); ScheduledFuture timerFuture = registerTimer(input, resultFuture); cancelTimerWhenComplete(resultFuture, timerFuture); handleQuery(conn.result(), params, input, resultFuture); @@ -141,10 +141,9 @@ private void connectWithRetry(Map inputParams, CRow input, Resul resultFuture.completeExceptionally(conn.cause()); finishFlag.set(true); } - conn.result().close(); return; } - CONN_STATUS.set(true); + connectionStatus.set(true); ScheduledFuture timerFuture = registerTimer(input, resultFuture); cancelTimerWhenComplete(resultFuture, timerFuture); handleQuery(conn.result(), inputParams, input, resultFuture); From 7abe4dfd07757d026052915834329efbf6b53e6a Mon Sep 17 00:00:00 2001 From: dapeng Date: Fri, 15 May 2020 16:22:16 +0800 Subject: [PATCH 106/109] fix kudu and async quest --- .../sql/side/kudu/KuduAsyncSideInfo.java | 19 +++++++++++++++++++ .../sql/side/rdb/async/RdbAsyncReqRow.java | 4 ++-- 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncSideInfo.java b/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncSideInfo.java index 5ace515f7..91b380594 100644 --- a/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncSideInfo.java +++ b/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncSideInfo.java @@ -4,6 +4,9 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.BaseSideInfo; import com.dtstack.flink.sql.side.AbstractSideTableInfo; +import com.dtstack.flink.sql.side.kudu.table.KuduSideTableInfo; +import com.dtstack.flink.sql.util.ParseUtils; +import com.google.common.collect.Lists; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; @@ -21,6 +24,22 @@ public KuduAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List sqlNodeList = Lists.newArrayList(); + ParseUtils.parseAnd(conditionNode, sqlNodeList); + + for (SqlNode sqlNode : sqlNodeList) { + dealOneEqualCon(sqlNode, sideTableName); + } + + sqlCondition = "select ${selectField} from ${tableName} "; + sqlCondition = sqlCondition.replace("${tableName}", kuduSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); } @Override 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 7340357a9..7a6b45be6 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 @@ -155,13 +155,13 @@ private void connectWithRetry(Map inputParams, CRow input, Resul } }); try { - latch.wait(); + latch.await(); } catch (InterruptedException e) { LOG.error("", e); } if(!finishFlag.get()){ try { - Thread.sleep(100); + Thread.sleep(3000); } catch (Exception e){ LOG.error("", e); } From 3ca492b90a94a91749deaef71a91748fbd84bfea Mon Sep 17 00:00:00 2001 From: dapeng Date: Fri, 15 May 2020 18:20:23 +0800 Subject: [PATCH 107/109] =?UTF-8?q?fix=20row=20=E7=9A=84=E6=95=B0=E6=8D=AE?= =?UTF-8?q?=E9=94=99=E4=B9=B1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/side/BaseAsyncReqRow.java | 3 ++- .../com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) 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 7e9937ad1..11240b973 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 @@ -160,7 +160,8 @@ protected void preInvoke(CRow input, ResultFuture resultFuture){ } @Override - public void asyncInvoke(CRow input, ResultFuture resultFuture) throws Exception { + public void asyncInvoke(CRow row, ResultFuture resultFuture) throws Exception { + CRow input = new CRow(Row.copy(row.row()), row.change()); preInvoke(input, resultFuture); Map inputParams = parseInputParam(input); if(MapUtils.isEmpty(inputParams)){ 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 71652042c..fbb0aa291 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 @@ -144,7 +144,7 @@ public void accept(Map values) { dealCacheData(key,CacheObj.buildCacheObj(ECacheContentType.MultiLine, values)); resultFuture.complete(Collections.singleton(new CRow(row, input.change()))); } catch (Exception e) { - dealFillDataError(resultFuture, e, input); + dealFillDataError(input, resultFuture, e); } } else { dealMissKey(input, resultFuture); From 46598c486160b88ecee2584db8be3ef9a6bba459 Mon Sep 17 00:00:00 2001 From: dapeng Date: Mon, 18 May 2020 10:43:31 +0800 Subject: [PATCH 108/109] =?UTF-8?q?=E4=BF=AE=E5=A4=8D=E5=BC=82=E6=AD=A5?= =?UTF-8?q?=E9=98=BB=E5=A1=9E=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../clickhouse/ClickhouseAsyncReqRow.java | 6 ++++ .../flink/sql/side/db2/Db2AsyncReqRow.java | 6 ++++ .../sql/side/impala/ImpalaAsyncReqRow.java | 6 ++++ .../sql/side/mysql/MysqlAsyncReqRow.java | 6 ++++ .../sql/side/oracle/OracleAsyncReqRow.java | 6 ++++ .../sql/side/polardb/PolardbAsyncReqRow.java | 6 ++++ .../postgresql/PostgresqlAsyncReqRow.java | 6 ++++ .../sql/side/rdb/async/RdbAsyncReqRow.java | 28 +++++++++---------- .../side/sqlserver/SqlserverAsyncReqRow.java | 6 ++++ 9 files changed, 62 insertions(+), 14 deletions(-) diff --git a/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java b/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java index 9da7d8eaa..d279973dc 100644 --- a/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java +++ b/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.side.clickhouse; +import com.dtstack.flink.sql.factory.DTThreadFactory; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.AbstractSideTableInfo; @@ -32,6 +33,9 @@ import org.apache.flink.configuration.Configuration; import java.util.List; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; public class ClickhouseAsyncReqRow extends RdbAsyncReqRow { @@ -63,6 +67,8 @@ public void open(Configuration parameters) throws Exception { vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); setRdbSqlClient(JDBCClient.createNonShared(vertx, clickhouseClientConfig)); + setExecutor(new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(10), new DTThreadFactory("clickhouseAsyncExec"))); } } diff --git a/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java b/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java index 8e7275977..3a348df29 100644 --- a/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java +++ b/db2/db2-side/db2-async-side/src/main/java/com/dtstack/flink/sql/side/db2/Db2AsyncReqRow.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.side.db2; +import com.dtstack.flink.sql.factory.DTThreadFactory; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.AbstractSideTableInfo; @@ -33,6 +34,9 @@ import org.slf4j.LoggerFactory; import java.util.List; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; /** * Reason: @@ -75,6 +79,8 @@ public void open(Configuration parameters) throws Exception { vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); setRdbSqlClient(JDBCClient.createNonShared(vertx, db2lientConfig)); + setExecutor(new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(10), new DTThreadFactory("dbAsyncExec"))); } } 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 b5e0ed5fb..3f2172f6d 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,6 +18,7 @@ package com.dtstack.flink.sql.side.impala; +import com.dtstack.flink.sql.factory.DTThreadFactory; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.AbstractSideTableInfo; @@ -35,6 +36,9 @@ import java.io.IOException; import java.util.List; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; /** * Date: 2019/11/12 @@ -79,6 +83,8 @@ public void open(Configuration parameters) throws Exception { vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); setRdbSqlClient(JDBCClient.createNonShared(vertx, impalaClientConfig)); + setExecutor(new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(10), new DTThreadFactory("impalaAsyncExec"))); } diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index d52f1918f..4ac7e8ba5 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.side.mysql; +import com.dtstack.flink.sql.factory.DTThreadFactory; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.AbstractSideTableInfo; @@ -32,6 +33,9 @@ import org.apache.flink.configuration.Configuration; import java.util.List; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; /** * Mysql dim table @@ -72,6 +76,8 @@ public void open(Configuration parameters) throws Exception { vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); setRdbSqlClient(JDBCClient.createNonShared(vertx, mysqlClientConfig)); + setExecutor(new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(10), new DTThreadFactory("mysqlAsyncExec"))); } } diff --git a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java index 938fd870c..53d6ff896 100644 --- a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.side.oracle; +import com.dtstack.flink.sql.factory.DTThreadFactory; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.AbstractSideTableInfo; @@ -32,6 +33,9 @@ import org.apache.flink.configuration.Configuration; import java.util.List; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; public class OracleAsyncReqRow extends RdbAsyncReqRow { @@ -64,5 +68,7 @@ public void open(Configuration parameters) throws Exception { vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); setRdbSqlClient(JDBCClient.createNonShared(vertx, oracleClientConfig)); + setExecutor(new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(10), new DTThreadFactory("oracleAsyncExec"))); } } diff --git a/polardb/polardb-side/polardb-async-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAsyncReqRow.java b/polardb/polardb-side/polardb-async-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAsyncReqRow.java index 13d25b111..63b80ef0d 100644 --- a/polardb/polardb-side/polardb-async-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAsyncReqRow.java +++ b/polardb/polardb-side/polardb-async-side/src/main/java/com/dtstack/flink/sql/side/polardb/PolardbAsyncReqRow.java @@ -17,6 +17,7 @@ */ package com.dtstack.flink.sql.side.polardb; +import com.dtstack.flink.sql.factory.DTThreadFactory; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.AbstractSideTableInfo; @@ -32,6 +33,9 @@ import org.slf4j.LoggerFactory; import java.util.List; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; /** * Date: 2019/12/20 @@ -71,5 +75,7 @@ public void open(Configuration parameters) throws Exception { vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); setRdbSqlClient(JDBCClient.createNonShared(vertx, mysqlClientConfig)); + setExecutor(new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(10), new DTThreadFactory("polardbAsyncExec"))); } } diff --git a/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncReqRow.java b/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncReqRow.java index 42aa39a51..cedcb9eca 100644 --- a/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncReqRow.java +++ b/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncReqRow.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.side.postgresql; +import com.dtstack.flink.sql.factory.DTThreadFactory; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.AbstractSideTableInfo; @@ -34,6 +35,9 @@ import org.slf4j.LoggerFactory; import java.util.List; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; /** * Date: 2019-08-11 @@ -72,6 +76,8 @@ public void open(Configuration parameters) throws Exception { vo.setWorkerPoolSize(rdbSideTableInfo.getAsyncPoolSize()); Vertx vertx = Vertx.vertx(vo); setRdbSqlClient(JDBCClient.createNonShared(vertx, pgClientConfig)); + setExecutor(new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(10), new DTThreadFactory("postgresqlAsyncExec"))); } } 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 7a6b45be6..e92ab282b 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 @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.side.rdb.async; import com.dtstack.flink.sql.enums.ECacheContentType; +import com.dtstack.flink.sql.factory.DTThreadFactory; import com.dtstack.flink.sql.side.BaseAsyncReqRow; import com.dtstack.flink.sql.side.BaseSideInfo; import com.dtstack.flink.sql.side.CacheMissVal; @@ -46,8 +47,7 @@ import java.time.Instant; import java.util.List; import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -84,6 +84,8 @@ public class RdbAsyncReqRow extends BaseAsyncReqRow { private AtomicBoolean connectionStatus = new AtomicBoolean(true); + private transient ThreadPoolExecutor executor; + public RdbAsyncReqRow(BaseSideInfo sideInfo) { super(sideInfo); init(sideInfo); @@ -112,18 +114,7 @@ public void handleAsyncInvoke(Map inputParams, CRow input, Resul Thread.sleep(100); } Map params = formatInputParam(inputParams); - rdbSqlClient.getConnection(conn -> { - if(conn.failed()){ - connectionStatus.set(false); - connectWithRetry(params, input, resultFuture, rdbSqlClient); - return; - } - connectionStatus.set(true); - ScheduledFuture timerFuture = registerTimer(input, resultFuture); - cancelTimerWhenComplete(resultFuture, timerFuture); - handleQuery(conn.result(), params, input, resultFuture); - }); - + executor.execute(() -> connectWithRetry(params, input, resultFuture, rdbSqlClient)); } private void connectWithRetry(Map inputParams, CRow input, ResultFuture resultFuture, SQLClient rdbSqlClient) { @@ -134,6 +125,7 @@ private void connectWithRetry(Map inputParams, CRow input, Resul rdbSqlClient.getConnection(conn -> { try { if(conn.failed()){ + connectionStatus.set(false); if(failCounter.getAndIncrement() % 1000 == 0){ LOG.error("getConnection error", conn.cause()); } @@ -210,12 +202,20 @@ public void close() throws Exception { rdbSqlClient.close(); } + if(executor != null){ + executor.shutdown(); + } + } public void setRdbSqlClient(SQLClient rdbSqlClient) { this.rdbSqlClient = rdbSqlClient; } + public void setExecutor(ThreadPoolExecutor executor) { + this.executor = executor; + } + private void handleQuery(SQLConnection connection, Map inputParams, CRow input, ResultFuture resultFuture){ String key = buildCacheKey(inputParams); JsonArray params = new JsonArray(Lists.newArrayList(inputParams.values())); diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java index 1f4fbf13a..e8a22dfe8 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.side.sqlserver; +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; @@ -34,6 +35,9 @@ import org.slf4j.LoggerFactory; import java.util.List; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; /** * Date: 2019/11/26 @@ -74,5 +78,7 @@ public void open(Configuration parameters) throws Exception { vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); setRdbSqlClient(JDBCClient.createNonShared(vertx, sqlserverClientConfig)); + setExecutor(new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(10), new DTThreadFactory("sqlServerAsyncExec"))); } } From 34cc5825fcd4b950cbcde4fa04376868365318df Mon Sep 17 00:00:00 2001 From: dapeng Date: Tue, 19 May 2020 17:28:20 +0800 Subject: [PATCH 109/109] =?UTF-8?q?fix=20=E9=87=8D=E6=9E=84=E4=B9=8B?= =?UTF-8?q?=E5=90=8E=E7=9A=84=E7=BC=93=E5=AD=98=E6=A0=BC=E5=BC=8F=E9=97=AE?= =?UTF-8?q?=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/side/BaseAsyncReqRow.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 0d8691d1c..094f5a6ea 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 @@ -203,7 +203,7 @@ private void invokeWithCache(Map inputParams, CRow input, Result return; }else if(ECacheContentType.SingleLine == val.getType()){ try { - Row row = fillData(input.row(), val); + Row row = fillData(input.row(), val.getContent()); resultFuture.complete(Collections.singleton(new CRow(row, input.change()))); } catch (Exception e) { dealFillDataError(input, resultFuture, e);