From 6cfd7cc5ba66387a82131738073014f3567351fd Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Wed, 6 Mar 2019 19:02:14 +0800 Subject: [PATCH 01/96] Fix data loss bug --- .../flinkx/rdb/datareader/JdbcConfigKeys.java | 1 + .../flinkx/rdb/datareader/JdbcDataReader.java | 32 ++-- .../rdb/inputformat/JdbcInputFormat.java | 171 ++++++++++++++---- .../inputformat/JdbcInputFormatBuilder.java | 24 ++- .../rdb/inputformat/JdbcInputSplit.java | 69 +++++++ .../rdb/inputformat/MaximumAccumulator.java | 10 + .../com/dtstack/flinkx/rdb/util/DBUtil.java | 116 ++++++++++-- 7 files changed, 348 insertions(+), 75 deletions(-) create mode 100644 flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputSplit.java diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java index 4efc0c5ae5..4184bdab59 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java @@ -42,4 +42,5 @@ public class JdbcConfigKeys { public static final String KEY_START_LOCATION = "startLocation"; + public static final String KEY_REALTIME_INCRE_SYNC = "realTimeIncreSync"; } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java index f3c2f6504e..77b8b60f8c 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java @@ -27,9 +27,11 @@ import com.dtstack.flinkx.rdb.util.DBUtil; import com.dtstack.flinkx.reader.DataReader; import com.dtstack.flinkx.reader.MetaColumn; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; + import java.util.List; /** @@ -66,6 +68,8 @@ public class JdbcDataReader extends DataReader { protected int queryTimeOut; + protected boolean realTimeIncreSync; + public void setDatabaseInterface(DatabaseInterface databaseInterface) { this.databaseInterface = databaseInterface; } @@ -90,6 +94,9 @@ public JdbcDataReader(DataTransferConfig config, StreamExecutionEnvironment env) splitKey = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_SPLIK_KEY); increColumn = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_INCRE_COLUMN); startLocation = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_START_LOCATION,null); + + realTimeIncreSync = readerConfig.getParameter().getBooleanVal(JdbcConfigKeys.KEY_REALTIME_INCRE_SYNC,true); + realTimeIncreSync = StringUtils.isNotEmpty(increColumn); } @Override @@ -109,28 +116,25 @@ public DataStream readData() { builder.setFetchSize(fetchSize == 0 ? databaseInterface.getFetchSize() : fetchSize); builder.setQueryTimeOut(queryTimeOut == 0 ? databaseInterface.getQueryTimeout() : queryTimeOut); builder.setIncreCol(increColumn); + builder.setIncreColType(getIncrementColType()); builder.setStartLocation(startLocation); + builder.setSplitKey(splitKey); + builder.setNumPartitions(numPartitions); + builder.setRealTimeIncreSync(realTimeIncreSync); - boolean isSplitByKey = false; - if(numPartitions > 1 && splitKey != null && splitKey.trim().length() != 0) { - builder.setParameterValues(DBUtil.getParameterValues(numPartitions)); - isSplitByKey = true; - } - - if(increColumn != null){ - String increColType = getIncreColType(); - where = DBUtil.buildWhereSql(databaseInterface,increColType,where,increColumn,startLocation); - builder.setIncreColType(increColType); - } - - String query = DBUtil.getQuerySql(databaseInterface,table,metaColumns,splitKey,where,isSplitByKey); + boolean isSplitByKey = numPartitions > 1 && StringUtils.isNotEmpty(splitKey); + String query = DBUtil.getQuerySql(databaseInterface, table, metaColumns, splitKey, where, isSplitByKey, realTimeIncreSync); builder.setQuery(query); RichInputFormat format = builder.finish(); return createInput(format, (databaseInterface.getDatabaseType() + "reader").toLowerCase()); } - private String getIncreColType(){ + private String getIncrementColType(){ + if (StringUtils.isEmpty(increColumn)){ + return null; + } + for (MetaColumn metaColumn : metaColumns) { if(metaColumn.getName().equals(increColumn)){ return metaColumn.getType(); diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 94d932f207..8448d551b9 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -33,7 +33,6 @@ import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; import org.apache.flink.types.Row; @@ -80,8 +79,6 @@ public class JdbcInputFormat extends RichInputFormat { protected boolean hasNext; - protected Object[][] parameterValues; - protected int columnCount; protected String table; @@ -96,12 +93,18 @@ public class JdbcInputFormat extends RichInputFormat { protected String startLocation; + protected String splitKey; + private int increColIndex; protected int fetchSize; protected int queryTimeOut; + protected boolean realTimeIncreSync; + + protected int numPartitions; + protected StringAccumulator tableColAccumulator; protected MaximumAccumulator endLocationAccumulator; @@ -118,7 +121,12 @@ public void configure(Configuration configuration) { } - private void setMetric(){ + private void initMetric(InputSplit split){ + + if (StringUtils.isEmpty(increCol)){ + return; + } + Map> accumulatorMap = getRuntimeContext().getAllAccumulators(); if(!accumulatorMap.containsKey(Metrics.TABLE_COL)){ @@ -127,18 +135,25 @@ private void setMetric(){ getRuntimeContext().addAccumulator(Metrics.TABLE_COL,tableColAccumulator); } - if(!accumulatorMap.containsKey(Metrics.END_LOCATION)){ + String endLocation = ((JdbcInputSplit)split).getEndLocation(); + if(!accumulatorMap.containsKey(Metrics.END_LOCATION) && endLocation != null){ endLocationAccumulator = new MaximumAccumulator(); + + if(realTimeIncreSync){ + endLocationAccumulator.add(endLocation); + } + getRuntimeContext().addAccumulator(Metrics.END_LOCATION,endLocationAccumulator); } - if (startLocation != null){ - endLocationAccumulator.add(startLocation); - if(!accumulatorMap.containsKey(Metrics.START_LOCATION)){ - startLocationAccumulator = new StringAccumulator(); - startLocationAccumulator.add(startLocation); - getRuntimeContext().addAccumulator(Metrics.START_LOCATION,startLocationAccumulator); + if (!accumulatorMap.containsKey(Metrics.START_LOCATION) && startLocation != null){ + if(!realTimeIncreSync){ + endLocationAccumulator.add(startLocation); } + + startLocationAccumulator = new StringAccumulator(); + startLocationAccumulator.add(startLocation); + getRuntimeContext().addAccumulator(Metrics.START_LOCATION,startLocationAccumulator); } for (int i = 0; i < metaColumns.size(); i++) { @@ -152,20 +167,22 @@ private void setMetric(){ @Override public void openInternal(InputSplit inputSplit) throws IOException { try { - ClassUtil.forName(drivername, getClass().getClassLoader()); - dbConn = DBUtil.getConnection(dbURL, username, password); - dbConn.setAutoCommit(false); + LOG.info(inputSplit.toString()); - Statement statement = dbConn.createStatement(resultSetType, resultSetConcurrency); + initMetric(inputSplit); - if (inputSplit != null && parameterValues != null) { - String n = parameterValues[inputSplit.getSplitNumber()][0].toString(); - String m = parameterValues[inputSplit.getSplitNumber()][1].toString(); - queryTemplate = queryTemplate.replace("${N}",n).replace("${M}",m); + if(!canReadData(inputSplit)){ + LOG.warn("Not read data when the start location are equal to end location"); - LOG.warn(String.format("Executing '%s' with parameters %s", queryTemplate, Arrays.deepToString(parameterValues[inputSplit.getSplitNumber()]))); + hasNext = false; + return; } + ClassUtil.forName(drivername, getClass().getClassLoader()); + dbConn = DBUtil.getConnection(dbURL, username, password); + dbConn.setAutoCommit(false); + Statement statement = dbConn.createStatement(resultSetType, resultSetConcurrency); + if(EDatabaseType.MySQL == databaseInterface.getDatabaseType()){ statement.setFetchSize(Integer.MIN_VALUE); } else { @@ -175,7 +192,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { if(EDatabaseType.Carbondata != databaseInterface.getDatabaseType()) { statement.setQueryTimeout(queryTimeOut); } - resultSet = statement.executeQuery(queryTemplate); + + String querySql = buildQuerySql(inputSplit); + resultSet = statement.executeQuery(querySql); columnCount = resultSet.getMetaData().getColumnCount(); hasNext = resultSet.next(); @@ -183,9 +202,6 @@ public void openInternal(InputSplit inputSplit) throws IOException { descColumnTypeList = DBUtil.analyzeTable(dbURL, username, password,databaseInterface,table,metaColumns); } - if(increCol != null){ - setMetric(); - } } catch (SQLException se) { throw new IllegalArgumentException("open() failed." + se.getMessage(), se); } @@ -201,23 +217,113 @@ public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOEx @Override public InputSplit[] createInputSplits(int minNumSplits) throws IOException { - if (parameterValues == null) { - return new GenericInputSplit[]{new GenericInputSplit(0, 1)}; + String endLocation = null; + if (realTimeIncreSync){ + endLocation = getEndLocation(); + + if(StringUtils.equals(startLocation, endLocation)){ + JdbcInputSplit[] splits = new JdbcInputSplit[1]; + splits[0] = new JdbcInputSplit(0, numPartitions, 0, startLocation, endLocation); + + return splits; + } } - GenericInputSplit[] ret = new GenericInputSplit[parameterValues.length]; - for (int i = 0; i < ret.length; i++) { - ret[i] = new GenericInputSplit(i, ret.length); + + JdbcInputSplit[] splits = new JdbcInputSplit[minNumSplits]; + for (int i = 0; i < minNumSplits; i++) { + splits[i] = new JdbcInputSplit(i, numPartitions, i, startLocation, endLocation); } - return ret; + + return splits; } + private boolean canReadData(InputSplit split){ + if (StringUtils.isEmpty(increCol)){ + return true; + } + + if (!realTimeIncreSync){ + return true; + } + + JdbcInputSplit jdbcInputSplit = (JdbcInputSplit) split; + return !StringUtils.equals(jdbcInputSplit.getStartLocation(), jdbcInputSplit.getEndLocation()); + } + + private String buildQuerySql(InputSplit inputSplit){ + String querySql = queryTemplate; + + if (inputSplit != null) { + JdbcInputSplit jdbcInputSplit = (JdbcInputSplit) inputSplit; + + if (StringUtils.isNotEmpty(splitKey)){ + querySql = queryTemplate.replace("${N}", String.valueOf(numPartitions)) + .replace("${M}", String.valueOf(jdbcInputSplit.getMod())); + } + + if (realTimeIncreSync){ + String incrementFilter = DBUtil.buildIncrementFilter(databaseInterface, increColType, increCol, + jdbcInputSplit.getStartLocation(), jdbcInputSplit.getEndLocation()); + + if(StringUtils.isNotEmpty(incrementFilter)){ + incrementFilter = " and " + incrementFilter; + } + + querySql = querySql.replace(DBUtil.INCREMENT_FILTER_PLACEHOLDER, incrementFilter); + } + } + + LOG.warn(String.format("Executing sql is: '%s'", querySql)); + + return querySql; + } + + private String getEndLocation() { + String maxValue = null; + Connection conn = null; + Statement st = null; + ResultSet rs = null; + try { + String queryMaxValueSql = String.format("select max(%s) as max_value from %s", increCol, table); + String startSql = DBUtil.buildStartLocationSql(databaseInterface, increColType, increCol, startLocation); + if(StringUtils.isNotEmpty(startSql)){ + queryMaxValueSql += " where " + startSql; + } + + LOG.info(String.format("Query max value sql is '%s'", queryMaxValueSql)); + + ClassUtil.forName(drivername, getClass().getClassLoader()); + conn = DBUtil.getConnection(dbURL, username, password); + st = conn.createStatement(); + rs = st.executeQuery(queryMaxValueSql); + if (rs.next()){ + if (ColumnType.isTimeType(increColType)){ + Timestamp increVal = rs.getTimestamp("max_value"); + if(increVal != null){ + maxValue = String.valueOf(getLocation(increVal)); + } + } else if(ColumnType.isNumberType(increColType)){ + maxValue = String.valueOf(rs.getLong("max_value")); + } else { + maxValue = rs.getString("max_value"); + } + } + + LOG.info(String.format("The max value of column %s is %s", increCol, maxValue)); + + return maxValue; + } catch (Throwable e){ + throw new RuntimeException("Get max value from " + table + " error",e); + } finally { + DBUtil.closeDBResources(rs,st,conn); + } + } @Override public InputSplitAssigner getInputSplitAssigner(InputSplit[] inputSplits) { return new DefaultInputSplitAssigner(inputSplits); } - @Override public boolean reachedEnd() throws IOException { return !hasNext; @@ -246,7 +352,7 @@ public Row nextRecordInternal(Row row) throws IOException { } } - if(increCol != null){ + if(increCol != null && !realTimeIncreSync){ if (ColumnType.isTimeType(increColType)){ Timestamp increVal = resultSet.getTimestamp(increColIndex + 1); if(increVal != null){ @@ -294,7 +400,6 @@ private long getLocation(Object increVal){ @Override public void closeInternal() throws IOException { DBUtil.closeDBResources(resultSet,statement,dbConn); - parameterValues = null; } } \ No newline at end of file diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java index d2abbb4819..86c5bd63ae 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java @@ -31,7 +31,6 @@ * Company: www.dtstack.com * @author huyifan.zju@163.com */ -@Deprecated public class JdbcInputFormatBuilder extends RichInputFormatBuilder { private JdbcInputFormat format; @@ -52,10 +51,6 @@ public void setQuery(String query) { format.queryTemplate = query; } - public void setParameterValues(Object[][] parameterValues) { - format.parameterValues = parameterValues; - } - public void setUsername(String username) { format.username = username; } @@ -96,24 +91,37 @@ public void setStartLocation(String startLocation){ format.startLocation = startLocation; } + public void setSplitKey(String splitKey){ + format.splitKey = splitKey; + } + public void setIncreColType(String increColType){ format.increColType = increColType; } + public void setRealTimeIncreSync(boolean realTimeIncreSync){ + format.realTimeIncreSync = realTimeIncreSync; + } + + public void setNumPartitions(int numPartitions){ + format.numPartitions = numPartitions; + } + @Override protected void checkFormat() { + if (format.username == null) { LOG.info("Username was not supplied separately."); } + if (format.password == null) { LOG.info("Password was not supplied separately."); } + if (format.dbURL == null) { throw new IllegalArgumentException("No database URL supplied"); } - if (format.queryTemplate == null) { - throw new IllegalArgumentException("No query supplied"); - } + if (format.drivername == null) { throw new IllegalArgumentException("No driver supplied"); } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputSplit.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputSplit.java new file mode 100644 index 0000000000..b665ed0aa1 --- /dev/null +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputSplit.java @@ -0,0 +1,69 @@ +/* + * 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.rdb.inputformat; + +import org.apache.flink.core.io.GenericInputSplit; + +/** + * @author jiangbo + * @explanation + * @date 2019/3/6 + */ +public class JdbcInputSplit extends GenericInputSplit { + + private int mod; + + private String endLocation; + + private String startLocation; + + /** + * Creates a generic input split with the given split number. + * + * @param partitionNumber The number of the split's partition. + * @param totalNumberOfPartitions The total number of the splits (partitions). + */ + public JdbcInputSplit(int partitionNumber, int totalNumberOfPartitions, int mod, String startLocation, String endLocation) { + super(partitionNumber, totalNumberOfPartitions); + this.mod = mod; + this.startLocation = startLocation; + this.endLocation = endLocation; + } + + public int getMod() { + return mod; + } + + public String getEndLocation() { + return endLocation; + } + + public String getStartLocation() { + return startLocation; + } + + @Override + public String toString() { + return "JdbcInputSplit{" + + "mod=" + mod + + ", endLocation='" + endLocation + '\'' + + ", startLocation='" + startLocation + '\'' + + '}'; + } +} diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java index 0ed159cb62..4a85f69de3 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java @@ -19,6 +19,7 @@ package com.dtstack.flinkx.rdb.inputformat; +import org.apache.commons.lang.StringUtils; import org.apache.flink.api.common.accumulators.Accumulator; import java.math.BigInteger; @@ -56,6 +57,15 @@ public void resetLocal() { @Override public void merge(Accumulator other) { + if (other == null || StringUtils.isEmpty(other.getLocalValue())){ + return; + } + + if (localValue == null){ + localValue = other.getLocalValue(); + return; + } + BigInteger local = new BigInteger(localValue); if(local.compareTo(new BigInteger(other.getLocalValue())) < 0){ localValue = other.getLocalValue(); diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java index 0c9d967881..6fd7177e46 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java @@ -23,10 +23,7 @@ import com.dtstack.flinkx.rdb.ParameterValuesProvider; import com.dtstack.flinkx.rdb.type.TypeConverterInterface; import com.dtstack.flinkx.reader.MetaColumn; -import com.dtstack.flinkx.util.ClassUtil; -import com.dtstack.flinkx.util.DateUtil; -import com.dtstack.flinkx.util.SysUtil; -import com.dtstack.flinkx.util.TelnetUtil; +import com.dtstack.flinkx.util.*; import org.apache.commons.lang.StringUtils; import org.apache.flink.types.Row; import org.slf4j.Logger; @@ -58,6 +55,8 @@ public class DBUtil { private static int MICRO_LENGTH = 16; private static int NANOS_LENGTH = 19; + public static final String INCREMENT_FILTER_PLACEHOLDER = "${incrementFilter}"; + private static Connection getConnectionInternal(String url, String username, String password) throws SQLException { Connection dbConn; synchronized (ClassUtil.lock_str){ @@ -377,13 +376,63 @@ public static Object clobToString(Object obj) throws Exception{ return dataStr; } - public static String buildWhereSql(DatabaseInterface databaseInterface,String increColType,String where, - String increCol,String startLocation){ - if (startLocation == null){ - return where; + public static String buildIncrementFilter(DatabaseInterface databaseInterface,String increColType,String increCol, + String startLocation,String endLocation){ + StringBuilder filter = new StringBuilder(); + + String startFilter = buildStartLocationSql(databaseInterface,increColType,increCol,startLocation); + if (StringUtils.isNotEmpty(startFilter)){ + filter.append(startFilter); + } + + String endFilter = buildEndLocationSql(databaseInterface,increColType,increCol,endLocation); + if (StringUtils.isNotEmpty(endFilter)){ + if (filter.length() > 0){ + filter.append(" and ").append(endFilter); + } else { + filter.append(endFilter); + } + } + + return filter.toString(); + } + + public static String buildEndLocationSql(DatabaseInterface databaseInterface,String increColType,String increCol,String endLocation){ + + if(StringUtils.isEmpty(endLocation)){ + return null; } - String increFilter; + String endLocationSql; + String endTimeStr; + + if(ColumnType.isTimeType(increColType) || (databaseInterface.getDatabaseType() == EDatabaseType.SQLServer && ColumnType.NVARCHAR.name().equals(increColType))){ + endTimeStr = getStartTimeStr(databaseInterface.getDatabaseType(),Long.parseLong(endLocation)); + + if (databaseInterface.getDatabaseType() == EDatabaseType.Oracle){ + endTimeStr = String.format("TO_TIMESTAMP('%s','YYYY-MM-DD HH24:MI:SS:FF6')",endTimeStr); + } else { + endTimeStr = String.format("'%s'",endTimeStr); + } + + endLocationSql = databaseInterface.quoteColumn(increCol) + " < " + endTimeStr; + } else if(ColumnType.isNumberType(increColType)){ + endLocationSql = databaseInterface.quoteColumn(increCol) + " < " + endLocation; + } else { + endTimeStr = String.format("'%s'",endLocation); + endLocationSql = databaseInterface.quoteColumn(increCol) + " < " + endTimeStr; + } + + return endLocationSql; + } + + public static String buildStartLocationSql(DatabaseInterface databaseInterface,String increColType,String increCol,String startLocation){ + + if(StringUtils.isEmpty(startLocation)){ + return null; + } + + String startLocationSql; String startTimeStr; if(ColumnType.isTimeType(increColType) || (databaseInterface.getDatabaseType() == EDatabaseType.SQLServer && ColumnType.NVARCHAR.name().equals(increColType))){ @@ -395,21 +444,39 @@ public static String buildWhereSql(DatabaseInterface databaseInterface,String in startTimeStr = String.format("'%s'",startTimeStr); } - increFilter = databaseInterface.quoteColumn(increCol) + " > " + startTimeStr; + startLocationSql = databaseInterface.quoteColumn(increCol) + " >= " + startTimeStr; } else if(ColumnType.isNumberType(increColType)){ - increFilter = databaseInterface.quoteColumn(increCol) + " > " + startLocation; + startLocationSql = databaseInterface.quoteColumn(increCol) + " >= " + startLocation; } else { startTimeStr = String.format("'%s'",startLocation); - increFilter = databaseInterface.quoteColumn(increCol) + " > " + startTimeStr; + startLocationSql = databaseInterface.quoteColumn(increCol) + " >= " + startTimeStr; } - if (where == null || where.length() == 0){ - where = increFilter; - } else { - where = where + " and " + increFilter; + return startLocationSql; + } + + public static String buildWhereSql(String where,String startSql,String endSql){ + StringBuilder whereBuilder = new StringBuilder(); + + if (StringUtils.isNotEmpty(where)){ + whereBuilder.append(where.trim()); } - return where; + if(StringUtils.isNotEmpty(startSql)){ + if(whereBuilder.toString().length() > 0){ + whereBuilder.append(" and "); + } + whereBuilder.append(startSql); + } + + if(StringUtils.isNotEmpty(endSql)){ + if(whereBuilder.toString().length() > 0){ + whereBuilder.append(" and "); + } + whereBuilder.append(endSql); + } + + return whereBuilder.toString(); } private static String getStartTimeStr(EDatabaseType databaseType,Long startLocation){ @@ -472,7 +539,12 @@ public static long getMillis(long startLocation){ } public static String getQuerySql(DatabaseInterface databaseInterface,String table,List metaColumns, - String splitKey,String where,boolean isSplitByKey) { + String splitKey,String customFilter,boolean isSplitByKey){ + return getQuerySql(databaseInterface, table, metaColumns, splitKey, customFilter, isSplitByKey, false); + } + + public static String getQuerySql(DatabaseInterface databaseInterface,String table,List metaColumns, + String splitKey,String customFilter,boolean isSplitByKey,boolean realTimeIncreSync) { StringBuilder sb = new StringBuilder(); List selectColumns = new ArrayList<>(); @@ -497,11 +569,15 @@ public static String getQuerySql(DatabaseInterface databaseInterface,String tabl filter.append(databaseInterface.getSplitFilter(splitKey)); } - if(where != null && where.trim().length() != 0) { + if(StringUtils.isNotEmpty(customFilter)) { if(filter.length() > 0) { filter.append(" AND "); } - filter.append(where); + filter.append(customFilter); + } + + if (realTimeIncreSync){ + filter.append(" ").append(INCREMENT_FILTER_PLACEHOLDER); } if(filter.length() != 0) { From a91a68ec8671cc75478d6c35045dec1438150cf0 Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Thu, 7 Mar 2019 14:51:06 +0800 Subject: [PATCH 02/96] Fix get max value error --- .../com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 8448d551b9..05dd839163 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -284,7 +284,8 @@ private String getEndLocation() { Statement st = null; ResultSet rs = null; try { - String queryMaxValueSql = String.format("select max(%s) as max_value from %s", increCol, table); + String queryMaxValueSql = String.format("select max(%s) as max_value from %s", + databaseInterface.quoteColumn(increCol), databaseInterface.quoteTable(table)); String startSql = DBUtil.buildStartLocationSql(databaseInterface, increColType, increCol, startLocation); if(StringUtils.isNotEmpty(startSql)){ queryMaxValueSql += " where " + startSql; From 62b848077ae5d698d8ee65b32572362e222b381e Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 11 Mar 2019 11:56:13 +0800 Subject: [PATCH 03/96] Initialization variable when close format --- .../dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java index f5a9565c90..d5114ad336 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java @@ -240,7 +240,11 @@ public HdfsParquetSplit[] createInputSplits(int minNumSplits) throws IOException public void closeInternal() throws IOException { if (currentFileReader != null){ currentFileReader.close(); + currentFileReader = null; } + + currentLine = null; + currentFileIndex = 0; } private String longToDecimalStr(long value,int scale){ From 837a3f17546862a16dea6d9dee6841e67874f25d Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 11 Mar 2019 16:17:21 +0800 Subject: [PATCH 04/96] Fix build sql error --- .../java/com/dtstack/flinkx/rdb/util/DBUtil.java | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java index 6fd7177e46..7bb5a07497 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java @@ -562,26 +562,24 @@ public static String getQuerySql(DatabaseInterface databaseInterface,String tabl sb.append("SELECT ").append(StringUtils.join(selectColumns,",")).append(" FROM "); sb.append(databaseInterface.quoteTable(table)); + sb.append(" WHERE 1=1 "); StringBuilder filter = new StringBuilder(); if(isSplitByKey) { - filter.append(databaseInterface.getSplitFilter(splitKey)); + filter.append(" AND ").append(databaseInterface.getSplitFilter(splitKey)); } if(StringUtils.isNotEmpty(customFilter)) { - if(filter.length() > 0) { - filter.append(" AND "); - } - filter.append(customFilter); + filter.append(" AND ").append(customFilter); } if (realTimeIncreSync){ filter.append(" ").append(INCREMENT_FILTER_PLACEHOLDER); } - if(filter.length() != 0) { - sb.append(" WHERE ").append(filter); + if(filter.length() > 0) { + sb.append(filter); } return sb.toString(); From c990acbabf5fa02300f563e39d9f94c28c1d3c2e Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 11 Mar 2019 16:38:22 +0800 Subject: [PATCH 05/96] Fix get max value error --- .../rdb/inputformat/MaximumAccumulator.java | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java index 4a85f69de3..3b6307a8db 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java @@ -20,6 +20,7 @@ package com.dtstack.flinkx.rdb.inputformat; import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.math.NumberUtils; import org.apache.flink.api.common.accumulators.Accumulator; import java.math.BigInteger; @@ -35,13 +36,19 @@ public class MaximumAccumulator implements Accumulator { @Override public void add(String value) { + if(StringUtils.isEmpty(value)){ + return; + } + if(localValue == null){ localValue = value; - } else { + } else if(NumberUtils.isNumber(localValue)){ BigInteger newVal = new BigInteger(value); if(newVal.compareTo(new BigInteger(localValue)) > 0){ localValue = value; } + } else { + localValue = localValue.compareTo(value) < 0 ? value : localValue; } } @@ -66,9 +73,13 @@ public void merge(Accumulator other) { return; } - BigInteger local = new BigInteger(localValue); - if(local.compareTo(new BigInteger(other.getLocalValue())) < 0){ - localValue = other.getLocalValue(); + if(NumberUtils.isNumber(localValue)){ + BigInteger local = new BigInteger(localValue); + if(local.compareTo(new BigInteger(other.getLocalValue())) < 0){ + localValue = other.getLocalValue(); + } + } else { + localValue = localValue.compareTo(other.getLocalValue()) < 0 ? other.getLocalValue() : localValue; } } From 8f65d70364ed207e82cbf15f3204aff26a784c03 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 11 Mar 2019 16:54:21 +0800 Subject: [PATCH 06/96] Fix type convert error --- .../com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java index 7da4847f35..12ddcb8d9b 100644 --- a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java +++ b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java @@ -137,9 +137,14 @@ private Record row2record(Row row, String[] columnTypes) throws WriteRecordExcep case INT: case TINYINT: case SMALLINT: + record.set(i,Integer.parseInt(rowData)); + break; case BIGINT: record.setBigint(i, Long.valueOf(rowData)); break; + case FLOAT: + record.set(i, Float.parseFloat(rowData)); + break; case DOUBLE: record.setDouble(i, Double.valueOf(rowData)); break; @@ -155,7 +160,7 @@ private Record row2record(Row row, String[] columnTypes) throws WriteRecordExcep record.setDatetime(i, DateUtil.columnToTimestamp(column,null)); break; default: - throw new IllegalArgumentException(); + record.set(i,column); } } From 66f329bb3925e5be0cff80f79e80d189c8363246 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 11 Mar 2019 20:00:14 +0800 Subject: [PATCH 07/96] Remove e.printStack() --- .../com/dtstack/flinkx/latch/MetricLatch.java | 18 ++++++++---------- .../dtstack/flinkx/reader/ByteRateLimiter.java | 3 +-- .../com/dtstack/flinkx/util/ValueUtil.java | 3 +-- .../dtstack/flinkx/writer/ErrorLimiter.java | 3 +-- .../dtstack/flinkx/ftp/StandardFtpHandler.java | 2 +- .../flinkx/hbase/reader/HbaseInputFormat.java | 3 +-- .../hdfs/writer/HdfsParquetOutputFormat.java | 3 +-- .../java/com/dtstack/flinkx/odps/OdpsUtil.java | 9 +-------- .../flinkx/odps/writer/OdpsOutputFormat.java | 2 +- .../com/dtstack/flinkx/rdb/util/DBUtil.java | 17 +---------------- 10 files changed, 17 insertions(+), 46 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java index 3200baaa4a..2b21dcd9ee 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java @@ -46,19 +46,23 @@ public class MetricLatch extends Latch { private RuntimeContext context; private static final String METRIC_PREFIX = "latch-"; - private boolean checkMonitorRoots() { + private void checkMonitorRoots() { boolean flag = false; int j = 0; + String msg = null; for(; j < monitorRoots.length; ++j) { String requestUrl = monitorRoots[j] + "/jobs/" + jobId + "/accumulators"; try(InputStream inputStream = URLUtil.open(requestUrl)) { flag = true; break; } catch (Exception e) { - e.printStackTrace(); + msg = e.getMessage(); } } - return flag; + + if (!flag){ + throw new IllegalArgumentException("Invalid monitor url:" + msg); + } } private int getIntMetricVal(String requestUrl) { @@ -96,13 +100,7 @@ public MetricLatch(RuntimeContext context, String monitors, String metricName) { } } - if(!checkMonitorRoots()) { - String msg = ""; - if(monitorRoots != null && monitorRoots.length >= 1) { - msg = monitorRoots[0]; - } - throw new RuntimeException("Invalid monitors: " + msg); - } + checkMonitorRoots(); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java index b3b6b3be96..7ef854b918 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java @@ -99,7 +99,6 @@ public ByteRateLimiter(RuntimeContext runtimeContext, String monitors, double ex try (InputStream inputStream = URLUtil.open(url)){ break; } catch (Exception e) { - e.printStackTrace(); LOG.error("connected error: " + url); } } @@ -174,7 +173,7 @@ public void start() { } } catch (Exception e) { - e.printStackTrace(); + LOG.error("Get metrics error:",e); } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ValueUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ValueUtil.java index baabd5174b..d1202865e8 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ValueUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ValueUtil.java @@ -40,10 +40,9 @@ public static Integer getInt(Object obj) { Method method = obj.getClass().getMethod("intValue"); return (int) method.invoke(obj); } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { - e.printStackTrace(); + throw new RuntimeException("Unable to convert " + obj + " into Interger",e); } } - throw new RuntimeException("Unable to convert " + obj + " into Interger"); } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java index 67df52929d..e187f518d0 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java @@ -105,7 +105,6 @@ public ErrorLimiter(RuntimeContext runtimeContext, String monitors, Integer maxE try (InputStream inputStream = URLUtil.open(url)){ break; } catch (Exception e) { - e.printStackTrace(); LOG.error("connected error: " + url); } } @@ -157,7 +156,7 @@ public void updateErrorInfo(){ } } } catch (Exception e) { - e.printStackTrace(); + LOG.error("Update data error:",e); } break; } diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/StandardFtpHandler.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/StandardFtpHandler.java index ca2d0414d4..0c02c9f8e6 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/StandardFtpHandler.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/StandardFtpHandler.java @@ -117,7 +117,7 @@ public boolean isDirExist(String directoryPath) { try { ftpClient.changeWorkingDirectory(originDir); } catch (IOException e) { - e.printStackTrace(); + LOG.error(e.getMessage()); } } } diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java index 277f6545eb..f5060b6597 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java @@ -270,9 +270,8 @@ public Row nextRecordInternal(Row row) throws IOException { } row.setField(i, col); } catch(Exception e) { - e.printStackTrace(); + throw new IOException("Couldn't read data:",e); } - } return row; diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java index 1f6510fb76..104b4507c6 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java @@ -74,7 +74,7 @@ public class HdfsParquetOutputFormat extends HdfsOutputFormat { try { cal.setTime(DateUtil.getDateFormatter().parse("1970-01-01")); } catch (Exception e){ - e.printStackTrace(); + throw new RuntimeException("Init calendar fail:",e); } } @@ -173,7 +173,6 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { writer.write(group); } catch (Exception e){ - e.printStackTrace(); if(i < row.getArity()) { throw new WriteRecordException(recordConvertDetailErrorMessage(i, row), e, i, row); } diff --git a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java index 1549f5dfce..dd977ca179 100644 --- a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java +++ b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java @@ -429,14 +429,7 @@ public static void truncateNonPartitionedTable(Odps odps, Table tab) { } public static Table getTable(Odps odps, String projectName, String tableName) { - Table table = odps.tables().get(projectName, tableName); -// try { -// table.getOwner(); -// } catch (Exception e) { -// e.printStackTrace(); -// throw new RuntimeException(e); -// } - return table; + return odps.tables().get(projectName, tableName); } public static TableTunnel.UploadSession createMasterTunnelUpload(final TableTunnel tunnel, final String projectName, diff --git a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java index 12ddcb8d9b..0a617cabc8 100644 --- a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java +++ b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java @@ -182,7 +182,7 @@ public void closeInternal() throws IOException { try { session.commit(); } catch (TunnelException e) { - e.printStackTrace(); + throw new IOException("commit session error:",e); } } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java index 7bb5a07497..5c89bb0173 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java @@ -192,27 +192,12 @@ public static void executeBatch(Connection dbConn, List sqls) { } stmt.executeBatch(); } catch (SQLException e) { - e.printStackTrace(); + throw new RuntimeException("execute batch sql error:{}",e); } finally { commit(dbConn); } } - public static void executeOneByOne(Connection dbConn, List sqls) { - if(sqls == null || sqls.size() == 0) { - return; - } - - try { - Statement stmt = dbConn.createStatement(); - for(String sql : sqls) { - stmt.execute(sql); - } - } catch (SQLException e) { - e.printStackTrace(); - } - } - public static Map> getPrimaryOrUniqueKeys(String table, Connection dbConn) throws SQLException { Map> keyMap = new HashMap<>(); DatabaseMetaData meta = dbConn.getMetaData(); From a89827d070318df9d1272f7f10b5e71084d6050a Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 11 Mar 2019 20:00:14 +0800 Subject: [PATCH 08/96] Remove e.printStack() --- .../com/dtstack/flinkx/latch/MetricLatch.java | 18 ++++++++---------- .../dtstack/flinkx/reader/ByteRateLimiter.java | 3 +-- .../com/dtstack/flinkx/util/ValueUtil.java | 3 +-- .../dtstack/flinkx/writer/ErrorLimiter.java | 3 +-- .../dtstack/flinkx/ftp/StandardFtpHandler.java | 2 +- .../flinkx/hbase/reader/HbaseInputFormat.java | 3 +-- .../hdfs/writer/HdfsParquetOutputFormat.java | 3 +-- .../java/com/dtstack/flinkx/odps/OdpsUtil.java | 9 +-------- .../flinkx/odps/writer/OdpsOutputFormat.java | 2 +- .../com/dtstack/flinkx/rdb/util/DBUtil.java | 17 +---------------- 10 files changed, 17 insertions(+), 46 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java index 3200baaa4a..2b21dcd9ee 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java @@ -46,19 +46,23 @@ public class MetricLatch extends Latch { private RuntimeContext context; private static final String METRIC_PREFIX = "latch-"; - private boolean checkMonitorRoots() { + private void checkMonitorRoots() { boolean flag = false; int j = 0; + String msg = null; for(; j < monitorRoots.length; ++j) { String requestUrl = monitorRoots[j] + "/jobs/" + jobId + "/accumulators"; try(InputStream inputStream = URLUtil.open(requestUrl)) { flag = true; break; } catch (Exception e) { - e.printStackTrace(); + msg = e.getMessage(); } } - return flag; + + if (!flag){ + throw new IllegalArgumentException("Invalid monitor url:" + msg); + } } private int getIntMetricVal(String requestUrl) { @@ -96,13 +100,7 @@ public MetricLatch(RuntimeContext context, String monitors, String metricName) { } } - if(!checkMonitorRoots()) { - String msg = ""; - if(monitorRoots != null && monitorRoots.length >= 1) { - msg = monitorRoots[0]; - } - throw new RuntimeException("Invalid monitors: " + msg); - } + checkMonitorRoots(); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java index b3b6b3be96..7ef854b918 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java @@ -99,7 +99,6 @@ public ByteRateLimiter(RuntimeContext runtimeContext, String monitors, double ex try (InputStream inputStream = URLUtil.open(url)){ break; } catch (Exception e) { - e.printStackTrace(); LOG.error("connected error: " + url); } } @@ -174,7 +173,7 @@ public void start() { } } catch (Exception e) { - e.printStackTrace(); + LOG.error("Get metrics error:",e); } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ValueUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ValueUtil.java index baabd5174b..d1202865e8 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ValueUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ValueUtil.java @@ -40,10 +40,9 @@ public static Integer getInt(Object obj) { Method method = obj.getClass().getMethod("intValue"); return (int) method.invoke(obj); } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { - e.printStackTrace(); + throw new RuntimeException("Unable to convert " + obj + " into Interger",e); } } - throw new RuntimeException("Unable to convert " + obj + " into Interger"); } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java index 67df52929d..e187f518d0 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java @@ -105,7 +105,6 @@ public ErrorLimiter(RuntimeContext runtimeContext, String monitors, Integer maxE try (InputStream inputStream = URLUtil.open(url)){ break; } catch (Exception e) { - e.printStackTrace(); LOG.error("connected error: " + url); } } @@ -157,7 +156,7 @@ public void updateErrorInfo(){ } } } catch (Exception e) { - e.printStackTrace(); + LOG.error("Update data error:",e); } break; } diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/StandardFtpHandler.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/StandardFtpHandler.java index ca2d0414d4..0c02c9f8e6 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/StandardFtpHandler.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/StandardFtpHandler.java @@ -117,7 +117,7 @@ public boolean isDirExist(String directoryPath) { try { ftpClient.changeWorkingDirectory(originDir); } catch (IOException e) { - e.printStackTrace(); + LOG.error(e.getMessage()); } } } diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java index 277f6545eb..f5060b6597 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java @@ -270,9 +270,8 @@ public Row nextRecordInternal(Row row) throws IOException { } row.setField(i, col); } catch(Exception e) { - e.printStackTrace(); + throw new IOException("Couldn't read data:",e); } - } return row; diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java index 1f6510fb76..104b4507c6 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java @@ -74,7 +74,7 @@ public class HdfsParquetOutputFormat extends HdfsOutputFormat { try { cal.setTime(DateUtil.getDateFormatter().parse("1970-01-01")); } catch (Exception e){ - e.printStackTrace(); + throw new RuntimeException("Init calendar fail:",e); } } @@ -173,7 +173,6 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { writer.write(group); } catch (Exception e){ - e.printStackTrace(); if(i < row.getArity()) { throw new WriteRecordException(recordConvertDetailErrorMessage(i, row), e, i, row); } diff --git a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java index 1549f5dfce..dd977ca179 100644 --- a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java +++ b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java @@ -429,14 +429,7 @@ public static void truncateNonPartitionedTable(Odps odps, Table tab) { } public static Table getTable(Odps odps, String projectName, String tableName) { - Table table = odps.tables().get(projectName, tableName); -// try { -// table.getOwner(); -// } catch (Exception e) { -// e.printStackTrace(); -// throw new RuntimeException(e); -// } - return table; + return odps.tables().get(projectName, tableName); } public static TableTunnel.UploadSession createMasterTunnelUpload(final TableTunnel tunnel, final String projectName, diff --git a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java index 7da4847f35..ea55239a22 100644 --- a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java +++ b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java @@ -177,7 +177,7 @@ public void closeInternal() throws IOException { try { session.commit(); } catch (TunnelException e) { - e.printStackTrace(); + throw new IOException("commit session error:",e); } } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java index 7bb5a07497..5c89bb0173 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java @@ -192,27 +192,12 @@ public static void executeBatch(Connection dbConn, List sqls) { } stmt.executeBatch(); } catch (SQLException e) { - e.printStackTrace(); + throw new RuntimeException("execute batch sql error:{}",e); } finally { commit(dbConn); } } - public static void executeOneByOne(Connection dbConn, List sqls) { - if(sqls == null || sqls.size() == 0) { - return; - } - - try { - Statement stmt = dbConn.createStatement(); - for(String sql : sqls) { - stmt.execute(sql); - } - } catch (SQLException e) { - e.printStackTrace(); - } - } - public static Map> getPrimaryOrUniqueKeys(String table, Connection dbConn) throws SQLException { Map> keyMap = new HashMap<>(); DatabaseMetaData meta = dbConn.getMetaData(); From 83cc1a22fe7c207c9f377e485859db84d4600a23 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 12 Mar 2019 10:55:59 +0800 Subject: [PATCH 09/96] add log --- .../java/com/dtstack/flinkx/latch/MetricLatch.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java index 2b21dcd9ee..75a315fbad 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java @@ -23,6 +23,9 @@ import com.google.gson.Gson; import com.google.gson.internal.LinkedTreeMap; import org.apache.flink.api.common.functions.RuntimeContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; @@ -39,6 +42,9 @@ * @author huyifan.zju@163.com */ public class MetricLatch extends Latch { + + public static Logger LOG = LoggerFactory.getLogger(MetricLatch.class); + private String metricName; private String[] monitorRoots; private String jobId; @@ -49,19 +55,19 @@ public class MetricLatch extends Latch { private void checkMonitorRoots() { boolean flag = false; int j = 0; - String msg = null; for(; j < monitorRoots.length; ++j) { String requestUrl = monitorRoots[j] + "/jobs/" + jobId + "/accumulators"; + LOG.info("Monitor url:" + requestUrl); try(InputStream inputStream = URLUtil.open(requestUrl)) { flag = true; break; } catch (Exception e) { - msg = e.getMessage(); + LOG.error("Open monitor url error:{}",e); } } if (!flag){ - throw new IllegalArgumentException("Invalid monitor url:" + msg); + throw new IllegalArgumentException("Invalid monitor url"); } } From aa8749f0af76fc592cd9509aa854a323446b0f8b Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 12 Mar 2019 11:16:39 +0800 Subject: [PATCH 10/96] Optimized Code --- .../main/java/com/dtstack/flinkx/latch/MetricLatch.java | 7 ++----- .../java/com/dtstack/flinkx/reader/ByteRateLimiter.java | 7 ++----- .../main/java/com/dtstack/flinkx/writer/ErrorLimiter.java | 4 ++-- 3 files changed, 6 insertions(+), 12 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java index 75a315fbad..4bc762f1ca 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java @@ -18,7 +18,6 @@ package com.dtstack.flinkx.latch; -import com.dtstack.flinkx.util.RetryUtil; import com.dtstack.flinkx.util.URLUtil; import com.google.gson.Gson; import com.google.gson.internal.LinkedTreeMap; @@ -26,14 +25,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; import java.io.Reader; -import java.net.URL; import java.util.List; import java.util.Map; -import java.util.concurrent.Callable; /** * Distributed implementation of Latch @@ -58,7 +54,8 @@ private void checkMonitorRoots() { for(; j < monitorRoots.length; ++j) { String requestUrl = monitorRoots[j] + "/jobs/" + jobId + "/accumulators"; LOG.info("Monitor url:" + requestUrl); - try(InputStream inputStream = URLUtil.open(requestUrl)) { + try{ + URLUtil.open(requestUrl); flag = true; break; } catch (Exception e) { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java index 7ef854b918..e7cbf79e2f 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java @@ -18,7 +18,6 @@ package com.dtstack.flinkx.reader; -import com.dtstack.flinkx.util.RetryUtil; import com.dtstack.flinkx.util.URLUtil; import com.google.common.util.concurrent.RateLimiter; import com.google.gson.Gson; @@ -28,14 +27,11 @@ import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; import java.io.Reader; -import java.net.URL; import java.util.List; import java.util.Map; -import java.util.concurrent.Callable; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -96,7 +92,8 @@ public ByteRateLimiter(RuntimeContext runtimeContext, String monitors, double ex for(; j < monitorUrls.length; ++j) { String url = monitorUrls[j]; LOG.info("monitor_url=" + url); - try (InputStream inputStream = URLUtil.open(url)){ + try { + URLUtil.open(url); break; } catch (Exception e) { LOG.error("connected error: " + url); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java index e187f518d0..39e4174f33 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java @@ -30,7 +30,6 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.io.Reader; -import java.net.URL; import java.util.List; import java.util.Map; import java.util.concurrent.Executors; @@ -102,7 +101,8 @@ public ErrorLimiter(RuntimeContext runtimeContext, String monitors, Integer maxE int j = 0; for(; j < monitorUrls.length; ++j) { String url = monitorUrls[j]; - try (InputStream inputStream = URLUtil.open(url)){ + try { + URLUtil.open(url); break; } catch (Exception e) { LOG.error("connected error: " + url); From cfeb2e3cf1cd7f494b48c62fe74cc3eb249d8017 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 12 Mar 2019 11:43:43 +0800 Subject: [PATCH 11/96] Revert and return exception msg --- .../src/main/java/com/dtstack/flinkx/latch/MetricLatch.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java index 75a315fbad..d68ec9c735 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java @@ -55,6 +55,7 @@ public class MetricLatch extends Latch { private void checkMonitorRoots() { boolean flag = false; int j = 0; + StringBuilder exceptionMsg = new StringBuilder(); for(; j < monitorRoots.length; ++j) { String requestUrl = monitorRoots[j] + "/jobs/" + jobId + "/accumulators"; LOG.info("Monitor url:" + requestUrl); @@ -62,12 +63,14 @@ private void checkMonitorRoots() { flag = true; break; } catch (Exception e) { + exceptionMsg.append("Monitor url:").append(requestUrl).append("\n"); + exceptionMsg.append("Error info:\n").append(e.getMessage()).append("\n"); LOG.error("Open monitor url error:{}",e); } } if (!flag){ - throw new IllegalArgumentException("Invalid monitor url"); + throw new IllegalArgumentException(exceptionMsg.toString()); } } From e4aec2d5ae1436fc6413e8a11a65e39c2ba51acb Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 12 Mar 2019 11:44:24 +0800 Subject: [PATCH 12/96] Revert "Optimized Code" --- .../main/java/com/dtstack/flinkx/latch/MetricLatch.java | 7 +++++-- .../java/com/dtstack/flinkx/reader/ByteRateLimiter.java | 7 +++++-- .../main/java/com/dtstack/flinkx/writer/ErrorLimiter.java | 4 ++-- 3 files changed, 12 insertions(+), 6 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java index 459a209400..d68ec9c735 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.latch; +import com.dtstack.flinkx.util.RetryUtil; import com.dtstack.flinkx.util.URLUtil; import com.google.gson.Gson; import com.google.gson.internal.LinkedTreeMap; @@ -25,11 +26,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; import java.io.Reader; +import java.net.URL; import java.util.List; import java.util.Map; +import java.util.concurrent.Callable; /** * Distributed implementation of Latch @@ -55,8 +59,7 @@ private void checkMonitorRoots() { for(; j < monitorRoots.length; ++j) { String requestUrl = monitorRoots[j] + "/jobs/" + jobId + "/accumulators"; LOG.info("Monitor url:" + requestUrl); - try{ - URLUtil.open(requestUrl); + try(InputStream inputStream = URLUtil.open(requestUrl)) { flag = true; break; } catch (Exception e) { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java index e7cbf79e2f..7ef854b918 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.reader; +import com.dtstack.flinkx.util.RetryUtil; import com.dtstack.flinkx.util.URLUtil; import com.google.common.util.concurrent.RateLimiter; import com.google.gson.Gson; @@ -27,11 +28,14 @@ import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; import java.io.Reader; +import java.net.URL; import java.util.List; import java.util.Map; +import java.util.concurrent.Callable; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -92,8 +96,7 @@ public ByteRateLimiter(RuntimeContext runtimeContext, String monitors, double ex for(; j < monitorUrls.length; ++j) { String url = monitorUrls[j]; LOG.info("monitor_url=" + url); - try { - URLUtil.open(url); + try (InputStream inputStream = URLUtil.open(url)){ break; } catch (Exception e) { LOG.error("connected error: " + url); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java index 39e4174f33..e187f518d0 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java @@ -30,6 +30,7 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.io.Reader; +import java.net.URL; import java.util.List; import java.util.Map; import java.util.concurrent.Executors; @@ -101,8 +102,7 @@ public ErrorLimiter(RuntimeContext runtimeContext, String monitors, Integer maxE int j = 0; for(; j < monitorUrls.length; ++j) { String url = monitorUrls[j]; - try { - URLUtil.open(url); + try (InputStream inputStream = URLUtil.open(url)){ break; } catch (Exception e) { LOG.error("connected error: " + url); From b46d8a8044c1f892dbf1eb1118533d8d3b3d4a13 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 12 Mar 2019 15:15:02 +0800 Subject: [PATCH 13/96] Fix bug of writing data to MaxCompute --- .../com/dtstack/flinkx/common/ColumnType.java | 2 +- .../flinkx/odps/writer/OdpsOutputFormat.java | 16 +++++++++++++--- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/common/ColumnType.java b/flinkx-core/src/main/java/com/dtstack/flinkx/common/ColumnType.java index 02f164ff36..e5f976f05d 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/common/ColumnType.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/common/ColumnType.java @@ -29,7 +29,7 @@ * @author huyifan.zju@163.com */ public enum ColumnType { - STRING, VARCHAR, CHAR,NVARCHAR,TEXT,KEYWORD, + STRING, VARCHAR, CHAR,NVARCHAR,TEXT,KEYWORD,BINARY, INT, MEDIUMINT, TINYINT, DATETIME, SMALLINT, BIGINT,LONG,SHORT,INTEGER, DOUBLE, FLOAT, BOOLEAN, diff --git a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java index 0a617cabc8..94bad7354e 100644 --- a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java +++ b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java @@ -20,6 +20,7 @@ import com.aliyun.odps.Odps; import com.aliyun.odps.Table; +import com.aliyun.odps.data.Binary; import com.aliyun.odps.data.Record; import com.aliyun.odps.tunnel.TableTunnel; import com.aliyun.odps.tunnel.TunnelException; @@ -134,16 +135,20 @@ private Record row2record(Row row, String[] columnTypes) throws WriteRecordExcep case BOOLEAN: record.setBoolean(i, Boolean.valueOf(rowData)); break; - case INT: case TINYINT: + record.set(i, Byte.valueOf(rowData)); + break; case SMALLINT: - record.set(i,Integer.parseInt(rowData)); + record.set(i, Short.valueOf(rowData)); + break; + case INT: + record.set(i, Integer.valueOf(rowData)); break; case BIGINT: record.setBigint(i, Long.valueOf(rowData)); break; case FLOAT: - record.set(i, Float.parseFloat(rowData)); + record.set(i, Float.valueOf(rowData)); break; case DOUBLE: record.setDouble(i, Double.valueOf(rowData)); @@ -156,9 +161,14 @@ private Record row2record(Row row, String[] columnTypes) throws WriteRecordExcep break; case DATE: case DATETIME: + record.set(i, DateUtil.columnToDate(column, null)); + break; case TIMESTAMP: record.setDatetime(i, DateUtil.columnToTimestamp(column,null)); break; + case BINARY: + record.set(i, new Binary(rowData.getBytes())); + break; default: record.set(i,column); } From 45bd40179d17857a7d2aa780f4a07359d65fbe8c Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 13 Mar 2019 20:56:14 +0800 Subject: [PATCH 14/96] Revert code --- .../flinkx/rdb/datareader/JdbcConfigKeys.java | 1 - .../flinkx/rdb/datareader/JdbcDataReader.java | 32 ++-- .../rdb/inputformat/JdbcInputFormat.java | 174 ++++-------------- .../inputformat/JdbcInputFormatBuilder.java | 24 +-- .../rdb/inputformat/JdbcInputSplit.java | 69 ------- .../rdb/inputformat/MaximumAccumulator.java | 27 +-- .../com/dtstack/flinkx/rdb/util/DBUtil.java | 141 +++++--------- 7 files changed, 101 insertions(+), 367 deletions(-) delete mode 100644 flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputSplit.java diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java index 4184bdab59..4efc0c5ae5 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java @@ -42,5 +42,4 @@ public class JdbcConfigKeys { public static final String KEY_START_LOCATION = "startLocation"; - public static final String KEY_REALTIME_INCRE_SYNC = "realTimeIncreSync"; } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java index 77b8b60f8c..f3c2f6504e 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java @@ -27,11 +27,9 @@ import com.dtstack.flinkx.rdb.util.DBUtil; import com.dtstack.flinkx.reader.DataReader; import com.dtstack.flinkx.reader.MetaColumn; -import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; - import java.util.List; /** @@ -68,8 +66,6 @@ public class JdbcDataReader extends DataReader { protected int queryTimeOut; - protected boolean realTimeIncreSync; - public void setDatabaseInterface(DatabaseInterface databaseInterface) { this.databaseInterface = databaseInterface; } @@ -94,9 +90,6 @@ public JdbcDataReader(DataTransferConfig config, StreamExecutionEnvironment env) splitKey = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_SPLIK_KEY); increColumn = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_INCRE_COLUMN); startLocation = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_START_LOCATION,null); - - realTimeIncreSync = readerConfig.getParameter().getBooleanVal(JdbcConfigKeys.KEY_REALTIME_INCRE_SYNC,true); - realTimeIncreSync = StringUtils.isNotEmpty(increColumn); } @Override @@ -116,25 +109,28 @@ public DataStream readData() { builder.setFetchSize(fetchSize == 0 ? databaseInterface.getFetchSize() : fetchSize); builder.setQueryTimeOut(queryTimeOut == 0 ? databaseInterface.getQueryTimeout() : queryTimeOut); builder.setIncreCol(increColumn); - builder.setIncreColType(getIncrementColType()); builder.setStartLocation(startLocation); - builder.setSplitKey(splitKey); - builder.setNumPartitions(numPartitions); - builder.setRealTimeIncreSync(realTimeIncreSync); - boolean isSplitByKey = numPartitions > 1 && StringUtils.isNotEmpty(splitKey); - String query = DBUtil.getQuerySql(databaseInterface, table, metaColumns, splitKey, where, isSplitByKey, realTimeIncreSync); + boolean isSplitByKey = false; + if(numPartitions > 1 && splitKey != null && splitKey.trim().length() != 0) { + builder.setParameterValues(DBUtil.getParameterValues(numPartitions)); + isSplitByKey = true; + } + + if(increColumn != null){ + String increColType = getIncreColType(); + where = DBUtil.buildWhereSql(databaseInterface,increColType,where,increColumn,startLocation); + builder.setIncreColType(increColType); + } + + String query = DBUtil.getQuerySql(databaseInterface,table,metaColumns,splitKey,where,isSplitByKey); builder.setQuery(query); RichInputFormat format = builder.finish(); return createInput(format, (databaseInterface.getDatabaseType() + "reader").toLowerCase()); } - private String getIncrementColType(){ - if (StringUtils.isEmpty(increColumn)){ - return null; - } - + private String getIncreColType(){ for (MetaColumn metaColumn : metaColumns) { if(metaColumn.getName().equals(increColumn)){ return metaColumn.getType(); diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 05dd839163..5da390da37 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -33,6 +33,7 @@ import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; import org.apache.flink.types.Row; @@ -79,6 +80,8 @@ public class JdbcInputFormat extends RichInputFormat { protected boolean hasNext; + protected Object[][] parameterValues; + protected int columnCount; protected String table; @@ -93,18 +96,12 @@ public class JdbcInputFormat extends RichInputFormat { protected String startLocation; - protected String splitKey; - private int increColIndex; protected int fetchSize; protected int queryTimeOut; - protected boolean realTimeIncreSync; - - protected int numPartitions; - protected StringAccumulator tableColAccumulator; protected MaximumAccumulator endLocationAccumulator; @@ -121,12 +118,7 @@ public void configure(Configuration configuration) { } - private void initMetric(InputSplit split){ - - if (StringUtils.isEmpty(increCol)){ - return; - } - + private void setMetric(){ Map> accumulatorMap = getRuntimeContext().getAllAccumulators(); if(!accumulatorMap.containsKey(Metrics.TABLE_COL)){ @@ -135,25 +127,18 @@ private void initMetric(InputSplit split){ getRuntimeContext().addAccumulator(Metrics.TABLE_COL,tableColAccumulator); } - String endLocation = ((JdbcInputSplit)split).getEndLocation(); - if(!accumulatorMap.containsKey(Metrics.END_LOCATION) && endLocation != null){ + if(!accumulatorMap.containsKey(Metrics.END_LOCATION)){ endLocationAccumulator = new MaximumAccumulator(); - - if(realTimeIncreSync){ - endLocationAccumulator.add(endLocation); - } - getRuntimeContext().addAccumulator(Metrics.END_LOCATION,endLocationAccumulator); } - if (!accumulatorMap.containsKey(Metrics.START_LOCATION) && startLocation != null){ - if(!realTimeIncreSync){ - endLocationAccumulator.add(startLocation); + if (startLocation != null){ + endLocationAccumulator.add(startLocation); + if(!accumulatorMap.containsKey(Metrics.START_LOCATION)){ + startLocationAccumulator = new StringAccumulator(); + startLocationAccumulator.add(startLocation); + getRuntimeContext().addAccumulator(Metrics.START_LOCATION,startLocationAccumulator); } - - startLocationAccumulator = new StringAccumulator(); - startLocationAccumulator.add(startLocation); - getRuntimeContext().addAccumulator(Metrics.START_LOCATION,startLocationAccumulator); } for (int i = 0; i < metaColumns.size(); i++) { @@ -167,22 +152,22 @@ private void initMetric(InputSplit split){ @Override public void openInternal(InputSplit inputSplit) throws IOException { try { - LOG.info(inputSplit.toString()); - - initMetric(inputSplit); - - if(!canReadData(inputSplit)){ - LOG.warn("Not read data when the start location are equal to end location"); - - hasNext = false; - return; - } - ClassUtil.forName(drivername, getClass().getClassLoader()); dbConn = DBUtil.getConnection(dbURL, username, password); dbConn.setAutoCommit(false); + Statement statement = dbConn.createStatement(resultSetType, resultSetConcurrency); + if (inputSplit != null && parameterValues != null) { + String n = parameterValues[inputSplit.getSplitNumber()][0].toString(); + String m = parameterValues[inputSplit.getSplitNumber()][1].toString(); + queryTemplate = queryTemplate.replace("${N}",n).replace("${M}",m); + + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("Executing '%s' with parameters %s", queryTemplate, Arrays.deepToString(parameterValues[inputSplit.getSplitNumber()]))); + } + } + if(EDatabaseType.MySQL == databaseInterface.getDatabaseType()){ statement.setFetchSize(Integer.MIN_VALUE); } else { @@ -192,9 +177,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { if(EDatabaseType.Carbondata != databaseInterface.getDatabaseType()) { statement.setQueryTimeout(queryTimeOut); } - - String querySql = buildQuerySql(inputSplit); - resultSet = statement.executeQuery(querySql); + resultSet = statement.executeQuery(queryTemplate); columnCount = resultSet.getMetaData().getColumnCount(); hasNext = resultSet.next(); @@ -202,6 +185,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { descColumnTypeList = DBUtil.analyzeTable(dbURL, username, password,databaseInterface,table,metaColumns); } + if(increCol != null){ + setMetric(); + } } catch (SQLException se) { throw new IllegalArgumentException("open() failed." + se.getMessage(), se); } @@ -217,114 +203,23 @@ public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOEx @Override public InputSplit[] createInputSplits(int minNumSplits) throws IOException { - String endLocation = null; - if (realTimeIncreSync){ - endLocation = getEndLocation(); - - if(StringUtils.equals(startLocation, endLocation)){ - JdbcInputSplit[] splits = new JdbcInputSplit[1]; - splits[0] = new JdbcInputSplit(0, numPartitions, 0, startLocation, endLocation); - - return splits; - } - } - - JdbcInputSplit[] splits = new JdbcInputSplit[minNumSplits]; - for (int i = 0; i < minNumSplits; i++) { - splits[i] = new JdbcInputSplit(i, numPartitions, i, startLocation, endLocation); - } - - return splits; - } - - private boolean canReadData(InputSplit split){ - if (StringUtils.isEmpty(increCol)){ - return true; + if (parameterValues == null) { + return new GenericInputSplit[]{new GenericInputSplit(0, 1)}; } - - if (!realTimeIncreSync){ - return true; - } - - JdbcInputSplit jdbcInputSplit = (JdbcInputSplit) split; - return !StringUtils.equals(jdbcInputSplit.getStartLocation(), jdbcInputSplit.getEndLocation()); - } - - private String buildQuerySql(InputSplit inputSplit){ - String querySql = queryTemplate; - - if (inputSplit != null) { - JdbcInputSplit jdbcInputSplit = (JdbcInputSplit) inputSplit; - - if (StringUtils.isNotEmpty(splitKey)){ - querySql = queryTemplate.replace("${N}", String.valueOf(numPartitions)) - .replace("${M}", String.valueOf(jdbcInputSplit.getMod())); - } - - if (realTimeIncreSync){ - String incrementFilter = DBUtil.buildIncrementFilter(databaseInterface, increColType, increCol, - jdbcInputSplit.getStartLocation(), jdbcInputSplit.getEndLocation()); - - if(StringUtils.isNotEmpty(incrementFilter)){ - incrementFilter = " and " + incrementFilter; - } - - querySql = querySql.replace(DBUtil.INCREMENT_FILTER_PLACEHOLDER, incrementFilter); - } + GenericInputSplit[] ret = new GenericInputSplit[parameterValues.length]; + for (int i = 0; i < ret.length; i++) { + ret[i] = new GenericInputSplit(i, ret.length); } - - LOG.warn(String.format("Executing sql is: '%s'", querySql)); - - return querySql; + return ret; } - private String getEndLocation() { - String maxValue = null; - Connection conn = null; - Statement st = null; - ResultSet rs = null; - try { - String queryMaxValueSql = String.format("select max(%s) as max_value from %s", - databaseInterface.quoteColumn(increCol), databaseInterface.quoteTable(table)); - String startSql = DBUtil.buildStartLocationSql(databaseInterface, increColType, increCol, startLocation); - if(StringUtils.isNotEmpty(startSql)){ - queryMaxValueSql += " where " + startSql; - } - - LOG.info(String.format("Query max value sql is '%s'", queryMaxValueSql)); - - ClassUtil.forName(drivername, getClass().getClassLoader()); - conn = DBUtil.getConnection(dbURL, username, password); - st = conn.createStatement(); - rs = st.executeQuery(queryMaxValueSql); - if (rs.next()){ - if (ColumnType.isTimeType(increColType)){ - Timestamp increVal = rs.getTimestamp("max_value"); - if(increVal != null){ - maxValue = String.valueOf(getLocation(increVal)); - } - } else if(ColumnType.isNumberType(increColType)){ - maxValue = String.valueOf(rs.getLong("max_value")); - } else { - maxValue = rs.getString("max_value"); - } - } - - LOG.info(String.format("The max value of column %s is %s", increCol, maxValue)); - - return maxValue; - } catch (Throwable e){ - throw new RuntimeException("Get max value from " + table + " error",e); - } finally { - DBUtil.closeDBResources(rs,st,conn); - } - } @Override public InputSplitAssigner getInputSplitAssigner(InputSplit[] inputSplits) { return new DefaultInputSplitAssigner(inputSplits); } + @Override public boolean reachedEnd() throws IOException { return !hasNext; @@ -353,7 +248,7 @@ public Row nextRecordInternal(Row row) throws IOException { } } - if(increCol != null && !realTimeIncreSync){ + if(increCol != null){ if (ColumnType.isTimeType(increColType)){ Timestamp increVal = resultSet.getTimestamp(increColIndex + 1); if(increVal != null){ @@ -401,6 +296,7 @@ private long getLocation(Object increVal){ @Override public void closeInternal() throws IOException { DBUtil.closeDBResources(resultSet,statement,dbConn); + parameterValues = null; } } \ No newline at end of file diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java index 86c5bd63ae..d2abbb4819 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java @@ -31,6 +31,7 @@ * Company: www.dtstack.com * @author huyifan.zju@163.com */ +@Deprecated public class JdbcInputFormatBuilder extends RichInputFormatBuilder { private JdbcInputFormat format; @@ -51,6 +52,10 @@ public void setQuery(String query) { format.queryTemplate = query; } + public void setParameterValues(Object[][] parameterValues) { + format.parameterValues = parameterValues; + } + public void setUsername(String username) { format.username = username; } @@ -91,37 +96,24 @@ public void setStartLocation(String startLocation){ format.startLocation = startLocation; } - public void setSplitKey(String splitKey){ - format.splitKey = splitKey; - } - public void setIncreColType(String increColType){ format.increColType = increColType; } - public void setRealTimeIncreSync(boolean realTimeIncreSync){ - format.realTimeIncreSync = realTimeIncreSync; - } - - public void setNumPartitions(int numPartitions){ - format.numPartitions = numPartitions; - } - @Override protected void checkFormat() { - if (format.username == null) { LOG.info("Username was not supplied separately."); } - if (format.password == null) { LOG.info("Password was not supplied separately."); } - if (format.dbURL == null) { throw new IllegalArgumentException("No database URL supplied"); } - + if (format.queryTemplate == null) { + throw new IllegalArgumentException("No query supplied"); + } if (format.drivername == null) { throw new IllegalArgumentException("No driver supplied"); } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputSplit.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputSplit.java deleted file mode 100644 index b665ed0aa1..0000000000 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputSplit.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.rdb.inputformat; - -import org.apache.flink.core.io.GenericInputSplit; - -/** - * @author jiangbo - * @explanation - * @date 2019/3/6 - */ -public class JdbcInputSplit extends GenericInputSplit { - - private int mod; - - private String endLocation; - - private String startLocation; - - /** - * Creates a generic input split with the given split number. - * - * @param partitionNumber The number of the split's partition. - * @param totalNumberOfPartitions The total number of the splits (partitions). - */ - public JdbcInputSplit(int partitionNumber, int totalNumberOfPartitions, int mod, String startLocation, String endLocation) { - super(partitionNumber, totalNumberOfPartitions); - this.mod = mod; - this.startLocation = startLocation; - this.endLocation = endLocation; - } - - public int getMod() { - return mod; - } - - public String getEndLocation() { - return endLocation; - } - - public String getStartLocation() { - return startLocation; - } - - @Override - public String toString() { - return "JdbcInputSplit{" + - "mod=" + mod + - ", endLocation='" + endLocation + '\'' + - ", startLocation='" + startLocation + '\'' + - '}'; - } -} diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java index 3b6307a8db..dd8042f9aa 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java @@ -19,8 +19,6 @@ package com.dtstack.flinkx.rdb.inputformat; -import org.apache.commons.lang.StringUtils; -import org.apache.commons.lang3.math.NumberUtils; import org.apache.flink.api.common.accumulators.Accumulator; import java.math.BigInteger; @@ -36,19 +34,13 @@ public class MaximumAccumulator implements Accumulator { @Override public void add(String value) { - if(StringUtils.isEmpty(value)){ - return; - } - if(localValue == null){ localValue = value; - } else if(NumberUtils.isNumber(localValue)){ + } else { BigInteger newVal = new BigInteger(value); if(newVal.compareTo(new BigInteger(localValue)) > 0){ localValue = value; } - } else { - localValue = localValue.compareTo(value) < 0 ? value : localValue; } } @@ -64,22 +56,9 @@ public void resetLocal() { @Override public void merge(Accumulator other) { - if (other == null || StringUtils.isEmpty(other.getLocalValue())){ - return; - } - - if (localValue == null){ + BigInteger local = new BigInteger(localValue); + if(local.compareTo(new BigInteger(other.getLocalValue())) > 0){ localValue = other.getLocalValue(); - return; - } - - if(NumberUtils.isNumber(localValue)){ - BigInteger local = new BigInteger(localValue); - if(local.compareTo(new BigInteger(other.getLocalValue())) < 0){ - localValue = other.getLocalValue(); - } - } else { - localValue = localValue.compareTo(other.getLocalValue()) < 0 ? other.getLocalValue() : localValue; } } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java index 5c89bb0173..6266244b43 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java @@ -23,7 +23,10 @@ import com.dtstack.flinkx.rdb.ParameterValuesProvider; import com.dtstack.flinkx.rdb.type.TypeConverterInterface; import com.dtstack.flinkx.reader.MetaColumn; -import com.dtstack.flinkx.util.*; +import com.dtstack.flinkx.util.ClassUtil; +import com.dtstack.flinkx.util.DateUtil; +import com.dtstack.flinkx.util.SysUtil; +import com.dtstack.flinkx.util.TelnetUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.types.Row; import org.slf4j.Logger; @@ -55,8 +58,6 @@ public class DBUtil { private static int MICRO_LENGTH = 16; private static int NANOS_LENGTH = 19; - public static final String INCREMENT_FILTER_PLACEHOLDER = "${incrementFilter}"; - private static Connection getConnectionInternal(String url, String username, String password) throws SQLException { Connection dbConn; synchronized (ClassUtil.lock_str){ @@ -198,6 +199,21 @@ public static void executeBatch(Connection dbConn, List sqls) { } } + public static void executeOneByOne(Connection dbConn, List sqls) { + if(sqls == null || sqls.size() == 0) { + return; + } + + try { + Statement stmt = dbConn.createStatement(); + for(String sql : sqls) { + stmt.execute(sql); + } + } catch (SQLException e) { + e.printStackTrace(); + } + } + public static Map> getPrimaryOrUniqueKeys(String table, Connection dbConn) throws SQLException { Map> keyMap = new HashMap<>(); DatabaseMetaData meta = dbConn.getMetaData(); @@ -361,63 +377,13 @@ public static Object clobToString(Object obj) throws Exception{ return dataStr; } - public static String buildIncrementFilter(DatabaseInterface databaseInterface,String increColType,String increCol, - String startLocation,String endLocation){ - StringBuilder filter = new StringBuilder(); - - String startFilter = buildStartLocationSql(databaseInterface,increColType,increCol,startLocation); - if (StringUtils.isNotEmpty(startFilter)){ - filter.append(startFilter); - } - - String endFilter = buildEndLocationSql(databaseInterface,increColType,increCol,endLocation); - if (StringUtils.isNotEmpty(endFilter)){ - if (filter.length() > 0){ - filter.append(" and ").append(endFilter); - } else { - filter.append(endFilter); - } - } - - return filter.toString(); - } - - public static String buildEndLocationSql(DatabaseInterface databaseInterface,String increColType,String increCol,String endLocation){ - - if(StringUtils.isEmpty(endLocation)){ - return null; - } - - String endLocationSql; - String endTimeStr; - - if(ColumnType.isTimeType(increColType) || (databaseInterface.getDatabaseType() == EDatabaseType.SQLServer && ColumnType.NVARCHAR.name().equals(increColType))){ - endTimeStr = getStartTimeStr(databaseInterface.getDatabaseType(),Long.parseLong(endLocation)); - - if (databaseInterface.getDatabaseType() == EDatabaseType.Oracle){ - endTimeStr = String.format("TO_TIMESTAMP('%s','YYYY-MM-DD HH24:MI:SS:FF6')",endTimeStr); - } else { - endTimeStr = String.format("'%s'",endTimeStr); - } - - endLocationSql = databaseInterface.quoteColumn(increCol) + " < " + endTimeStr; - } else if(ColumnType.isNumberType(increColType)){ - endLocationSql = databaseInterface.quoteColumn(increCol) + " < " + endLocation; - } else { - endTimeStr = String.format("'%s'",endLocation); - endLocationSql = databaseInterface.quoteColumn(increCol) + " < " + endTimeStr; - } - - return endLocationSql; - } - - public static String buildStartLocationSql(DatabaseInterface databaseInterface,String increColType,String increCol,String startLocation){ - - if(StringUtils.isEmpty(startLocation)){ - return null; + public static String buildWhereSql(DatabaseInterface databaseInterface,String increColType,String where, + String increCol,String startLocation){ + if (startLocation == null){ + return where; } - String startLocationSql; + String increFilter; String startTimeStr; if(ColumnType.isTimeType(increColType) || (databaseInterface.getDatabaseType() == EDatabaseType.SQLServer && ColumnType.NVARCHAR.name().equals(increColType))){ @@ -429,39 +395,21 @@ public static String buildStartLocationSql(DatabaseInterface databaseInterface,S startTimeStr = String.format("'%s'",startTimeStr); } - startLocationSql = databaseInterface.quoteColumn(increCol) + " >= " + startTimeStr; + increFilter = databaseInterface.quoteColumn(increCol) + " > " + startTimeStr; } else if(ColumnType.isNumberType(increColType)){ - startLocationSql = databaseInterface.quoteColumn(increCol) + " >= " + startLocation; + increFilter = databaseInterface.quoteColumn(increCol) + " > " + startLocation; } else { startTimeStr = String.format("'%s'",startLocation); - startLocationSql = databaseInterface.quoteColumn(increCol) + " >= " + startTimeStr; - } - - return startLocationSql; - } - - public static String buildWhereSql(String where,String startSql,String endSql){ - StringBuilder whereBuilder = new StringBuilder(); - - if (StringUtils.isNotEmpty(where)){ - whereBuilder.append(where.trim()); + increFilter = databaseInterface.quoteColumn(increCol) + " > " + startTimeStr; } - if(StringUtils.isNotEmpty(startSql)){ - if(whereBuilder.toString().length() > 0){ - whereBuilder.append(" and "); - } - whereBuilder.append(startSql); - } - - if(StringUtils.isNotEmpty(endSql)){ - if(whereBuilder.toString().length() > 0){ - whereBuilder.append(" and "); - } - whereBuilder.append(endSql); + if (where == null || where.length() == 0){ + where = increFilter; + } else { + where = where + " and " + increFilter; } - return whereBuilder.toString(); + return where; } private static String getStartTimeStr(EDatabaseType databaseType,Long startLocation){ @@ -524,12 +472,7 @@ public static long getMillis(long startLocation){ } public static String getQuerySql(DatabaseInterface databaseInterface,String table,List metaColumns, - String splitKey,String customFilter,boolean isSplitByKey){ - return getQuerySql(databaseInterface, table, metaColumns, splitKey, customFilter, isSplitByKey, false); - } - - public static String getQuerySql(DatabaseInterface databaseInterface,String table,List metaColumns, - String splitKey,String customFilter,boolean isSplitByKey,boolean realTimeIncreSync) { + String splitKey,String where,boolean isSplitByKey) { StringBuilder sb = new StringBuilder(); List selectColumns = new ArrayList<>(); @@ -547,24 +490,22 @@ public static String getQuerySql(DatabaseInterface databaseInterface,String tabl sb.append("SELECT ").append(StringUtils.join(selectColumns,",")).append(" FROM "); sb.append(databaseInterface.quoteTable(table)); - sb.append(" WHERE 1=1 "); StringBuilder filter = new StringBuilder(); if(isSplitByKey) { - filter.append(" AND ").append(databaseInterface.getSplitFilter(splitKey)); + filter.append(databaseInterface.getSplitFilter(splitKey)); } - if(StringUtils.isNotEmpty(customFilter)) { - filter.append(" AND ").append(customFilter); - } - - if (realTimeIncreSync){ - filter.append(" ").append(INCREMENT_FILTER_PLACEHOLDER); + if(where != null && where.trim().length() != 0) { + if(filter.length() > 0) { + filter.append(" AND "); + } + filter.append(where); } - if(filter.length() > 0) { - sb.append(filter); + if(filter.length() != 0) { + sb.append(" WHERE ").append(filter); } return sb.toString(); From d696f40b67587fabebb873f4c54391d0aa7feb91 Mon Sep 17 00:00:00 2001 From: toutian Date: Thu, 14 Mar 2019 18:01:59 +0800 Subject: [PATCH 15/96] add group --- flinkx-core/pom.xml | 20 + .../groups/PipelineInputMetricGroup.java | 143 ++++++ .../groups/PipelineOutputMetricGroup.java | 147 ++++++ .../AbstractPrometheusReporter.java | 315 +++++++++++++ .../PrometheusPushGatewayReporter.java | 91 ++++ .../PrometheusPushGatewayReporterOptions.java | 62 +++ .../promethues/PrometheusReporter.java | 80 ++++ .../metrics/scope/PipelineScopeFormat.java | 40 ++ .../runtime/metrics/MetricRegistryImpl.java | 436 ++++++++++++++++++ .../runtime/metrics/scope/ScopeFormat.java | 264 +++++++++++ .../runtime/metrics/scope/ScopeFormats.java | 111 +++++ 11 files changed, 1709 insertions(+) create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineInputMetricGroup.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineOutputMetricGroup.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/AbstractPrometheusReporter.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporter.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporterOptions.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusReporter.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/PipelineScopeFormat.java create mode 100644 flinkx-core/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java create mode 100644 flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormat.java create mode 100644 flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormats.java diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index 161f3e2eed..97d9ca88b6 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -16,6 +16,7 @@ 2.10 1.5.4 ${basedir}/../dev + 0.5.0 @@ -101,6 +102,25 @@ 1.2.17 + + + io.prometheus + simpleclient + ${prometheus.version} + + + + io.prometheus + simpleclient_httpserver + ${prometheus.version} + + + + io.prometheus + simpleclient_pushgateway + ${prometheus.version} + + diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineInputMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineInputMetricGroup.java new file mode 100644 index 0000000000..a94f5e4177 --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineInputMetricGroup.java @@ -0,0 +1,143 @@ +/* + * 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.metrics.groups; + + +import com.dtstack.flinkx.constants.Metrics; +import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; +import org.apache.flink.runtime.metrics.groups.ComponentMetricGroup; +import org.apache.flink.runtime.metrics.scope.ScopeFormat; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Metric group that contains shareable pre-defined IO-related metrics. The metrics registration is + * forwarded to the parent task metric group. + */ +public class PipelineInputMetricGroup> extends ComponentMetricGroup { + + private final Counter numBytesInLocal; + private final Meter numBytesInRateLocal; + + private final SumCounter numRead; + + + private final String hostname; + private final String pluginType; + private final String pluginName; + private final String jobName; + + public PipelineInputMetricGroup(MetricRegistry registry, + String hostname, + String pluginType, + String pluginName, + String jobName) { + super(registry, registry.getScopeFormats().getPipelineScopeFormat().formatScope(hostname, pluginType, pluginName, jobName), null); + + this.hostname = hostname; + this.pluginType = pluginType; + this.pluginName = pluginName; + this.jobName = jobName; + + this.numBytesInLocal = counter(MetricNames.IO_NUM_BYTES_IN_LOCAL); + this.numBytesInRateLocal = meter(MetricNames.IO_NUM_BYTES_IN_LOCAL_RATE, new MeterView(numBytesInLocal, 60)); + + this.numRead = (SumCounter) counter(Metrics.NUM_READS, new SumCounter()); + } + + // ============================================================================================ + // Getters + // ============================================================================================ + + + public Counter getNumBytesInLocal() { + return numBytesInLocal; + } + + public Meter getNumBytesInRateLocal() { + return numBytesInRateLocal; + } + + public SumCounter getNumRead() { + return numRead; + } + + @Override + protected void putVariables(Map variables) { + variables.put(ScopeFormat.SCOPE_HOST, hostname); + variables.put(ScopeFormat.SCOPE_PLUGINE_TYPE, pluginType); + variables.put(ScopeFormat.SCOPE_PLUGINE_NAME, pluginName); + variables.put(ScopeFormat.SCOPE_JOB_NAME, jobName); + } + + @Override + protected Iterable subComponents() { + return null; + } + + @Override + protected String getGroupName(CharacterFilter filter) { + return "pipeline"; + } + + @Override + protected QueryScopeInfo createQueryServiceMetricInfo(CharacterFilter filter) { + return null; + } + + // ============================================================================================ + // Metric Reuse + // ============================================================================================ + public void reuseRecordsInputCounter(Counter numRecordsInCounter) { + this.numRead.addCounter(numRecordsInCounter); + } + + /** + * A {@link SimpleCounter} that can contain other {@link Counter}s. A call to {@link SumCounter#getCount()} returns + * the sum of this counters and all contained counters. + */ + private static class SumCounter extends SimpleCounter { + private final List internalCounters = new ArrayList<>(); + + SumCounter() { + } + + public void addCounter(Counter toAdd) { + internalCounters.add(toAdd); + } + + @Override + public long getCount() { + long sum = super.getCount(); + for (Counter counter : internalCounters) { + sum += counter.getCount(); + } + return sum; + } + } +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineOutputMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineOutputMetricGroup.java new file mode 100644 index 0000000000..b8aad9bfb5 --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineOutputMetricGroup.java @@ -0,0 +1,147 @@ +/* + * 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.metrics.groups; + + +import com.dtstack.flinkx.constants.Metrics; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; +import org.apache.flink.runtime.metrics.groups.ComponentMetricGroup; +import org.apache.flink.runtime.metrics.scope.ScopeFormat; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Metric group that contains shareable pre-defined IO-related metrics. The metrics registration is + * forwarded to the parent task metric group. + */ +public class PipelineOutputMetricGroup> extends ComponentMetricGroup { + + private final Counter numBytesOut; + private final SumCounter numRecordsOut; + + private final SumCounter numErrors; + private final SumCounter numNullErrors; + private final SumCounter numDuplicateErrors; + private final SumCounter numConversionErrors; + private final SumCounter numOtherErrors; + private final SumCounter numWrite; + + private final Meter numBytesOutRate; + private final Meter numRecordsOutRate; + + private final String hostname; + private final String pluginType; + private final String pluginName; + private final String jobName; + + public PipelineOutputMetricGroup(MetricRegistry registry, + String hostname, + String pluginType, + String pluginName, + String jobName) { + super(registry, registry.getScopeFormats().getPipelineScopeFormat().formatScope(hostname, pluginType, pluginName, jobName), null); + + this.hostname = hostname; + this.pluginType = pluginType; + this.pluginName = pluginName; + this.jobName = jobName; + + + this.numBytesOut = counter(MetricNames.IO_NUM_BYTES_OUT); + this.numBytesOutRate = meter(MetricNames.IO_NUM_BYTES_OUT_RATE, new MeterView(numBytesOut, 60)); + this.numRecordsOut = (SumCounter) counter(MetricNames.IO_NUM_RECORDS_OUT, new SumCounter()); + this.numRecordsOutRate = meter(MetricNames.IO_NUM_RECORDS_OUT_RATE, new MeterView(numRecordsOut, 60)); + + this.numErrors = (SumCounter) counter(Metrics.NUM_ERRORS, new SumCounter()); + this.numNullErrors = (SumCounter) counter(Metrics.NUM_NULL_ERRORS, new SumCounter()); + this.numDuplicateErrors = (SumCounter) counter(Metrics.NUM_DUPLICATE_ERRORS, new SumCounter()); + this.numConversionErrors = (SumCounter) counter(Metrics.NUM_CONVERSION_ERRORS, new SumCounter()); + this.numOtherErrors = (SumCounter) counter(Metrics.NUM_OTHER_ERRORS, new SumCounter()); + this.numWrite = (SumCounter) counter(Metrics.NUM_WRITES, new SumCounter()); + } + + // ============================================================================================ + // Getters + // ============================================================================================ + + + + @Override + protected void putVariables(Map variables) { + variables.put(ScopeFormat.SCOPE_HOST, hostname); + variables.put(ScopeFormat.SCOPE_PLUGINE_TYPE, pluginType); + variables.put(ScopeFormat.SCOPE_PLUGINE_NAME, pluginName); + variables.put(ScopeFormat.SCOPE_JOB_NAME, jobName); + } + + @Override + protected Iterable subComponents() { + return null; + } + + @Override + protected String getGroupName(CharacterFilter filter) { + return "pipeline"; + } + + @Override + protected QueryScopeInfo createQueryServiceMetricInfo(CharacterFilter filter) { + return null; + } + + // ============================================================================================ + // Metric Reuse + // ============================================================================================ + public void reuseRecordsOutputCounter(Counter numRecordsOutCounter) { + this.numRecordsOut.addCounter(numRecordsOutCounter); + } + + /** + * A {@link SimpleCounter} that can contain other {@link Counter}s. A call to {@link SumCounter#getCount()} returns + * the sum of this counters and all contained counters. + */ + private static class SumCounter extends SimpleCounter { + private final List internalCounters = new ArrayList<>(); + + SumCounter() { + } + + public void addCounter(Counter toAdd) { + internalCounters.add(toAdd); + } + + @Override + public long getCount() { + long sum = super.getCount(); + for (Counter counter : internalCounters) { + sum += counter.getCount(); + } + return sum; + } + } +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/AbstractPrometheusReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/AbstractPrometheusReporter.java new file mode 100644 index 0000000000..3a0dd2653e --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/AbstractPrometheusReporter.java @@ -0,0 +1,315 @@ +/* + * 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.metrics.promethues; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; +import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; +import io.prometheus.client.Collector; +import io.prometheus.client.CollectorRegistry; +import org.apache.flink.metrics.CharacterFilter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; + +import static com.dtstack.flinkx.metrics.promethues.PrometheusPushGatewayReporterOptions.FILTER_LABEL_VALUE_CHARACTER; + +/** + * base prometheus reporter for prometheus metrics. + */ +public abstract class AbstractPrometheusReporter implements MetricReporter { + + protected final Logger log = LoggerFactory.getLogger(getClass()); + + private static final Pattern UNALLOWED_CHAR_PATTERN = Pattern.compile("[^a-zA-Z0-9:_]"); + private static final CharacterFilter CHARACTER_FILTER = new CharacterFilter() { + @Override + public String filterCharacters(String input) { + return replaceInvalidChars(input); + } + }; + + private static final char SCOPE_SEPARATOR = '_'; + private static final String SCOPE_PREFIX = "flinkx" + SCOPE_SEPARATOR; + + private final Map> collectorsWithCountByMetricName = new HashMap<>(); + + static String replaceInvalidChars(final String input) { + // https://prometheus.io/docs/instrumenting/writing_exporters/ + // Only [a-zA-Z0-9:_] are valid in metric names, any other characters should be sanitized to an underscore. + return UNALLOWED_CHAR_PATTERN.matcher(input).replaceAll("_"); + } + + private CharacterFilter labelValueCharactersFilter = CHARACTER_FILTER; + + @Override + public void open(MetricConfig config) { + boolean filterLabelValueCharacters = config.getBoolean( + FILTER_LABEL_VALUE_CHARACTER.key(), FILTER_LABEL_VALUE_CHARACTER.defaultValue()); + + if (!filterLabelValueCharacters) { + labelValueCharactersFilter = input -> input; + } + } + + @Override + public void close() { + CollectorRegistry.defaultRegistry.clear(); + } + + @Override + public void notifyOfAddedMetric(final Metric metric, final String metricName, final MetricGroup group) { + + List dimensionKeys = new LinkedList<>(); + List dimensionValues = new LinkedList<>(); + for (final Map.Entry dimension : group.getAllVariables().entrySet()) { + final String key = dimension.getKey(); + dimensionKeys.add(CHARACTER_FILTER.filterCharacters(key.substring(1, key.length() - 1))); + dimensionValues.add(labelValueCharactersFilter.filterCharacters(dimension.getValue())); + } + + final String scopedMetricName = getScopedName(metricName, group); + final String helpString = metricName + " (scope: " + getLogicalScope(group) + ")"; + + final Collector collector; + Integer count = 0; + + synchronized (this) { + if (collectorsWithCountByMetricName.containsKey(scopedMetricName)) { + final AbstractMap.SimpleImmutableEntry collectorWithCount = collectorsWithCountByMetricName.get(scopedMetricName); + collector = collectorWithCount.getKey(); + count = collectorWithCount.getValue(); + } else { + collector = createCollector(metric, dimensionKeys, dimensionValues, scopedMetricName, helpString); + try { + collector.register(); + } catch (Exception e) { + log.warn("There was a problem registering metric {}.", metricName, e); + } + } + addMetric(metric, dimensionValues, collector); + collectorsWithCountByMetricName.put(scopedMetricName, new AbstractMap.SimpleImmutableEntry<>(collector, count + 1)); + } + } + + private static String getScopedName(String metricName, MetricGroup group) { + return SCOPE_PREFIX + getLogicalScope(group) + SCOPE_SEPARATOR + CHARACTER_FILTER.filterCharacters(metricName); + } + + private Collector createCollector(Metric metric, List dimensionKeys, List dimensionValues, String scopedMetricName, String helpString) { + Collector collector; + if (metric instanceof Gauge || metric instanceof Counter || metric instanceof Meter) { + collector = io.prometheus.client.Gauge + .build() + .name(scopedMetricName) + .help(helpString) + .labelNames(toArray(dimensionKeys)) + .create(); + } else if (metric instanceof Histogram) { + collector = new HistogramSummaryProxy((Histogram) metric, scopedMetricName, helpString, dimensionKeys, dimensionValues); + } else { + log.warn("Cannot create collector for unknown metric type: {}. This indicates that the metric type is not supported by this reporter.", + metric.getClass().getName()); + collector = null; + } + return collector; + } + + private void addMetric(Metric metric, List dimensionValues, Collector collector) { + if (metric instanceof Gauge) { + ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((Gauge) metric), toArray(dimensionValues)); + } else if (metric instanceof Counter) { + ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((Counter) metric), toArray(dimensionValues)); + } else if (metric instanceof Meter) { + ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((Meter) metric), toArray(dimensionValues)); + } else if (metric instanceof Histogram) { + ((HistogramSummaryProxy) collector).addChild((Histogram) metric, dimensionValues); + } else { + log.warn("Cannot add unknown metric type: {}. This indicates that the metric type is not supported by this reporter.", + metric.getClass().getName()); + } + } + + private void removeMetric(Metric metric, List dimensionValues, Collector collector) { + if (metric instanceof Gauge) { + ((io.prometheus.client.Gauge) collector).remove(toArray(dimensionValues)); + } else if (metric instanceof Counter) { + ((io.prometheus.client.Gauge) collector).remove(toArray(dimensionValues)); + } else if (metric instanceof Meter) { + ((io.prometheus.client.Gauge) collector).remove(toArray(dimensionValues)); + } else if (metric instanceof Histogram) { + ((HistogramSummaryProxy) collector).remove(dimensionValues); + } else { + log.warn("Cannot remove unknown metric type: {}. This indicates that the metric type is not supported by this reporter.", + metric.getClass().getName()); + } + } + + @Override + public void notifyOfRemovedMetric(final Metric metric, final String metricName, final MetricGroup group) { + + List dimensionValues = new LinkedList<>(); + for (final Map.Entry dimension : group.getAllVariables().entrySet()) { + dimensionValues.add(labelValueCharactersFilter.filterCharacters(dimension.getValue())); + } + + final String scopedMetricName = getScopedName(metricName, group); + synchronized (this) { + final AbstractMap.SimpleImmutableEntry collectorWithCount = collectorsWithCountByMetricName.get(scopedMetricName); + final Integer count = collectorWithCount.getValue(); + final Collector collector = collectorWithCount.getKey(); + + removeMetric(metric, dimensionValues, collector); + + if (count == 1) { + try { + CollectorRegistry.defaultRegistry.unregister(collector); + } catch (Exception e) { + log.warn("There was a problem unregistering metric {}.", scopedMetricName, e); + } + collectorsWithCountByMetricName.remove(scopedMetricName); + } else { + collectorsWithCountByMetricName.put(scopedMetricName, new AbstractMap.SimpleImmutableEntry<>(collector, count - 1)); + } + } + } + + @SuppressWarnings("unchecked") + private static String getLogicalScope(MetricGroup group) { + return ((FrontMetricGroup>) group).getLogicalScope(CHARACTER_FILTER, SCOPE_SEPARATOR); + } + + io.prometheus.client.Gauge.Child gaugeFrom(Gauge gauge) { + return new io.prometheus.client.Gauge.Child() { + @Override + public double get() { + final Object value = gauge.getValue(); + if (value == null) { + log.debug("Gauge {} is null-valued, defaulting to 0.", gauge); + return 0; + } + if (value instanceof Double) { + return (double) value; + } + if (value instanceof Number) { + return ((Number) value).doubleValue(); + } + if (value instanceof Boolean) { + return ((Boolean) value) ? 1 : 0; + } + log.debug("Invalid type for Gauge {}: {}, only number types and booleans are supported by this reporter.", + gauge, value.getClass().getName()); + return 0; + } + }; + } + + private static io.prometheus.client.Gauge.Child gaugeFrom(Counter counter) { + return new io.prometheus.client.Gauge.Child() { + @Override + public double get() { + return (double) counter.getCount(); + } + }; + } + + private static io.prometheus.client.Gauge.Child gaugeFrom(Meter meter) { + return new io.prometheus.client.Gauge.Child() { + @Override + public double get() { + return meter.getRate(); + } + }; + } + + @VisibleForTesting + static class HistogramSummaryProxy extends Collector { + static final List QUANTILES = Arrays.asList(.5, .75, .95, .98, .99, .999); + + private final String metricName; + private final String helpString; + private final List labelNamesWithQuantile; + + private final Map, Histogram> histogramsByLabelValues = new HashMap<>(); + + HistogramSummaryProxy(final Histogram histogram, final String metricName, final String helpString, final List labelNames, final List labelValues) { + this.metricName = metricName; + this.helpString = helpString; + this.labelNamesWithQuantile = addToList(labelNames, "quantile"); + histogramsByLabelValues.put(labelValues, histogram); + } + + @Override + public List collect() { + // We cannot use SummaryMetricFamily because it is impossible to get a sum of all values (at least for Dropwizard histograms, + // whose snapshot's values array only holds a sample of recent values). + + List samples = new LinkedList<>(); + for (Map.Entry, Histogram> labelValuesToHistogram : histogramsByLabelValues.entrySet()) { + addSamples(labelValuesToHistogram.getKey(), labelValuesToHistogram.getValue(), samples); + } + return Collections.singletonList(new MetricFamilySamples(metricName, Type.SUMMARY, helpString, samples)); + } + + void addChild(final Histogram histogram, final List labelValues) { + histogramsByLabelValues.put(labelValues, histogram); + } + + void remove(final List labelValues) { + histogramsByLabelValues.remove(labelValues); + } + + private void addSamples(final List labelValues, final Histogram histogram, final List samples) { + samples.add(new MetricFamilySamples.Sample(metricName + "_count", + labelNamesWithQuantile.subList(0, labelNamesWithQuantile.size() - 1), labelValues, histogram.getCount())); + for (final Double quantile : QUANTILES) { + samples.add(new MetricFamilySamples.Sample(metricName, labelNamesWithQuantile, + addToList(labelValues, quantile.toString()), + histogram.getStatistics().getQuantile(quantile))); + } + } + } + + private static List addToList(List list, String element) { + final List result = new ArrayList<>(list); + result.add(element); + return result; + } + + private static String[] toArray(List list) { + return list.toArray(new String[list.size()]); + } +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporter.java new file mode 100644 index 0000000000..9fb982b07a --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporter.java @@ -0,0 +1,91 @@ +/* + * 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.metrics.promethues; + +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.metrics.reporter.Scheduled; +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.exporter.PushGateway; +import org.apache.flink.util.AbstractID; + +import java.io.IOException; + +import static com.dtstack.flinkx.metrics.promethues.PrometheusPushGatewayReporterOptions.DELETE_ON_SHUTDOWN; +import static com.dtstack.flinkx.metrics.promethues.PrometheusPushGatewayReporterOptions.HOST; +import static com.dtstack.flinkx.metrics.promethues.PrometheusPushGatewayReporterOptions.JOB_NAME; +import static com.dtstack.flinkx.metrics.promethues.PrometheusPushGatewayReporterOptions.PORT; +import static com.dtstack.flinkx.metrics.promethues.PrometheusPushGatewayReporterOptions.RANDOM_JOB_NAME_SUFFIX; + +/** + * {@link MetricReporter} that exports {@link Metric Metrics} via Prometheus {@link PushGateway}. + */ +public class PrometheusPushGatewayReporter extends AbstractPrometheusReporter implements Scheduled { + + private PushGateway pushGateway; + private String jobName; + private boolean deleteOnShutdown; + + @Override + public void open(MetricConfig config) { + super.open(config); + + String host = config.getString(HOST.key(), HOST.defaultValue()); + int port = config.getInteger(PORT.key(), PORT.defaultValue()); + String configuredJobName = config.getString(JOB_NAME.key(), JOB_NAME.defaultValue()); + boolean randomSuffix = config.getBoolean(RANDOM_JOB_NAME_SUFFIX.key(), RANDOM_JOB_NAME_SUFFIX.defaultValue()); + deleteOnShutdown = config.getBoolean(DELETE_ON_SHUTDOWN.key(), DELETE_ON_SHUTDOWN.defaultValue()); + + if (host == null || host.isEmpty() || port < 1) { + throw new IllegalArgumentException("Invalid host/port configuration. Host: " + host + " Port: " + port); + } + + if (randomSuffix) { + this.jobName = configuredJobName + new AbstractID(); + } else { + this.jobName = configuredJobName; + } + + + pushGateway = new PushGateway(host + ':' + port); + log.info("Configured PrometheusPushGatewayReporter with {host:{}, port:{}, jobName: {}, deleteOnShutdown:{}}", host, port, jobName, deleteOnShutdown); + } + + @Override + public void report() { + try { + pushGateway.push(CollectorRegistry.defaultRegistry, jobName); + } catch (Exception e) { + log.warn("Failed to push metrics to PushGateway with jobName {}.", jobName, e); + } + } + + @Override + public void close() { + if (deleteOnShutdown && pushGateway != null) { + try { + pushGateway.delete(jobName); + } catch (IOException e) { + log.warn("Failed to delete metrics from PushGateway with jobName {}.", jobName, e); + } + } + super.close(); + } +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporterOptions.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporterOptions.java new file mode 100644 index 0000000000..b6c2f9b132 --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporterOptions.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.flinkx.metrics.promethues; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** + * Config options for the {@link PrometheusPushGatewayReporter}. + */ +public class PrometheusPushGatewayReporterOptions { + + public static final ConfigOption HOST = ConfigOptions + .key("host") + .noDefaultValue() + .withDescription("The PushGateway server host."); + + public static final ConfigOption PORT = ConfigOptions + .key("port") + .defaultValue(-1) + .withDescription("The PushGateway server port."); + + public static final ConfigOption JOB_NAME = ConfigOptions + .key("jobName") + .defaultValue("") + .withDescription("The job name under which metrics will be pushed"); + + public static final ConfigOption RANDOM_JOB_NAME_SUFFIX = ConfigOptions + .key("randomJobNameSuffix") + .defaultValue(true) + .withDescription("Specifies whether a random suffix should be appended to the job name."); + + public static final ConfigOption DELETE_ON_SHUTDOWN = ConfigOptions + .key("deleteOnShutdown") + .defaultValue(true) + .withDescription("Specifies whether to delete metrics from the PushGateway on shutdown."); + + public static final ConfigOption FILTER_LABEL_VALUE_CHARACTER = ConfigOptions + .key("filterLabelValueCharacters") + .defaultValue(true) + .withDescription("Specifies whether to filter label value characters." + + " If enabled, all characters not matching [a-zA-Z0-9:_] will be removed," + + " otherwise no characters will be removed." + + " Before disabling this option please ensure that your" + + " label values meet the Prometheus requirements(https://prometheus.io/docs/concepts/data_model/#metric-names-and-label)s."); +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusReporter.java new file mode 100644 index 0000000000..67b6c05b8e --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusReporter.java @@ -0,0 +1,80 @@ +/* + * 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.metrics.promethues; + +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.util.NetUtils; +import org.apache.flink.util.Preconditions; +import io.prometheus.client.exporter.HTTPServer; + +import java.io.IOException; +import java.util.Iterator; + +/** + * {@link MetricReporter} that exports {@link Metric Metrics} via Prometheus. + */ +public class PrometheusReporter extends AbstractPrometheusReporter { + + static final String ARG_PORT = "port"; + private static final String DEFAULT_PORT = "9249"; + + private HTTPServer httpServer; + private int port; + + int getPort() { + Preconditions.checkState(httpServer != null, "Server has not been initialized."); + return port; + } + + @Override + public void open(MetricConfig config) { + super.open(config); + + String portsConfig = config.getString(ARG_PORT, DEFAULT_PORT); + Iterator ports = NetUtils.getPortRangeFromString(portsConfig); + + while (ports.hasNext()) { + int port = ports.next(); + try { + // internally accesses CollectorRegistry.defaultRegistry + httpServer = new HTTPServer(port); + this.port = port; + log.info("Started PrometheusReporter HTTP server on port {}.", port); + break; + } catch (IOException ioe) { //assume port conflict + log.debug("Could not start PrometheusReporter HTTP server on port {}.", port, ioe); + } + } + if (httpServer == null) { + throw new RuntimeException("Could not start PrometheusReporter HTTP server on any configured port. Ports: " + portsConfig); + } + } + + @Override + public void close() { + if (httpServer != null) { + httpServer.stop(); + } + + super.close(); + } + +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/PipelineScopeFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/PipelineScopeFormat.java new file mode 100644 index 0000000000..883bf93c7b --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/PipelineScopeFormat.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.metrics.scope; + + +import org.apache.flink.runtime.metrics.scope.ScopeFormat; + +public class PipelineScopeFormat extends ScopeFormat { + + public PipelineScopeFormat(String format) { + super(format, null, new String[]{ + SCOPE_HOST, + SCOPE_PLUGINE_TYPE, + SCOPE_PLUGINE_NAME, + SCOPE_JOB_NAME + }); + } + + public String[] formatScope(String hostname, String pluginType, String pluginName, String jobName) { + final String[] template = copyTemplate(); + final String[] values = {hostname, pluginType, pluginName, jobName}; + return bindVariables(template, values); + } +} diff --git a/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java b/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java new file mode 100644 index 0000000000..630211837d --- /dev/null +++ b/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java @@ -0,0 +1,436 @@ +/* + * 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.runtime.metrics; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.View; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.metrics.reporter.Scheduled; +import org.apache.flink.runtime.akka.ActorUtils; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.metrics.dump.MetricQueryService; +import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; +import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; +import org.apache.flink.runtime.metrics.scope.ScopeFormats; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.ExecutorUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.TimerTask; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +/** + * A MetricRegistry keeps track of all registered {@link Metric Metrics}. It serves as the + * connection between {@link MetricGroup MetricGroups} and {@link MetricReporter MetricReporters}. + */ +public class MetricRegistryImpl implements MetricRegistry { + static final Logger LOG = LoggerFactory.getLogger(MetricRegistryImpl.class); + + private final Object lock = new Object(); + + private final List reporters; + private final ScheduledExecutorService executor; + + private final ScopeFormats scopeFormats = ScopeFormats.fromConfig(new Configuration()); + private final char globalDelimiter; + private final List delimiters; + + private final CompletableFuture terminationFuture; + + @Nullable + private ActorRef queryService; + + @Nullable + private String metricQueryServicePath; + + private ViewUpdater viewUpdater; + + private boolean isShutdown; + + public MetricRegistryImpl createMetricRegistry() { + MetricRegistryConfiguration config = MetricRegistryConfiguration.fromConfiguration(new Configuration()); + return new MetricRegistryImpl(config); + } + + /** + * Creates a new MetricRegistry and starts the configured reporter. + */ + public MetricRegistryImpl(MetricRegistryConfiguration config) { + + this.globalDelimiter = config.getDelimiter(); + this.delimiters = new ArrayList<>(10); + this.terminationFuture = new CompletableFuture<>(); + this.isShutdown = false; + + // second, instantiate any custom configured reporters + this.reporters = new ArrayList<>(4); + + List> reporterConfigurations = config.getReporterConfigurations(); + + this.executor = Executors.newSingleThreadScheduledExecutor(new ExecutorThreadFactory("Flink-MetricRegistry")); + + this.queryService = null; + this.metricQueryServicePath = null; + + if (reporterConfigurations.isEmpty()) { + // no reporters defined + // by default, don't report anything + LOG.info("No metrics reporter configured, no metrics will be exposed/reported."); + } else { + // we have some reporters so + for (Tuple2 reporterConfiguration: reporterConfigurations) { + String namedReporter = reporterConfiguration.f0; + Configuration reporterConfig = reporterConfiguration.f1; + + final String className = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, null); + if (className == null) { + LOG.error("No reporter class set for reporter " + namedReporter + ". Metrics might not be exposed/reported."); + continue; + } + + try { + String configuredPeriod = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_INTERVAL_SUFFIX, null); + TimeUnit timeunit = TimeUnit.SECONDS; + long period = 10; + + if (configuredPeriod != null) { + try { + String[] interval = configuredPeriod.split(" "); + period = Long.parseLong(interval[0]); + timeunit = TimeUnit.valueOf(interval[1]); + } + catch (Exception e) { + LOG.error("Cannot parse report interval from config: " + configuredPeriod + + " - please use values like '10 SECONDS' or '500 MILLISECONDS'. " + + "Using default reporting interval."); + } + } + + Class reporterClass = Class.forName(className); + MetricReporter reporterInstance = (MetricReporter) reporterClass.newInstance(); + + MetricConfig metricConfig = new MetricConfig(); + reporterConfig.addAllToProperties(metricConfig); + LOG.info("Configuring {} with {}.", namedReporter, metricConfig); + reporterInstance.open(metricConfig); + + if (reporterInstance instanceof Scheduled) { + LOG.info("Periodically reporting metrics in intervals of {} {} for reporter {} of type {}.", period, timeunit.name(), namedReporter, className); + + executor.scheduleWithFixedDelay( + new MetricRegistryImpl.ReporterTask((Scheduled) reporterInstance), period, period, timeunit); + } else { + LOG.info("Reporting metrics for reporter {} of type {}.", namedReporter, className); + } + reporters.add(reporterInstance); + + String delimiterForReporter = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, String.valueOf(globalDelimiter)); + if (delimiterForReporter.length() != 1) { + LOG.warn("Failed to parse delimiter '{}' for reporter '{}', using global delimiter '{}'.", delimiterForReporter, namedReporter, globalDelimiter); + delimiterForReporter = String.valueOf(globalDelimiter); + } + this.delimiters.add(delimiterForReporter.charAt(0)); + } + catch (Throwable t) { + LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", namedReporter, t); + } + } + } + } + + /** + * Initializes the MetricQueryService. + * + * @param actorSystem ActorSystem to create the MetricQueryService on + * @param resourceID resource ID used to disambiguate the actor name + */ + public void startQueryService(ActorSystem actorSystem, ResourceID resourceID) { + synchronized (lock) { + Preconditions.checkState(!isShutdown(), "The metric registry has already been shut down."); + + try { + queryService = MetricQueryService.startMetricQueryService(actorSystem, resourceID); + metricQueryServicePath = AkkaUtils.getAkkaURL(actorSystem, queryService); + } catch (Exception e) { + LOG.warn("Could not start MetricDumpActor. No metrics will be submitted to the WebInterface.", e); + } + } + } + + /** + * Returns the address under which the {@link MetricQueryService} is reachable. + * + * @return address of the metric query service + */ + @Override + @Nullable + public String getMetricQueryServicePath() { + return metricQueryServicePath; + } + + @Override + public char getDelimiter() { + return this.globalDelimiter; + } + + @Override + public char getDelimiter(int reporterIndex) { + try { + return delimiters.get(reporterIndex); + } catch (IndexOutOfBoundsException e) { + LOG.warn("Delimiter for reporter index {} not found, returning global delimiter.", reporterIndex); + return this.globalDelimiter; + } + } + + @Override + public int getNumberReporters() { + return reporters.size(); + } + + @VisibleForTesting + public List getReporters() { + return reporters; + } + + /** + * Returns whether this registry has been shutdown. + * + * @return true, if this registry was shutdown, otherwise false + */ + public boolean isShutdown() { + synchronized (lock) { + return isShutdown; + } + } + + /** + * Shuts down this registry and the associated {@link MetricReporter}. + * + *

NOTE: This operation is asynchronous and returns a future which is completed + * once the shutdown operation has been completed. + * + * @return Future which is completed once the {@link MetricRegistryImpl} + * is shut down. + */ + public CompletableFuture shutdown() { + synchronized (lock) { + if (isShutdown) { + return terminationFuture; + } else { + isShutdown = true; + final Collection> terminationFutures = new ArrayList<>(3); + final Time gracePeriod = Time.seconds(1L); + + if (queryService != null) { + final CompletableFuture queryServiceTerminationFuture = ActorUtils.nonBlockingShutDown( + gracePeriod.toMilliseconds(), + TimeUnit.MILLISECONDS, + queryService); + + terminationFutures.add(queryServiceTerminationFuture); + } + + Throwable throwable = null; + for (MetricReporter reporter : reporters) { + try { + reporter.close(); + } catch (Throwable t) { + throwable = ExceptionUtils.firstOrSuppressed(t, throwable); + } + } + reporters.clear(); + + if (throwable != null) { + terminationFutures.add( + FutureUtils.completedExceptionally( + new FlinkException("Could not shut down the metric reporters properly.", throwable))); + } + + final CompletableFuture executorShutdownFuture = ExecutorUtils.nonBlockingShutdown( + gracePeriod.toMilliseconds(), + TimeUnit.MILLISECONDS, + executor); + + terminationFutures.add(executorShutdownFuture); + + FutureUtils + .completeAll(terminationFutures) + .whenComplete( + (Void ignored, Throwable error) -> { + if (error != null) { + terminationFuture.completeExceptionally(error); + } else { + terminationFuture.complete(null); + } + }); + + return terminationFuture; + } + } + } + + @Override + public ScopeFormats getScopeFormats() { + return scopeFormats; + } + + // ------------------------------------------------------------------------ + // Metrics (de)registration + // ------------------------------------------------------------------------ + + @Override + public void register(Metric metric, String metricName, AbstractMetricGroup group) { + synchronized (lock) { + if (isShutdown()) { + LOG.warn("Cannot register metric, because the MetricRegistry has already been shut down."); + } else { + if (reporters != null) { + for (int i = 0; i < reporters.size(); i++) { + MetricReporter reporter = reporters.get(i); + try { + if (reporter != null) { + FrontMetricGroup front = new FrontMetricGroup>(i, group); + reporter.notifyOfAddedMetric(metric, metricName, front); + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + } + } + try { + if (queryService != null) { + MetricQueryService.notifyOfAddedMetric(queryService, metric, metricName, group); + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + try { + if (metric instanceof View) { + if (viewUpdater == null) { + viewUpdater = new ViewUpdater(executor); + } + viewUpdater.notifyOfAddedView((View) metric); + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + } + } + } + + @Override + public void unregister(Metric metric, String metricName, AbstractMetricGroup group) { + synchronized (lock) { + if (isShutdown()) { + LOG.warn("Cannot unregister metric, because the MetricRegistry has already been shut down."); + } else { + if (reporters != null) { + for (int i = 0; i < reporters.size(); i++) { + try { + MetricReporter reporter = reporters.get(i); + if (reporter != null) { + FrontMetricGroup front = new FrontMetricGroup>(i, group); + reporter.notifyOfRemovedMetric(metric, metricName, front); + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + } + } + try { + if (queryService != null) { + MetricQueryService.notifyOfRemovedMetric(queryService, metric); + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + try { + if (metric instanceof View) { + if (viewUpdater != null) { + viewUpdater.notifyOfRemovedView((View) metric); + } + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + } + } + } + + // ------------------------------------------------------------------------ + + @VisibleForTesting + @Nullable + public ActorRef getQueryService() { + return queryService; + } + + // ------------------------------------------------------------------------ + + /** + * This task is explicitly a static class, so that it does not hold any references to the enclosing + * MetricsRegistry instance. + * + *

This is a subtle difference, but very important: With this static class, the enclosing class instance + * may become garbage-collectible, whereas with an anonymous inner class, the timer thread + * (which is a GC root) will hold a reference via the timer task and its enclosing instance pointer. + * Making the MetricsRegistry garbage collectible makes the java.util.Timer garbage collectible, + * which acts as a fail-safe to stop the timer thread and prevents resource leaks. + */ + private static final class ReporterTask extends TimerTask { + + private final Scheduled reporter; + + private ReporterTask(Scheduled reporter) { + this.reporter = reporter; + } + + @Override + public void run() { + try { + reporter.report(); + } catch (Throwable t) { + LOG.warn("Error while reporting metrics", t); + } + } + } +} diff --git a/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormat.java b/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormat.java new file mode 100644 index 0000000000..ac37356ac2 --- /dev/null +++ b/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormat.java @@ -0,0 +1,264 @@ +/* + * 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.runtime.metrics.scope; + +import org.apache.flink.metrics.CharacterFilter; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * This class represents the format after which the "scope" (or namespace) of the various + * component metric groups is built. Component metric groups are for example + * "TaskManager", "Task", or "Operator". + * + *

User defined scope formats allow users to include or exclude + * certain identifiers from the scope. The scope for metrics belonging to the "Task" + * group could for example include the task attempt number (more fine grained identification), or + * exclude it (continuity of the namespace across failure and recovery). + */ +public abstract class ScopeFormat { + + private static CharacterFilter defaultFilter = new CharacterFilter() { + @Override + public String filterCharacters(String input) { + return input; + } + }; + + // ------------------------------------------------------------------------ + // Scope Format Special Characters + // ------------------------------------------------------------------------ + + /** + * If the scope format starts with this character, then the parent components scope + * format will be used as a prefix. + * + *

For example, if the TaskManager's job format is {@code "*."}, and the + * TaskManager format is {@code ""}, then the job's metrics + * will have {@code "."} as their scope. + */ + public static final String SCOPE_INHERIT_PARENT = "*"; + + public static final String SCOPE_SEPARATOR = "."; + + private static final String SCOPE_VARIABLE_PREFIX = "<"; + private static final String SCOPE_VARIABLE_SUFFIX = ">"; + + // ------------------------------------------------------------------------ + // Scope Variables + // ------------------------------------------------------------------------ + + public static final String SCOPE_HOST = asVariable("host"); + + // ----- Task Manager ---- + + public static final String SCOPE_TASKMANAGER_ID = asVariable("tm_id"); + + // ----- Job ----- + + public static final String SCOPE_JOB_ID = asVariable("job_id"); + public static final String SCOPE_JOB_NAME = asVariable("job_name"); + + // ----- Task ---- + + public static final String SCOPE_TASK_VERTEX_ID = asVariable("task_id"); + public static final String SCOPE_TASK_NAME = asVariable("task_name"); + public static final String SCOPE_TASK_ATTEMPT_ID = asVariable("task_attempt_id"); + public static final String SCOPE_TASK_ATTEMPT_NUM = asVariable("task_attempt_num"); + public static final String SCOPE_TASK_SUBTASK_INDEX = asVariable("subtask_index"); + + + public static final String SCOPE_PLUGINE_TYPE = asVariable("plugin_type"); + + public static final String SCOPE_PLUGINE_NAME = asVariable("plugin_name"); + + // ----- Operator ---- + + public static final String SCOPE_OPERATOR_ID = asVariable("operator_id"); + public static final String SCOPE_OPERATOR_NAME = asVariable("operator_name"); + + + // ------------------------------------------------------------------------ + // Scope Format Base + // ------------------------------------------------------------------------ + + /** The scope format. */ + private final String format; + + /** The format, split into components. */ + private final String[] template; + + private final int[] templatePos; + + private final int[] valuePos; + + // ------------------------------------------------------------------------ + + protected ScopeFormat(String format, ScopeFormat parent, String[] variables) { + checkNotNull(format, "format is null"); + + final String[] rawComponents = format.split("\\" + SCOPE_SEPARATOR); + + // compute the template array + final boolean parentAsPrefix = rawComponents.length > 0 && rawComponents[0].equals(SCOPE_INHERIT_PARENT); + if (parentAsPrefix) { + if (parent == null) { + throw new IllegalArgumentException("Component scope format requires parent prefix (starts with '" + + SCOPE_INHERIT_PARENT + "'), but this component has no parent (is root component)."); + } + + this.format = format.length() > 2 ? format.substring(2) : ""; + + String[] parentTemplate = parent.template; + int parentLen = parentTemplate.length; + + this.template = new String[parentLen + rawComponents.length - 1]; + System.arraycopy(parentTemplate, 0, this.template, 0, parentLen); + System.arraycopy(rawComponents, 1, this.template, parentLen, rawComponents.length - 1); + } + else { + this.format = format.isEmpty() ? "" : format; + this.template = rawComponents; + } + + // --- compute the replacement matrix --- + // a bit of clumsy Java collections code ;-) + + HashMap varToValuePos = arrayToMap(variables); + List templatePos = new ArrayList<>(); + List valuePos = new ArrayList<>(); + + for (int i = 0; i < template.length; i++) { + final String component = template[i]; + + // check if that is a variable + if (component != null && component.length() >= 3 && + component.charAt(0) == '<' && component.charAt(component.length() - 1) == '>') { + + // this is a variable + Integer replacementPos = varToValuePos.get(component); + if (replacementPos != null) { + templatePos.add(i); + valuePos.add(replacementPos); + } + } + } + + this.templatePos = integerListToArray(templatePos); + this.valuePos = integerListToArray(valuePos); + } + + // ------------------------------------------------------------------------ + + public String format() { + return format; + } + + protected final String[] copyTemplate() { + String[] copy = new String[template.length]; + System.arraycopy(template, 0, copy, 0, template.length); + return copy; + } + + protected final String[] bindVariables(String[] template, String[] values) { + final int len = templatePos.length; + for (int i = 0; i < len; i++) { + template[templatePos[i]] = values[valuePos[i]]; + } + return template; + } + + // ------------------------------------------------------------------------ + + @Override + public String toString() { + return "ScopeFormat '" + format + '\''; + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * Formats the given string to resemble a scope variable. + * + * @param scope The string to format + * @return The formatted string + */ + public static String asVariable(String scope) { + return SCOPE_VARIABLE_PREFIX + scope + SCOPE_VARIABLE_SUFFIX; + } + + public static String concat(String... components) { + return concat(defaultFilter, '.', components); + } + + public static String concat(CharacterFilter filter, String... components) { + return concat(filter, '.', components); + } + + public static String concat(Character delimiter, String... components) { + return concat(defaultFilter, delimiter, components); + } + + /** + * Concatenates the given component names separated by the delimiter character. Additionally + * the character filter is applied to all component names. + * + * @param filter Character filter to be applied to the component names + * @param delimiter Delimiter to separate component names + * @param components Array of component names + * @return The concatenated component name + */ + public static String concat(CharacterFilter filter, Character delimiter, String... components) { + StringBuilder sb = new StringBuilder(); + sb.append(filter.filterCharacters(components[0])); + for (int x = 1; x < components.length; x++) { + sb.append(delimiter); + sb.append(filter.filterCharacters(components[x])); + } + return sb.toString(); + } + + protected static String valueOrNull(Object value) { + return (value == null || (value instanceof String && ((String) value).isEmpty())) ? + "null" : value.toString(); + } + + protected static HashMap arrayToMap(String[] array) { + HashMap map = new HashMap<>(array.length); + for (int i = 0; i < array.length; i++) { + map.put(array[i], i); + } + return map; + } + + private static int[] integerListToArray(List list) { + int[] array = new int[list.size()]; + int pos = 0; + for (Integer i : list) { + array[pos++] = i; + } + return array; + } +} diff --git a/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormats.java b/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormats.java new file mode 100644 index 0000000000..a395881956 --- /dev/null +++ b/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormats.java @@ -0,0 +1,111 @@ +/* + * 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.runtime.metrics.scope; + +import com.dtstack.flinkx.metrics.scope.PipelineScopeFormat; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MetricOptions; + +/** + * A container for component scope formats. + */ +public final class ScopeFormats { + + private final JobManagerScopeFormat jobManagerFormat; + private final JobManagerJobScopeFormat jobManagerJobFormat; + private final TaskManagerScopeFormat taskManagerFormat; + private final TaskManagerJobScopeFormat taskManagerJobFormat; + private final TaskScopeFormat taskFormat; + private final OperatorScopeFormat operatorFormat; + private final PipelineScopeFormat pipelineScopeFormat; + + // ------------------------------------------------------------------------ + + /** + * Creates all scope formats, based on the given scope format strings. + */ + private ScopeFormats( + String jobManagerFormat, + String jobManagerJobFormat, + String taskManagerFormat, + String taskManagerJobFormat, + String taskFormat, + String operatorFormat) { + this.jobManagerFormat = new JobManagerScopeFormat(jobManagerFormat); + this.jobManagerJobFormat = new JobManagerJobScopeFormat(jobManagerJobFormat, this.jobManagerFormat); + this.taskManagerFormat = new TaskManagerScopeFormat(taskManagerFormat); + this.taskManagerJobFormat = new TaskManagerJobScopeFormat(taskManagerJobFormat, this.taskManagerFormat); + this.taskFormat = new TaskScopeFormat(taskFormat, this.taskManagerJobFormat); + this.operatorFormat = new OperatorScopeFormat(operatorFormat, this.taskFormat); + this.pipelineScopeFormat = new PipelineScopeFormat(".flinkx..."); + } + + // ------------------------------------------------------------------------ + // Accessors + // ------------------------------------------------------------------------ + + public JobManagerScopeFormat getJobManagerFormat() { + return this.jobManagerFormat; + } + + public TaskManagerScopeFormat getTaskManagerFormat() { + return this.taskManagerFormat; + } + + public TaskManagerJobScopeFormat getTaskManagerJobFormat() { + return this.taskManagerJobFormat; + } + + public JobManagerJobScopeFormat getJobManagerJobFormat() { + return this.jobManagerJobFormat; + } + + public TaskScopeFormat getTaskFormat() { + return this.taskFormat; + } + + public OperatorScopeFormat getOperatorFormat() { + return this.operatorFormat; + } + + public PipelineScopeFormat getPipelineScopeFormat() { + return pipelineScopeFormat; + } + + // ------------------------------------------------------------------------ + // Parsing from Config + // ------------------------------------------------------------------------ + + /** + * Creates the scope formats as defined in the given configuration. + * + * @param config The configuration that defines the formats + * @return The ScopeFormats parsed from the configuration + */ + public static ScopeFormats fromConfig(Configuration config) { + String jmFormat = config.getString(MetricOptions.SCOPE_NAMING_JM); + String jmJobFormat = config.getString(MetricOptions.SCOPE_NAMING_JM_JOB); + String tmFormat = config.getString(MetricOptions.SCOPE_NAMING_TM); + String tmJobFormat = config.getString(MetricOptions.SCOPE_NAMING_TM_JOB); + String taskFormat = config.getString(MetricOptions.SCOPE_NAMING_TASK); + String operatorFormat = config.getString(MetricOptions.SCOPE_NAMING_OPERATOR); + + return new ScopeFormats(jmFormat, jmJobFormat, tmFormat, tmJobFormat, taskFormat, operatorFormat); + } +} From 0f5518cfa153060b5deafaae9f5997d3f570e4ef Mon Sep 17 00:00:00 2001 From: jiangbo Date: Thu, 14 Mar 2019 19:58:58 +0800 Subject: [PATCH 16/96] Get max value in one channel --- .../com/dtstack/flinkx/constants/Metrics.java | 2 + .../flinkx/rdb/datareader/JdbcConfigKeys.java | 3 + .../flinkx/rdb/datareader/JdbcDataReader.java | 36 +- .../rdb/inputformat/JdbcInputFormat.java | 307 ++++++++++++++---- .../inputformat/JdbcInputFormatBuilder.java | 28 +- .../rdb/inputformat/JdbcInputSplit.java | 81 +++++ .../rdb/inputformat/MaximumAccumulator.java | 27 +- .../com/dtstack/flinkx/rdb/util/DBUtil.java | 141 +++++--- 8 files changed, 496 insertions(+), 129 deletions(-) create mode 100644 flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputSplit.java diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/Metrics.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/Metrics.java index 964705999d..ca99ff8d32 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/Metrics.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/Metrics.java @@ -51,4 +51,6 @@ public class Metrics { public static String START_LOCATION = "startLocation"; public static String TABLE_COL = "tableCol"; + + public static String MAX_VALUE = "maxValue"; } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java index 4efc0c5ae5..ca3180e2fe 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java @@ -38,8 +38,11 @@ public class JdbcConfigKeys { public static final String KEY_QUERY_TIME_OUT = "queryTimeOut"; + public static final String KEY_REQUEST_ACCUMULATOR_INTERVAL = "requestAccumulatorInterval"; + public static final String KEY_INCRE_COLUMN = "increColumn"; public static final String KEY_START_LOCATION = "startLocation"; + public static final String KEY_REALTIME_INCRE_SYNC = "realTimeIncreSync"; } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java index f3c2f6504e..2b2d233052 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java @@ -27,9 +27,11 @@ import com.dtstack.flinkx.rdb.util.DBUtil; import com.dtstack.flinkx.reader.DataReader; import com.dtstack.flinkx.reader.MetaColumn; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; + import java.util.List; /** @@ -66,6 +68,10 @@ public class JdbcDataReader extends DataReader { protected int queryTimeOut; + protected int requestAccumulatorInterval; + + protected boolean realTimeIncreSync; + public void setDatabaseInterface(DatabaseInterface databaseInterface) { this.databaseInterface = databaseInterface; } @@ -87,9 +93,13 @@ public JdbcDataReader(DataTransferConfig config, StreamExecutionEnvironment env) metaColumns = MetaColumn.getMetaColumns(readerConfig.getParameter().getColumn()); fetchSize = readerConfig.getParameter().getIntVal(JdbcConfigKeys.KEY_FETCH_SIZE,0); queryTimeOut = readerConfig.getParameter().getIntVal(JdbcConfigKeys.KEY_QUERY_TIME_OUT,0); + requestAccumulatorInterval = readerConfig.getParameter().getIntVal(JdbcConfigKeys.KEY_REQUEST_ACCUMULATOR_INTERVAL,2); splitKey = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_SPLIK_KEY); increColumn = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_INCRE_COLUMN); startLocation = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_START_LOCATION,null); + + realTimeIncreSync = readerConfig.getParameter().getBooleanVal(JdbcConfigKeys.KEY_REALTIME_INCRE_SYNC,true); + realTimeIncreSync = StringUtils.isNotEmpty(increColumn); } @Override @@ -108,29 +118,27 @@ public DataStream readData() { builder.setMetaColumn(metaColumns); builder.setFetchSize(fetchSize == 0 ? databaseInterface.getFetchSize() : fetchSize); builder.setQueryTimeOut(queryTimeOut == 0 ? databaseInterface.getQueryTimeout() : queryTimeOut); + builder.setRequestAccumulatorInterval(requestAccumulatorInterval); builder.setIncreCol(increColumn); + builder.setIncreColType(getIncrementColType()); builder.setStartLocation(startLocation); + builder.setSplitKey(splitKey); + builder.setNumPartitions(numPartitions); + builder.setRealTimeIncreSync(realTimeIncreSync); - boolean isSplitByKey = false; - if(numPartitions > 1 && splitKey != null && splitKey.trim().length() != 0) { - builder.setParameterValues(DBUtil.getParameterValues(numPartitions)); - isSplitByKey = true; - } - - if(increColumn != null){ - String increColType = getIncreColType(); - where = DBUtil.buildWhereSql(databaseInterface,increColType,where,increColumn,startLocation); - builder.setIncreColType(increColType); - } - - String query = DBUtil.getQuerySql(databaseInterface,table,metaColumns,splitKey,where,isSplitByKey); + boolean isSplitByKey = numPartitions > 1 && StringUtils.isNotEmpty(splitKey); + String query = DBUtil.getQuerySql(databaseInterface, table, metaColumns, splitKey, where, isSplitByKey, realTimeIncreSync); builder.setQuery(query); RichInputFormat format = builder.finish(); return createInput(format, (databaseInterface.getDatabaseType() + "reader").toLowerCase()); } - private String getIncreColType(){ + private String getIncrementColType(){ + if (StringUtils.isEmpty(increColumn)){ + return null; + } + for (MetaColumn metaColumn : metaColumns) { if(metaColumn.getName().equals(increColumn)){ return metaColumn.getType(); diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 5da390da37..8e2fef9189 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -28,16 +28,20 @@ import com.dtstack.flinkx.util.ClassUtil; import com.dtstack.flinkx.util.DateUtil; import com.dtstack.flinkx.util.StringUtil; +import com.dtstack.flinkx.util.URLUtil; +import com.google.gson.Gson; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; import org.apache.flink.types.Row; import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.Reader; import java.sql.*; import java.util.*; import java.util.Date; @@ -80,8 +84,6 @@ public class JdbcInputFormat extends RichInputFormat { protected boolean hasNext; - protected Object[][] parameterValues; - protected int columnCount; protected String table; @@ -96,14 +98,24 @@ public class JdbcInputFormat extends RichInputFormat { protected String startLocation; + protected String splitKey; + private int increColIndex; protected int fetchSize; protected int queryTimeOut; + protected int requestAccumulatorInterval; + + protected boolean realTimeIncreSync; + + protected int numPartitions; + protected StringAccumulator tableColAccumulator; + protected StringAccumulator maxValueAccumulator; + protected MaximumAccumulator endLocationAccumulator; protected StringAccumulator startLocationAccumulator; @@ -118,56 +130,30 @@ public void configure(Configuration configuration) { } - private void setMetric(){ - Map> accumulatorMap = getRuntimeContext().getAllAccumulators(); - - if(!accumulatorMap.containsKey(Metrics.TABLE_COL)){ - tableColAccumulator = new StringAccumulator(); - tableColAccumulator.add(table + "-" + increCol); - getRuntimeContext().addAccumulator(Metrics.TABLE_COL,tableColAccumulator); - } + @Override + public void openInternal(InputSplit inputSplit) throws IOException { + try { + LOG.info(inputSplit.toString()); - if(!accumulatorMap.containsKey(Metrics.END_LOCATION)){ - endLocationAccumulator = new MaximumAccumulator(); - getRuntimeContext().addAccumulator(Metrics.END_LOCATION,endLocationAccumulator); - } + ClassUtil.forName(drivername, getClass().getClassLoader()); - if (startLocation != null){ - endLocationAccumulator.add(startLocation); - if(!accumulatorMap.containsKey(Metrics.START_LOCATION)){ - startLocationAccumulator = new StringAccumulator(); - startLocationAccumulator.add(startLocation); - getRuntimeContext().addAccumulator(Metrics.START_LOCATION,startLocationAccumulator); + if (realTimeIncreSync){ + getMaxValue(inputSplit); } - } - for (int i = 0; i < metaColumns.size(); i++) { - if (metaColumns.get(i).getName().equals(increCol)){ - increColIndex = i; - break; + initMetric(inputSplit); + + if(!canReadData(inputSplit)){ + LOG.warn("Not read data when the start location are equal to end location"); + + hasNext = false; + return; } - } - } - @Override - public void openInternal(InputSplit inputSplit) throws IOException { - try { - ClassUtil.forName(drivername, getClass().getClassLoader()); dbConn = DBUtil.getConnection(dbURL, username, password); dbConn.setAutoCommit(false); - Statement statement = dbConn.createStatement(resultSetType, resultSetConcurrency); - if (inputSplit != null && parameterValues != null) { - String n = parameterValues[inputSplit.getSplitNumber()][0].toString(); - String m = parameterValues[inputSplit.getSplitNumber()][1].toString(); - queryTemplate = queryTemplate.replace("${N}",n).replace("${M}",m); - - if (LOG.isDebugEnabled()) { - LOG.debug(String.format("Executing '%s' with parameters %s", queryTemplate, Arrays.deepToString(parameterValues[inputSplit.getSplitNumber()]))); - } - } - if(EDatabaseType.MySQL == databaseInterface.getDatabaseType()){ statement.setFetchSize(Integer.MIN_VALUE); } else { @@ -177,7 +163,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { if(EDatabaseType.Carbondata != databaseInterface.getDatabaseType()) { statement.setQueryTimeout(queryTimeOut); } - resultSet = statement.executeQuery(queryTemplate); + + String querySql = buildQuerySql(inputSplit); + resultSet = statement.executeQuery(querySql); columnCount = resultSet.getMetaData().getColumnCount(); hasNext = resultSet.next(); @@ -185,9 +173,6 @@ public void openInternal(InputSplit inputSplit) throws IOException { descColumnTypeList = DBUtil.analyzeTable(dbURL, username, password,databaseInterface,table,metaColumns); } - if(increCol != null){ - setMetric(); - } } catch (SQLException se) { throw new IllegalArgumentException("open() failed." + se.getMessage(), se); } @@ -203,23 +188,19 @@ public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOEx @Override public InputSplit[] createInputSplits(int minNumSplits) throws IOException { - if (parameterValues == null) { - return new GenericInputSplit[]{new GenericInputSplit(0, 1)}; + JdbcInputSplit[] splits = new JdbcInputSplit[minNumSplits]; + for (int i = 0; i < minNumSplits; i++) { + splits[i] = new JdbcInputSplit(i, numPartitions, i, startLocation, null); } - GenericInputSplit[] ret = new GenericInputSplit[parameterValues.length]; - for (int i = 0; i < ret.length; i++) { - ret[i] = new GenericInputSplit(i, ret.length); - } - return ret; - } + return splits; + } @Override public InputSplitAssigner getInputSplitAssigner(InputSplit[] inputSplits) { return new DefaultInputSplitAssigner(inputSplits); } - @Override public boolean reachedEnd() throws IOException { return !hasNext; @@ -248,7 +229,7 @@ public Row nextRecordInternal(Row row) throws IOException { } } - if(increCol != null){ + if(increCol != null && !realTimeIncreSync){ if (ColumnType.isTimeType(increColType)){ Timestamp increVal = resultSet.getTimestamp(increColIndex + 1); if(increVal != null){ @@ -274,6 +255,212 @@ public Row nextRecordInternal(Row row) throws IOException { } } + @Override + public void closeInternal() throws IOException { + DBUtil.closeDBResources(resultSet,statement,dbConn); + } + + private void initMetric(InputSplit split){ + + if (StringUtils.isEmpty(increCol)){ + return; + } + + Map> accumulatorMap = getRuntimeContext().getAllAccumulators(); + + if(!accumulatorMap.containsKey(Metrics.TABLE_COL)){ + tableColAccumulator = new StringAccumulator(); + tableColAccumulator.add(table + "-" + increCol); + getRuntimeContext().addAccumulator(Metrics.TABLE_COL,tableColAccumulator); + } + + String endLocation = ((JdbcInputSplit)split).getEndLocation(); + if(!accumulatorMap.containsKey(Metrics.END_LOCATION) && endLocation != null){ + endLocationAccumulator = new MaximumAccumulator(); + + if(realTimeIncreSync){ + endLocationAccumulator.add(endLocation); + } + + getRuntimeContext().addAccumulator(Metrics.END_LOCATION,endLocationAccumulator); + } + + if (!accumulatorMap.containsKey(Metrics.START_LOCATION) && startLocation != null){ + if(!realTimeIncreSync){ + endLocationAccumulator.add(startLocation); + } + + startLocationAccumulator = new StringAccumulator(); + startLocationAccumulator.add(startLocation); + getRuntimeContext().addAccumulator(Metrics.START_LOCATION,startLocationAccumulator); + } + + for (int i = 0; i < metaColumns.size(); i++) { + if (metaColumns.get(i).getName().equals(increCol)){ + increColIndex = i; + break; + } + } + } + + private void getMaxValue(InputSplit inputSplit){ + String maxValue = null; + if (inputSplit.getSplitNumber() == 0){ + maxValue = getMaxValueFromDb(); + maxValueAccumulator = new StringAccumulator(); + maxValueAccumulator.add(maxValue); + getRuntimeContext().addAccumulator(Metrics.MAX_VALUE, maxValueAccumulator); + } else { + Map vars = getRuntimeContext().getMetricGroup().getAllVariables(); + String jobId = vars.get(""); + + String[] monitors; + if (monitorUrls.startsWith("http")){ + monitors = new String[] {String.format("%s/jobs/%s/accumulators", monitorUrls, jobId)}; + } else { + monitors = monitorUrls.split(","); + for (int i = 0; i < monitors.length; i++) { + monitors[i] = String.format("http://%s/jobs/%s/accumulators", monitors[i], jobId); + } + } + + /** + * The extra 10 times is to ensure that accumulator is updated + */ + int maxAcquireTimes = (queryTimeOut / requestAccumulatorInterval) + 10; + + int acquireTimes = 0; + while (StringUtils.isEmpty(maxValue) && acquireTimes < maxAcquireTimes){ + try { + Thread.sleep(requestAccumulatorInterval * 1000); + } catch (InterruptedException ignore) { + } + + maxValue = getMaxvalueFromAccumulator(monitors); + acquireTimes++; + } + + if (StringUtils.isEmpty(maxValue)){ + throw new RuntimeException("Can't get the max value from accumulator"); + } + } + + ((JdbcInputSplit) inputSplit).setEndLocation(maxValue); + } + + private String getMaxvalueFromAccumulator(String[] monitors){ + String maxValue = null; + Gson gson = new Gson(); + for (String monitor : monitors) { + LOG.info("Request url:" + monitor); + try (InputStream inputStream = URLUtil.open(monitor); Reader rd = new InputStreamReader(inputStream)) { + Map map = gson.fromJson(rd, Map.class); + + LOG.info("Accumulator data:" + gson.toJson(map)); + + List userTaskAccumulators = (List) map.get("user-task-accumulators"); + for (Map accumulator : userTaskAccumulators) { + if (Metrics.MAX_VALUE.equals(accumulator.get("name"))) { + maxValue = (String) accumulator.get("value"); + break; + } + } + + if (StringUtils.isNotEmpty(maxValue)) { + break; + } + } catch (Exception e) { + LOG.error("Get max value from accumulator error:", e); + } + } + + return maxValue; + } + + private boolean canReadData(InputSplit split){ + if (StringUtils.isEmpty(increCol)){ + return true; + } + + if (!realTimeIncreSync){ + return true; + } + + JdbcInputSplit jdbcInputSplit = (JdbcInputSplit) split; + return !StringUtils.equals(jdbcInputSplit.getStartLocation(), jdbcInputSplit.getEndLocation()); + } + + private String buildQuerySql(InputSplit inputSplit){ + String querySql = queryTemplate; + + if (inputSplit != null) { + JdbcInputSplit jdbcInputSplit = (JdbcInputSplit) inputSplit; + + if (StringUtils.isNotEmpty(splitKey)){ + querySql = queryTemplate.replace("${N}", String.valueOf(numPartitions)) + .replace("${M}", String.valueOf(jdbcInputSplit.getMod())); + } + + if (realTimeIncreSync){ + String incrementFilter = DBUtil.buildIncrementFilter(databaseInterface, increColType, increCol, + jdbcInputSplit.getStartLocation(), jdbcInputSplit.getEndLocation()); + + if(StringUtils.isNotEmpty(incrementFilter)){ + incrementFilter = " and " + incrementFilter; + } + + querySql = querySql.replace(DBUtil.INCREMENT_FILTER_PLACEHOLDER, incrementFilter); + } + } + + LOG.warn(String.format("Executing sql is: '%s'", querySql)); + + return querySql; + } + + private String getMaxValueFromDb() { + String maxValue = null; + Connection conn = null; + Statement st = null; + ResultSet rs = null; + try { + long startTime = System.currentTimeMillis(); + + String queryMaxValueSql = String.format("select max(%s) as max_value from %s", + databaseInterface.quoteColumn(increCol), databaseInterface.quoteTable(table)); + String startSql = DBUtil.buildStartLocationSql(databaseInterface, increColType, increCol, startLocation); + if(StringUtils.isNotEmpty(startSql)){ + queryMaxValueSql += " where " + startSql; + } + + LOG.info(String.format("Query max value sql is '%s'", queryMaxValueSql)); + + conn = DBUtil.getConnection(dbURL, username, password); + st = conn.createStatement(); + rs = st.executeQuery(queryMaxValueSql); + if (rs.next()){ + if (ColumnType.isTimeType(increColType)){ + Timestamp increVal = rs.getTimestamp("max_value"); + if(increVal != null){ + maxValue = String.valueOf(getLocation(increVal)); + } + } else if(ColumnType.isNumberType(increColType)){ + maxValue = String.valueOf(rs.getLong("max_value")); + } else { + maxValue = rs.getString("max_value"); + } + } + + LOG.info(String.format("Takes [%s] milliseconds to get the maximum value [%s]", System.currentTimeMillis() - startTime, maxValue)); + + return maxValue; + } catch (Throwable e){ + throw new RuntimeException("Get max value from " + table + " error",e); + } finally { + DBUtil.closeDBResources(rs,st,conn); + } + } + private long getLocation(Object increVal){ if(increVal instanceof Timestamp){ long time = ((Timestamp)increVal).getTime() / 1000; @@ -293,10 +480,4 @@ private long getLocation(Object increVal){ } } - @Override - public void closeInternal() throws IOException { - DBUtil.closeDBResources(resultSet,statement,dbConn); - parameterValues = null; - } - } \ No newline at end of file diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java index d2abbb4819..240c3116d9 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java @@ -31,7 +31,6 @@ * Company: www.dtstack.com * @author huyifan.zju@163.com */ -@Deprecated public class JdbcInputFormatBuilder extends RichInputFormatBuilder { private JdbcInputFormat format; @@ -52,10 +51,6 @@ public void setQuery(String query) { format.queryTemplate = query; } - public void setParameterValues(Object[][] parameterValues) { - format.parameterValues = parameterValues; - } - public void setUsername(String username) { format.username = username; } @@ -88,6 +83,10 @@ public void setQueryTimeOut(int queryTimeOut){ format.queryTimeOut = queryTimeOut; } + public void setRequestAccumulatorInterval(int requestAccumulatorInterval){ + format.requestAccumulatorInterval = requestAccumulatorInterval; + } + public void setIncreCol(String increCol){ format.increCol = increCol; } @@ -96,24 +95,37 @@ public void setStartLocation(String startLocation){ format.startLocation = startLocation; } + public void setSplitKey(String splitKey){ + format.splitKey = splitKey; + } + public void setIncreColType(String increColType){ format.increColType = increColType; } + public void setRealTimeIncreSync(boolean realTimeIncreSync){ + format.realTimeIncreSync = realTimeIncreSync; + } + + public void setNumPartitions(int numPartitions){ + format.numPartitions = numPartitions; + } + @Override protected void checkFormat() { + if (format.username == null) { LOG.info("Username was not supplied separately."); } + if (format.password == null) { LOG.info("Password was not supplied separately."); } + if (format.dbURL == null) { throw new IllegalArgumentException("No database URL supplied"); } - if (format.queryTemplate == null) { - throw new IllegalArgumentException("No query supplied"); - } + if (format.drivername == null) { throw new IllegalArgumentException("No driver supplied"); } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputSplit.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputSplit.java new file mode 100644 index 0000000000..cc03f46bbf --- /dev/null +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputSplit.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.rdb.inputformat; + +import org.apache.flink.core.io.GenericInputSplit; + +/** + * @author jiangbo + * @explanation + * @date 2019/3/6 + */ +public class JdbcInputSplit extends GenericInputSplit { + + private int mod; + + private String endLocation; + + private String startLocation; + + /** + * Creates a generic input split with the given split number. + * + * @param partitionNumber The number of the split's partition. + * @param totalNumberOfPartitions The total number of the splits (partitions). + */ + public JdbcInputSplit(int partitionNumber, int totalNumberOfPartitions, int mod, String startLocation, String endLocation) { + super(partitionNumber, totalNumberOfPartitions); + this.mod = mod; + this.startLocation = startLocation; + this.endLocation = endLocation; + } + + public int getMod() { + return mod; + } + + public String getEndLocation() { + return endLocation; + } + + public String getStartLocation() { + return startLocation; + } + + public void setMod(int mod) { + this.mod = mod; + } + + public void setEndLocation(String endLocation) { + this.endLocation = endLocation; + } + + public void setStartLocation(String startLocation) { + this.startLocation = startLocation; + } + + @Override + public String toString() { + return "JdbcInputSplit{" + + "mod=" + mod + + ", endLocation='" + endLocation + '\'' + + ", startLocation='" + startLocation + '\'' + + '}'; + } +} diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java index dd8042f9aa..3b6307a8db 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/MaximumAccumulator.java @@ -19,6 +19,8 @@ package com.dtstack.flinkx.rdb.inputformat; +import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.math.NumberUtils; import org.apache.flink.api.common.accumulators.Accumulator; import java.math.BigInteger; @@ -34,13 +36,19 @@ public class MaximumAccumulator implements Accumulator { @Override public void add(String value) { + if(StringUtils.isEmpty(value)){ + return; + } + if(localValue == null){ localValue = value; - } else { + } else if(NumberUtils.isNumber(localValue)){ BigInteger newVal = new BigInteger(value); if(newVal.compareTo(new BigInteger(localValue)) > 0){ localValue = value; } + } else { + localValue = localValue.compareTo(value) < 0 ? value : localValue; } } @@ -56,9 +64,22 @@ public void resetLocal() { @Override public void merge(Accumulator other) { - BigInteger local = new BigInteger(localValue); - if(local.compareTo(new BigInteger(other.getLocalValue())) > 0){ + if (other == null || StringUtils.isEmpty(other.getLocalValue())){ + return; + } + + if (localValue == null){ localValue = other.getLocalValue(); + return; + } + + if(NumberUtils.isNumber(localValue)){ + BigInteger local = new BigInteger(localValue); + if(local.compareTo(new BigInteger(other.getLocalValue())) < 0){ + localValue = other.getLocalValue(); + } + } else { + localValue = localValue.compareTo(other.getLocalValue()) < 0 ? other.getLocalValue() : localValue; } } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java index 6266244b43..5c89bb0173 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java @@ -23,10 +23,7 @@ import com.dtstack.flinkx.rdb.ParameterValuesProvider; import com.dtstack.flinkx.rdb.type.TypeConverterInterface; import com.dtstack.flinkx.reader.MetaColumn; -import com.dtstack.flinkx.util.ClassUtil; -import com.dtstack.flinkx.util.DateUtil; -import com.dtstack.flinkx.util.SysUtil; -import com.dtstack.flinkx.util.TelnetUtil; +import com.dtstack.flinkx.util.*; import org.apache.commons.lang.StringUtils; import org.apache.flink.types.Row; import org.slf4j.Logger; @@ -58,6 +55,8 @@ public class DBUtil { private static int MICRO_LENGTH = 16; private static int NANOS_LENGTH = 19; + public static final String INCREMENT_FILTER_PLACEHOLDER = "${incrementFilter}"; + private static Connection getConnectionInternal(String url, String username, String password) throws SQLException { Connection dbConn; synchronized (ClassUtil.lock_str){ @@ -199,21 +198,6 @@ public static void executeBatch(Connection dbConn, List sqls) { } } - public static void executeOneByOne(Connection dbConn, List sqls) { - if(sqls == null || sqls.size() == 0) { - return; - } - - try { - Statement stmt = dbConn.createStatement(); - for(String sql : sqls) { - stmt.execute(sql); - } - } catch (SQLException e) { - e.printStackTrace(); - } - } - public static Map> getPrimaryOrUniqueKeys(String table, Connection dbConn) throws SQLException { Map> keyMap = new HashMap<>(); DatabaseMetaData meta = dbConn.getMetaData(); @@ -377,13 +361,63 @@ public static Object clobToString(Object obj) throws Exception{ return dataStr; } - public static String buildWhereSql(DatabaseInterface databaseInterface,String increColType,String where, - String increCol,String startLocation){ - if (startLocation == null){ - return where; + public static String buildIncrementFilter(DatabaseInterface databaseInterface,String increColType,String increCol, + String startLocation,String endLocation){ + StringBuilder filter = new StringBuilder(); + + String startFilter = buildStartLocationSql(databaseInterface,increColType,increCol,startLocation); + if (StringUtils.isNotEmpty(startFilter)){ + filter.append(startFilter); + } + + String endFilter = buildEndLocationSql(databaseInterface,increColType,increCol,endLocation); + if (StringUtils.isNotEmpty(endFilter)){ + if (filter.length() > 0){ + filter.append(" and ").append(endFilter); + } else { + filter.append(endFilter); + } + } + + return filter.toString(); + } + + public static String buildEndLocationSql(DatabaseInterface databaseInterface,String increColType,String increCol,String endLocation){ + + if(StringUtils.isEmpty(endLocation)){ + return null; + } + + String endLocationSql; + String endTimeStr; + + if(ColumnType.isTimeType(increColType) || (databaseInterface.getDatabaseType() == EDatabaseType.SQLServer && ColumnType.NVARCHAR.name().equals(increColType))){ + endTimeStr = getStartTimeStr(databaseInterface.getDatabaseType(),Long.parseLong(endLocation)); + + if (databaseInterface.getDatabaseType() == EDatabaseType.Oracle){ + endTimeStr = String.format("TO_TIMESTAMP('%s','YYYY-MM-DD HH24:MI:SS:FF6')",endTimeStr); + } else { + endTimeStr = String.format("'%s'",endTimeStr); + } + + endLocationSql = databaseInterface.quoteColumn(increCol) + " < " + endTimeStr; + } else if(ColumnType.isNumberType(increColType)){ + endLocationSql = databaseInterface.quoteColumn(increCol) + " < " + endLocation; + } else { + endTimeStr = String.format("'%s'",endLocation); + endLocationSql = databaseInterface.quoteColumn(increCol) + " < " + endTimeStr; + } + + return endLocationSql; + } + + public static String buildStartLocationSql(DatabaseInterface databaseInterface,String increColType,String increCol,String startLocation){ + + if(StringUtils.isEmpty(startLocation)){ + return null; } - String increFilter; + String startLocationSql; String startTimeStr; if(ColumnType.isTimeType(increColType) || (databaseInterface.getDatabaseType() == EDatabaseType.SQLServer && ColumnType.NVARCHAR.name().equals(increColType))){ @@ -395,21 +429,39 @@ public static String buildWhereSql(DatabaseInterface databaseInterface,String in startTimeStr = String.format("'%s'",startTimeStr); } - increFilter = databaseInterface.quoteColumn(increCol) + " > " + startTimeStr; + startLocationSql = databaseInterface.quoteColumn(increCol) + " >= " + startTimeStr; } else if(ColumnType.isNumberType(increColType)){ - increFilter = databaseInterface.quoteColumn(increCol) + " > " + startLocation; + startLocationSql = databaseInterface.quoteColumn(increCol) + " >= " + startLocation; } else { startTimeStr = String.format("'%s'",startLocation); - increFilter = databaseInterface.quoteColumn(increCol) + " > " + startTimeStr; + startLocationSql = databaseInterface.quoteColumn(increCol) + " >= " + startTimeStr; } - if (where == null || where.length() == 0){ - where = increFilter; - } else { - where = where + " and " + increFilter; + return startLocationSql; + } + + public static String buildWhereSql(String where,String startSql,String endSql){ + StringBuilder whereBuilder = new StringBuilder(); + + if (StringUtils.isNotEmpty(where)){ + whereBuilder.append(where.trim()); + } + + if(StringUtils.isNotEmpty(startSql)){ + if(whereBuilder.toString().length() > 0){ + whereBuilder.append(" and "); + } + whereBuilder.append(startSql); + } + + if(StringUtils.isNotEmpty(endSql)){ + if(whereBuilder.toString().length() > 0){ + whereBuilder.append(" and "); + } + whereBuilder.append(endSql); } - return where; + return whereBuilder.toString(); } private static String getStartTimeStr(EDatabaseType databaseType,Long startLocation){ @@ -472,7 +524,12 @@ public static long getMillis(long startLocation){ } public static String getQuerySql(DatabaseInterface databaseInterface,String table,List metaColumns, - String splitKey,String where,boolean isSplitByKey) { + String splitKey,String customFilter,boolean isSplitByKey){ + return getQuerySql(databaseInterface, table, metaColumns, splitKey, customFilter, isSplitByKey, false); + } + + public static String getQuerySql(DatabaseInterface databaseInterface,String table,List metaColumns, + String splitKey,String customFilter,boolean isSplitByKey,boolean realTimeIncreSync) { StringBuilder sb = new StringBuilder(); List selectColumns = new ArrayList<>(); @@ -490,22 +547,24 @@ public static String getQuerySql(DatabaseInterface databaseInterface,String tabl sb.append("SELECT ").append(StringUtils.join(selectColumns,",")).append(" FROM "); sb.append(databaseInterface.quoteTable(table)); + sb.append(" WHERE 1=1 "); StringBuilder filter = new StringBuilder(); if(isSplitByKey) { - filter.append(databaseInterface.getSplitFilter(splitKey)); + filter.append(" AND ").append(databaseInterface.getSplitFilter(splitKey)); } - if(where != null && where.trim().length() != 0) { - if(filter.length() > 0) { - filter.append(" AND "); - } - filter.append(where); + if(StringUtils.isNotEmpty(customFilter)) { + filter.append(" AND ").append(customFilter); + } + + if (realTimeIncreSync){ + filter.append(" ").append(INCREMENT_FILTER_PLACEHOLDER); } - if(filter.length() != 0) { - sb.append(" WHERE ").append(filter); + if(filter.length() > 0) { + sb.append(filter); } return sb.toString(); From 5b80f5fcf2f9f85cd5a632a4b01011e30cc20f75 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 15 Mar 2019 10:01:29 +0800 Subject: [PATCH 17/96] The ORC format supports binary type writes --- .../src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java | 4 ++++ .../dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java | 4 ++++ .../src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java | 8 ++++---- .../com/dtstack/flinkx/odps/reader/OdpsInputFormat.java | 2 +- 4 files changed, 13 insertions(+), 5 deletions(-) diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java index fdce472e6d..ee56fa339d 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java @@ -24,6 +24,7 @@ import org.apache.commons.lang3.math.NumberUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat; import org.apache.hadoop.hive.serde2.io.DateWritable; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; @@ -198,6 +199,9 @@ public static ObjectInspector columnTypeToObjectInspetor(ColumnType columnType) case BOOLEAN: objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Boolean.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); break; + case BINARY: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(BytesWritable.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; default: throw new IllegalArgumentException("You should not be here"); } diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java index e7250c56c5..860d41be89 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.compress.SnappyCodec; import org.apache.hadoop.mapred.FileOutputFormat; @@ -169,6 +170,9 @@ public void writeSingleRecordInternal(Row row) throws WriteRecordException { case TIMESTAMP: recordList.add(DateUtil.columnToTimestamp(column,null)); break; + case BINARY: + recordList.add(new BytesWritable(rowData.getBytes())); + break; default: throw new IllegalArgumentException(); } diff --git a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java index dd977ca179..0057fdf889 100644 --- a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java +++ b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java @@ -136,7 +136,7 @@ public static TableTunnel.DownloadSession createMasterSessionForNonPartitionedTa final String projectName, final String tableName) { final TableTunnel tunnel = new TableTunnel(odps); - if (StringUtils.isNoneBlank(tunnelServer)) { + if (StringUtils.isNotEmpty(tunnelServer)) { tunnel.setEndpoint(tunnelServer); } @@ -158,7 +158,7 @@ public static TableTunnel.DownloadSession createMasterSessionForPartitionedTable final String projectName, final String tableName, String partition) { final TableTunnel tunnel = new TableTunnel(odps); - if (StringUtils.isNoneBlank(tunnelServer)) { + if (StringUtils.isNotEmpty(tunnelServer)) { tunnel.setEndpoint(tunnelServer); } @@ -183,7 +183,7 @@ public TableTunnel.DownloadSession call() throws Exception { public static TableTunnel.DownloadSession getSlaveSessionForNonPartitionedTable(Odps odps, final String sessionId, String tunnelServer, final String projectName, final String tableName) { final TableTunnel tunnel = new TableTunnel(odps); - if (StringUtils.isNoneBlank(tunnelServer)) { + if (StringUtils.isNotEmpty(tunnelServer)) { tunnel.setEndpoint(tunnelServer); } @@ -203,7 +203,7 @@ public TableTunnel.DownloadSession call() throws Exception { public static TableTunnel.DownloadSession getSlaveSessionForPartitionedTable(Odps odps, final String sessionId, String tunnelServer, final String projectName, final String tableName, String partition) { final TableTunnel tunnel = new TableTunnel(odps); - if (StringUtils.isNoneBlank(tunnelServer)) { + if (StringUtils.isNotEmpty(tunnelServer)) { tunnel.setEndpoint(tunnelServer); } diff --git a/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormat.java b/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormat.java index e6179e51dc..0d0651d05a 100644 --- a/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormat.java +++ b/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormat.java @@ -186,7 +186,7 @@ public Row nextRecordInternal(Row row) throws IOException { val = metaColumn.getValue(); } - if(val != null){ + if(val != null && val instanceof String){ val = StringUtil.string2col(String.valueOf(val),metaColumn.getType(),metaColumn.getTimeFormat()); } From 91073998770191f12127a0d0b6712911f2afac31 Mon Sep 17 00:00:00 2001 From: toutian Date: Fri, 15 Mar 2019 10:48:00 +0800 Subject: [PATCH 18/96] fix --- .../flinkx/metrics/MetricRegistry.java | 73 +++ .../flinkx/metrics/MetricRegistryImpl.java | 310 +++++++++++++ .../flinkx/metrics/base/MetricGroup.java | 168 +++++++ .../metrics/base/reporter/MetricReporter.java | 54 +++ .../metrics/groups/AbstractMetricGroup.java | 387 ++++++++++++++++ .../metrics/groups/ComponentMetricGroup.java | 111 +++++ .../metrics/groups/FrontMetricGroup.java | 48 ++ .../metrics/groups/GenericMetricGroup.java | 57 +++ .../groups/PipelineInputMetricGroup.java | 11 +- .../groups/PipelineOutputMetricGroup.java | 11 +- .../metrics/groups/ProxyMetricGroup.java | 112 +++++ .../AbstractPrometheusReporter.java | 8 +- .../PrometheusPushGatewayReporter.java | 3 +- .../promethues/PrometheusReporter.java | 3 +- .../metrics/scope/PipelineScopeFormat.java | 2 - .../flinkx}/metrics/scope/ScopeFormat.java | 2 +- .../flinkx/metrics/scope/ScopeFormats.java | 57 +++ .../runtime/metrics/MetricRegistryImpl.java | 436 ------------------ .../runtime/metrics/scope/ScopeFormats.java | 111 ----- 19 files changed, 1388 insertions(+), 576 deletions(-) create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistry.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistryImpl.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/MetricGroup.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/reporter/MetricReporter.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/AbstractMetricGroup.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ComponentMetricGroup.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/FrontMetricGroup.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/GenericMetricGroup.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ProxyMetricGroup.java rename flinkx-core/src/main/java/{org/apache/flink/runtime => com/dtstack/flinkx}/metrics/scope/ScopeFormat.java (99%) create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormats.java delete mode 100644 flinkx-core/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java delete mode 100644 flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormats.java diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistry.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistry.java new file mode 100644 index 0000000000..ba8112445e --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistry.java @@ -0,0 +1,73 @@ +/* + * 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.metrics; + +import com.dtstack.flinkx.metrics.groups.AbstractMetricGroup; +import com.dtstack.flinkx.metrics.scope.ScopeFormats; +import org.apache.flink.metrics.Metric; + +/** + * copy from https://github.com/apache/flink + * + * Interface for a metric registry. + */ +public interface MetricRegistry { + + /** + * Returns the global delimiter. + * + * @return global delimiter + */ + char getDelimiter(); + + /** + * Returns the configured delimiter for the reporter with the given index. + * + * @param index index of the reporter whose delimiter should be used + * @return configured reporter delimiter, or global delimiter if index is invalid + */ + char getDelimiter(int index); + + /** + * Returns the number of registered reporters. + */ + int getNumberReporters(); + + /** + * Registers a new {@link Metric} with this registry. + * + * @param metric the metric that was added + * @param metricName the name of the metric + * @param group the group that contains the metric + */ + void register(Metric metric, String metricName, AbstractMetricGroup group); + + /** + * Un-registers the given {@link Metric} with this registry. + * + * @param metric the metric that should be removed + * @param metricName the name of the metric + * @param group the group that contains the metric + */ + void unregister(Metric metric, String metricName, AbstractMetricGroup group); + + + ScopeFormats getScopeFormats(); +} + diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistryImpl.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistryImpl.java new file mode 100644 index 0000000000..c804dac7db --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistryImpl.java @@ -0,0 +1,310 @@ +/* + * 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.metrics; + +import com.dtstack.flinkx.metrics.base.reporter.MetricReporter; +import com.dtstack.flinkx.metrics.groups.AbstractMetricGroup; +import com.dtstack.flinkx.metrics.groups.FrontMetricGroup; +import com.dtstack.flinkx.metrics.scope.ScopeFormats; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.View; +import org.apache.flink.metrics.reporter.Scheduled; +import org.apache.commons.collections.MapUtils; +import org.apache.flink.runtime.metrics.ViewUpdater; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.TimerTask; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +/** + * copy from https://github.com/apache/flink + * + * A MetricRegistry keeps track of all registered {@link Metric Metrics}. It serves as the + * connection between {@link com.dtstack.flinkx.metrics.base.MetricGroup MetricGroups} and {@link MetricReporter MetricReporters}. + */ +public class MetricRegistryImpl implements MetricRegistry { + static final Logger LOG = LoggerFactory.getLogger(MetricRegistryImpl.class); + + private final Object lock = new Object(); + + private List reporters; + private ScheduledExecutorService executor; + + private final ScopeFormats scopeFormats = ScopeFormats.fromDefault(); + private final char globalDelimiter = '.'; + private final List delimiters = new ArrayList<>(); + + private ViewUpdater viewUpdater; + + /** + * Creates a new MetricRegistry and starts the configured reporter. + */ + public MetricRegistryImpl(List reporterPlugins) { + + // instantiate any custom configured reporters + this.reporters = new ArrayList<>(); + this.executor = Executors.newSingleThreadScheduledExecutor(new ExecutorThreadFactory("flinkx-MetricRegistry")); + + + if (reporterPlugins.isEmpty()) { + // no reporters defined + // by default, don't report anything + LOG.info("No metrics reporter configured, no metrics will be exposed/reported."); + } else { + // we have some reporters so + for (Map reporterPlugin: reporterPlugins) { + Iterator>> reportIt = reporterPlugin.entrySet().iterator(); + while (reportIt.hasNext()) { + Map.Entry> reportEntry = reportIt.next(); + String namedReporter = reportEntry.getKey(); + Map reporterConfig = reportEntry.getValue(); + final String className = MapUtils.getString(reporterConfig,"class", null); + if (className == null) { + LOG.error("No reporter class set for reporter " + namedReporter + ". Metrics might not be exposed/reported."); + continue; + } + + try { + String configuredPeriod = MapUtils.getString(reporterConfig,"interval", null); + TimeUnit timeunit = TimeUnit.SECONDS; + long period = 10; + + if (configuredPeriod != null) { + try { + String[] interval = configuredPeriod.split(" "); + period = Long.parseLong(interval[0]); + timeunit = TimeUnit.valueOf(interval[1]); + } catch (Exception e) { + LOG.error("Cannot parse report interval from config: " + configuredPeriod + + " - please use values like '10 SECONDS' or '500 MILLISECONDS'. " + + "Using default reporting interval."); + } + } + + Class reporterClass = Class.forName(className); + MetricReporter reporterInstance = (MetricReporter) reporterClass.newInstance(); + + MetricConfig metricConfig = new MetricConfig(); + addAllToProperties(reporterConfig, metricConfig); + LOG.info("Configuring {} with {}.", reporterClass.getSimpleName(), metricConfig); + reporterInstance.open(metricConfig); + + if (reporterInstance instanceof Scheduled) { + LOG.info("Periodically reporting metrics in intervals of {} {} for reporter {} of type {}.", period, timeunit.name(), namedReporter, className); + + executor.scheduleWithFixedDelay( + new MetricRegistryImpl.ReporterTask((Scheduled) reporterInstance), period, period, timeunit); + } else { + LOG.info("Reporting metrics for reporter {} of type {}.", namedReporter, className); + } + reporters.add(reporterInstance); + + String delimiterForReporter = MapUtils.getString(reporterConfig,"scope.delimiter", String.valueOf(globalDelimiter)); + if (delimiterForReporter.length() != 1) { + LOG.warn("Failed to parse delimiter '{}' for reporter '{}', using global delimiter '{}'.", delimiterForReporter, namedReporter, globalDelimiter); + delimiterForReporter = String.valueOf(globalDelimiter); + } + this.delimiters.add(delimiterForReporter.charAt(0)); + } catch (Throwable t) { + LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", namedReporter, t); + } + } + } + } + } + + private void addAllToProperties(Map confData, Properties props) { + for (Map.Entry entry : confData.entrySet()) { + props.put(entry.getKey(), entry.getValue()); + } + } + + + @Override + public char getDelimiter() { + return this.globalDelimiter; + } + + @Override + public char getDelimiter(int reporterIndex) { + try { + return delimiters.get(reporterIndex); + } catch (IndexOutOfBoundsException e) { + LOG.warn("Delimiter for reporter index {} not found, returning global delimiter.", reporterIndex); + return this.globalDelimiter; + } + } + + @Override + public int getNumberReporters() { + return reporters.size(); + } + + public List getReporters() { + return reporters; + } + + /** + * Returns whether this registry has been shutdown. + * + * @return true, if this registry was shutdown, otherwise false + */ + public boolean isShutdown() { + synchronized (lock) { + return reporters == null && executor.isShutdown(); + } + } + + /** + * Shuts down this registry and the associated {@link MetricReporter}. + */ + public void shutdown() { + synchronized (lock) { + if (reporters != null) { + for (MetricReporter reporter : reporters) { + try { + reporter.close(); + } catch (Throwable t) { + LOG.warn("Metrics reporter did not shut down cleanly", t); + } + } + reporters = null; + } + shutdownExecutor(); + } + } + + private void shutdownExecutor() { + if (executor != null) { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1L, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException e) { + executor.shutdownNow(); + } + } + } + + // ------------------------------------------------------------------------ + // Metrics (de)registration + // ------------------------------------------------------------------------ + + @Override + public void register(Metric metric, String metricName, AbstractMetricGroup group) { + synchronized (lock) { + if (isShutdown()) { + LOG.warn("Cannot register metric, because the MetricRegistry has already been shut down."); + } else { + if (reporters != null) { + for (int i = 0; i < reporters.size(); i++) { + MetricReporter reporter = reporters.get(i); + try { + if (reporter != null) { + FrontMetricGroup front = new FrontMetricGroup>(i, group); + reporter.notifyOfAddedMetric(metric, metricName, front); + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + } + } + try { + if (metric instanceof View) { + if (viewUpdater == null) { + viewUpdater = new ViewUpdater(executor); + } + viewUpdater.notifyOfAddedView((View) metric); + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + } + } + } + + @Override + public void unregister(Metric metric, String metricName, AbstractMetricGroup group) { + synchronized (lock) { + if (isShutdown()) { + LOG.warn("Cannot unregister metric, because the MetricRegistry has already been shut down."); + } else { + if (reporters != null) { + for (int i = 0; i < reporters.size(); i++) { + try { + MetricReporter reporter = reporters.get(i); + if (reporter != null) { + FrontMetricGroup front = new FrontMetricGroup>(i, group); + reporter.notifyOfRemovedMetric(metric, metricName, front); + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + } + } + } + } + } + + @Override + public ScopeFormats getScopeFormats() { + return scopeFormats; + } + + + /** + * This task is explicitly a static class, so that it does not hold any references to the enclosing + * MetricsRegistry instance. + * + *

This is a subtle difference, but very important: With this static class, the enclosing class instance + * may become garbage-collectible, whereas with an anonymous inner class, the timer thread + * (which is a GC root) will hold a reference via the timer task and its enclosing instance pointer. + * Making the MetricsRegistry garbage collectible makes the java.util.Timer garbage collectible, + * which acts as a fail-safe to stop the timer thread and prevents resource leaks. + */ + private static final class ReporterTask extends TimerTask { + + private final Scheduled reporter; + + private ReporterTask(Scheduled reporter) { + this.reporter = reporter; + } + + @Override + public void run() { + try { + reporter.report(); + } catch (Throwable t) { + LOG.warn("Error while reporting metrics", t); + } + } + } +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/MetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/MetricGroup.java new file mode 100644 index 0000000000..ebb45be8da --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/MetricGroup.java @@ -0,0 +1,168 @@ +/* + * 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.metrics.base; + + +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Meter; + +import java.util.Map; + +/** + * + * copy from https://github.com/apache/flink + * + * A MetricGroup is a named container for {@link org.apache.flink.metrics.Metric Metrics} and further metric subgroups. + * + *

Instances of this class can be used to register new metrics with jLogstash and to create a nested + * hierarchy based on the group names. + * + *

A MetricGroup is uniquely identified by it's place in the hierarchy and name. + */ +public interface MetricGroup { + + // ------------------------------------------------------------------------ + // Metrics + // ------------------------------------------------------------------------ + + /** + * Creates and registers a new {@link Counter} with jLogstash. + * + * @param name name of the counter + * @return the created counter + */ + Counter counter(int name); + + /** + * Creates and registers a new {@link Counter} with jLogstash. + * + * @param name name of the counter + * @return the created counter + */ + Counter counter(String name); + + /** + * Registers a {@link Counter} with jLogstash. + * + * @param name name of the counter + * @param counter counter to register + * @param counter type + * @return the given counter + */ + C counter(int name, C counter); + + /** + * Registers a {@link Counter} with jLogstash. + * + * @param name name of the counter + * @param counter counter to register + * @param counter type + * @return the given counter + */ + C counter(String name, C counter); + + /** + * Registers a new {@link Gauge} with jLogstash. + * + * @param name name of the gauge + * @param gauge gauge to register + * @param return type of the gauge + * @return the given gauge + */ + > G gauge(int name, G gauge); + + /** + * Registers a new {@link Gauge} with jLogstash. + * + * @param name name of the gauge + * @param gauge gauge to register + * @param return type of the gauge + * @return the given gauge + */ + > G gauge(String name, G gauge); + + + /** + * Registers a new {@link Meter} with Flink. + * + * @param name name of the meter + * @param meter meter to register + * @param meter type + * @return the registered meter + */ + M meter(String name, M meter); + + /** + * Registers a new {@link Meter} with Flink. + * + * @param name name of the meter + * @param meter meter to register + * @param meter type + * @return the registered meter + */ + M meter(int name, M meter); + + + // ------------------------------------------------------------------------ + // Groups + // ------------------------------------------------------------------------ + + /** + * Creates a new MetricGroup and adds it to this groups sub-groups. + * + * @param name name of the group + * @return the created group + */ + MetricGroup addGroup(int name); + + /** + * Creates a new MetricGroup and adds it to this groups sub-groups. + * + * @param name name of the group + * @return the created group + */ + MetricGroup addGroup(String name); + + /** + * Gets the scope as an array of the scope components, for example + * {@code ["host-7", "taskmanager-2", "window_word_count", "my-mapper"]}. + * + * @see #getMetricIdentifier(String) + */ + String[] getScopeComponents(); + + + /** + * Returns a map of all variables and their associated value, for example + * {@code {""="host-7", ""="taskmanager-2"}}. + * + * @return map of all variables and their associated value + */ + Map getAllVariables(); + + /** + * Returns the fully qualified metric name, for example + * {@code "host-7.taskmanager-2.window_word_count.my-mapper.metricName"}. + * + * @param metricName metric name + * @return fully qualified metric name + */ + String getMetricIdentifier(String metricName); +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/reporter/MetricReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/reporter/MetricReporter.java new file mode 100644 index 0000000000..c37a794df6 --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/reporter/MetricReporter.java @@ -0,0 +1,54 @@ +package com.dtstack.flinkx.metrics.base.reporter; + + +import com.dtstack.flinkx.metrics.base.MetricGroup; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; + +/** + * copy from https://github.com/apache/flink + * + * Reporters are used to export {@link Metric Metrics} to an external backend. + * + *

Reporters are instantiated via reflection and must be public, non-abstract, and have a + * public no-argument constructor. + */ +public interface MetricReporter { + + /** + * Configures this reporter. Since reporters are instantiated generically and hence parameter-less, + * this method is the place where the reporters set their basic fields based on configuration values. + * + *

This method is always called first on a newly instantiated reporter. + * + * @param config A properties object that contains all parameters set for this reporter. + */ + void open(MetricConfig config); + + /** + * Closes this reporter. Should be used to close channels, streams and release resources. + */ + void close(); + + // ------------------------------------------------------------------------ + // adding / removing metrics + // ------------------------------------------------------------------------ + + /** + * Called when a new {@link Metric} was added. + * + * @param metric the metric that was added + * @param metricName the name of the metric + * @param group the group that contains the metric + */ + void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup group); + + /** + * Called when a {@link Metric} was should be removed. + * + * @param metric the metric that should be removed + * @param metricName the name of the metric + * @param group the group that contains the metric + */ + void notifyOfRemovedMetric(Metric metric, String metricName, MetricGroup group); +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/AbstractMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/AbstractMetricGroup.java new file mode 100644 index 0000000000..6645952681 --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/AbstractMetricGroup.java @@ -0,0 +1,387 @@ +/* + * 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.metrics.groups; + +import com.dtstack.flinkx.metrics.MetricRegistry; +import com.dtstack.flinkx.metrics.base.MetricGroup; +import com.dtstack.flinkx.metrics.scope.ScopeFormat; +import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.SimpleCounter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * copy from https://github.com/apache/flink + * + * Abstract {@link MetricGroup} that contains key functionality for adding metrics and groups. + * + *

IMPORTANT IMPLEMENTATION NOTE + * + *

This class uses locks for adding and removing metrics objects. This is done to + * prevent resource leaks in the presence of concurrently closing a group and adding + * metrics and subgroups. + * Since closing groups recursively closes the subgroups, the lock acquisition order must + * be strictly from parent group to subgroup. If at any point, a subgroup holds its group + * lock and calls a parent method that also acquires the lock, it will create a deadlock + * condition. + * + *

An AbstractMetricGroup can be {@link #close() closed}. Upon closing, the group de-register all metrics + * from any metrics reporter and any internal maps. Note that even closed metrics groups + * return Counters, Gauges, etc to the code, to prevent exceptions in the monitored code. + * These metrics simply do not get reported any more, when created on a closed group. + * + * @param The type of the parent MetricGroup + */ +public abstract class AbstractMetricGroup> implements MetricGroup { + + protected static final Logger LOG = LoggerFactory.getLogger(MetricGroup.class); + + // ------------------------------------------------------------------------ + + /** The parent group containing this group. */ + protected final A parent; + + /** The map containing all variables and their associated values, lazily computed. */ + protected volatile Map variables; + + /** The registry that this metrics group belongs to. */ + protected final MetricRegistry registry; + + /** All metrics that are directly contained in this group. */ + private final Map metrics = new HashMap<>(); + + /** All metric subgroups of this group. */ + private final Map groups = new HashMap<>(); + + /** The metrics scope represented by this group. + * For example ["host-7", "taskmanager-2", "window_word_count", "my-mapper" ]. */ + private final String[] scopeComponents; + + /** Array containing the metrics scope represented by this group for each reporter, as a concatenated string, lazily computed. + * For example: "host-7.taskmanager-2.window_word_count.my-mapper" */ + private final String[] scopeStrings; + + /** The logical metrics scope represented by this group, as a concatenated string, lazily computed. + * For example: "taskmanager.job.task" */ + private String logicalScopeString; + + /** Flag indicating whether this group has been closed. */ + private volatile boolean closed; + + // ------------------------------------------------------------------------ + + public AbstractMetricGroup(MetricRegistry registry, String[] scope, A parent) { + this.registry = checkNotNull(registry); + this.scopeComponents = checkNotNull(scope); + this.parent = parent; + this.scopeStrings = new String[registry.getNumberReporters()]; + } + + public Map getAllVariables() { + if (variables == null) { // avoid synchronization for common case + synchronized (this) { + if (variables == null) { + if (parent != null) { + variables = parent.getAllVariables(); + } else { // this case should only be true for mock groups + variables = new HashMap<>(); + } + } + } + } + return variables; + } + + /** + * Returns the logical scope of this group, for example + * {@code "taskmanager.job.task"}. + * + * @param filter character filter which is applied to the scope components + * @return logical scope + */ + public String getLogicalScope(CharacterFilter filter) { + return getLogicalScope(filter, registry.getDelimiter()); + } + + /** + * Returns the logical scope of this group, for example + * {@code "taskmanager.job.task"}. + * + * @param filter character filter which is applied to the scope components + * @return logical scope + */ + public String getLogicalScope(CharacterFilter filter, char delimiter) { + if (logicalScopeString == null) { + if (parent == null) { + logicalScopeString = getGroupName(filter); + } else { + logicalScopeString = parent.getLogicalScope(filter, delimiter) + delimiter + getGroupName(filter); + } + } + return logicalScopeString; + } + + /** + * Returns the name for this group, meaning what kind of entity it represents, for example "inputs". + * + * @return logical name for this group + */ + protected abstract String getGroupName(CharacterFilter filter); + + /** + * Gets the scope as an array of the scope components, for example + * {@code ["host-7", "taskmanager-2", "window_word_count", "my-mapper"]}. + * + * @see #getMetricIdentifier(String) + */ + public String[] getScopeComponents() { + return scopeComponents; + } + + /** + * Returns the fully qualified metric name, for example + * {@code "host-7.taskmanager-2.window_word_count.my-mapper.metricName"}. + * + * @param metricName metric name + * @return fully qualified metric name + */ + public String getMetricIdentifier(String metricName) { + return getMetricIdentifier(metricName, null); + } + + + /** + * Returns the fully qualified metric name, for example + * {@code "host-7.taskmanager-2.window_word_count.my-mapper.metricName"}. + * + * @param metricName metric name + * @param filter character filter which is applied to the scope components if not null. + * @return fully qualified metric name + */ + public String getMetricIdentifier(String metricName, CharacterFilter filter) { + return getMetricIdentifier(metricName, filter, -1); + } + + /** + * Returns the fully qualified metric name using the configured delimiter for the reporter with the given index, for example + * {@code "host-7.taskmanager-2.window_word_count.my-mapper.metricName"}. + * + * @param metricName metric name + * @param filter character filter which is applied to the scope components if not null. + * @param reporterIndex index of the reporter whose delimiter should be used + * @return fully qualified metric name + */ + public String getMetricIdentifier(String metricName, CharacterFilter filter, int reporterIndex) { + if (scopeStrings.length == 0 || (reporterIndex < 0 || reporterIndex >= scopeStrings.length)) { + char delimiter = registry.getDelimiter(); + String newScopeString; + if (filter != null) { + newScopeString = ScopeFormat.concat(filter, delimiter, scopeComponents); + metricName = filter.filterCharacters(metricName); + } else { + newScopeString = ScopeFormat.concat(delimiter, scopeComponents); + } + return newScopeString + delimiter + metricName; + } else { + char delimiter = registry.getDelimiter(reporterIndex); + if (scopeStrings[reporterIndex] == null) { + if (filter != null) { + scopeStrings[reporterIndex] = ScopeFormat.concat(filter, delimiter, scopeComponents); + } else { + scopeStrings[reporterIndex] = ScopeFormat.concat(delimiter, scopeComponents); + } + } + if (filter != null) { + metricName = filter.filterCharacters(metricName); + } + return scopeStrings[reporterIndex] + delimiter + metricName; + } + } + + // ------------------------------------------------------------------------ + // Closing + // ------------------------------------------------------------------------ + + public void close() { + synchronized (this) { + if (!closed) { + closed = true; + + // close all subgroups + for (AbstractMetricGroup group : groups.values()) { + group.close(); + } + groups.clear(); + + // un-register all directly contained metrics + for (Map.Entry metric : metrics.entrySet()) { + registry.unregister(metric.getValue(), metric.getKey(), this); + } + metrics.clear(); + } + } + } + + public final boolean isClosed() { + return closed; + } + + // ----------------------------------------------------------------------------------------------------------------- + // Metrics + // ----------------------------------------------------------------------------------------------------------------- + + @Override + public Counter counter(int name) { + return counter(String.valueOf(name)); + } + + @Override + public Counter counter(String name) { + return counter(name, new SimpleCounter()); + } + + @Override + public C counter(int name, C counter) { + return counter(String.valueOf(name), counter); + } + + @Override + public C counter(String name, C counter) { + addMetric(name, counter); + return counter; + } + + @Override + public > G gauge(int name, G gauge) { + return gauge(String.valueOf(name), gauge); + } + + @Override + public > G gauge(String name, G gauge) { + addMetric(name, gauge); + return gauge; + } + + @Override + public M meter(int name, M meter) { + return meter(String.valueOf(name), meter); + } + + @Override + public M meter(String name, M meter) { + addMetric(name, meter); + return meter; + } + + /** + * Adds the given metric to the group and registers it at the registry, if the group + * is not yet closed, and if no metric with the same name has been registered before. + * + * @param name the name to register the metric under + * @param metric the metric to register + */ + protected void addMetric(String name, Metric metric) { + if (metric == null) { + LOG.warn("Ignoring attempted registration of a metric due to being null for name {}.", name); + return; + } + // add the metric only if the group is still open + synchronized (this) { + if (!closed) { + // immediately put without a 'contains' check to optimize the common case (no collision) + // collisions are resolved later + Metric prior = metrics.put(name, metric); + + // check for collisions with other metric names + if (prior == null) { + // no other metric with this name yet + + if (groups.containsKey(name)) { + // we warn here, rather than failing, because metrics are tools that should not fail the + // program when used incorrectly + LOG.warn("Name collision: Adding a metric with the same name as a metric subgroup: '" + + name + "'. Metric might not get properly reported. " + Arrays.toString(scopeComponents)); + } + + registry.register(metric, name, this); + } + else { + // we had a collision. put back the original value + metrics.put(name, prior); + + // we warn here, rather than failing, because metrics are tools that should not fail the + // program when used incorrectly + LOG.warn("Name collision: Group already contains a Metric with the name '" + + name + "'. Metric will not be reported." + Arrays.toString(scopeComponents)); + } + } + } + } + + // ------------------------------------------------------------------------ + // Groups + // ------------------------------------------------------------------------ + + @Override + public MetricGroup addGroup(int name) { + return addGroup(String.valueOf(name)); + } + + @Override + public MetricGroup addGroup(String name) { + synchronized (this) { + if (!closed) { + // adding a group with the same name as a metric creates problems in many reporters/dashboards + // we warn here, rather than failing, because metrics are tools that should not fail the + // program when used incorrectly + if (metrics.containsKey(name)) { + LOG.warn("Name collision: Adding a metric subgroup with the same name as an existing metric: '" + + name + "'. Metric might not get properly reported. " + Arrays.toString(scopeComponents)); + } + + AbstractMetricGroup newGroup = new GenericMetricGroup(registry, this, name); + AbstractMetricGroup prior = groups.put(name, newGroup); + if (prior == null) { + // no prior group with that name + return newGroup; + } else { + // had a prior group with that name, add the prior group back + groups.put(name, prior); + return prior; + } + } + else { + // return a non-registered group that is immediately closed already + GenericMetricGroup closedGroup = new GenericMetricGroup(registry, this, name); + closedGroup.close(); + return closedGroup; + } + } + } +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ComponentMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ComponentMetricGroup.java new file mode 100644 index 0000000000..e42dae093f --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ComponentMetricGroup.java @@ -0,0 +1,111 @@ +/* + * 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.metrics.groups; + + +import com.dtstack.flinkx.metrics.MetricRegistry; + +import java.util.HashMap; +import java.util.Map; + +/** + * + * copy from https://github.com/apache/flink + * + * Abstract {@link com.dtstack.flinkx.metrics.base.MetricGroup} for system components (e.g., + * TaskManager, Job, Task, Operator). + * + *

Usually, the scope of metrics is simply the hierarchy of the containing groups. For example + * the Metric {@code "MyMetric"} in group {@code "B"} nested in group {@code "A"} would have a + * fully scoped name of {@code "A.B.MyMetric"}, with {@code "A.B"} being the Metric's scope. + * + *

Component groups, however, have configurable scopes. This allow users to include or exclude + * certain identifiers from the scope. The scope for metrics belonging to the "Task" + * group could for example include the task attempt number (more fine grained identification), or + * exclude it (for continuity of the namespace across failure and recovery). + * + * @param

The type of the parent MetricGroup. + */ +public abstract class ComponentMetricGroup

> extends AbstractMetricGroup

{ + + /** + * Creates a new ComponentMetricGroup. + * + * @param registry registry to register new metrics with + * @param scope the scope of the group + */ + public ComponentMetricGroup(MetricRegistry registry, String[] scope, P parent) { + super(registry, scope, parent); + } + + @Override + public Map getAllVariables() { + if (variables == null) { // avoid synchronization for common case + synchronized (this) { + if (variables == null) { + Map tmpVariables = new HashMap<>(); + putVariables(tmpVariables); + if (parent != null) { // not true for Job-/TaskManagerMetricGroup + tmpVariables.putAll(parent.getAllVariables()); + } + variables = tmpVariables; + } + } + } + return variables; + } + + /** + * Enters all variables specific to this ComponentMetricGroup and their associated values into the map. + * + * @param variables map to enter variables and their values into + */ + protected abstract void putVariables(Map variables); + + /** + * Closes the component group by removing and closing all metrics and subgroups + * (inherited from {@link AbstractMetricGroup}), plus closing and removing all dedicated + * component subgroups. + */ + @Override + public void close() { + synchronized (this) { + if (!isClosed()) { + // remove all metrics and generic subgroups + super.close(); + + // remove and close all subcomponent metrics + for (ComponentMetricGroup group : subComponents()) { + group.close(); + } + } + } + } + + // ------------------------------------------------------------------------ + // Component Metric Group Specifics + // ------------------------------------------------------------------------ + + /** + * Gets all component metric groups that are contained in this component metric group. + * + * @return All component metric groups that are contained in this component metric group. + */ + protected abstract Iterable subComponents(); +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/FrontMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/FrontMetricGroup.java new file mode 100644 index 0000000000..710c4e7818 --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/FrontMetricGroup.java @@ -0,0 +1,48 @@ +/* + * 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.metrics.groups; + + +import org.apache.flink.metrics.CharacterFilter; + +/** + * copy from https://github.com/apache/flink + */ +public class FrontMetricGroup

> extends ProxyMetricGroup

{ + + protected int reporterIndex; + + public FrontMetricGroup(int reporterIndex, P reference) { + super(reference); + this.reporterIndex = reporterIndex; + } + + @Override + public String getMetricIdentifier(String metricName) { + return parentMetricGroup.getMetricIdentifier(metricName); + } + + public String getLogicalScope(CharacterFilter filter) { + return parentMetricGroup.getLogicalScope(filter); + } + + public String getLogicalScope(CharacterFilter filter, char delimiter) { + return parentMetricGroup.getLogicalScope(filter, delimiter); + } +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/GenericMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/GenericMetricGroup.java new file mode 100644 index 0000000000..5cffd60c47 --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/GenericMetricGroup.java @@ -0,0 +1,57 @@ +/* + * 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.metrics.groups; + + +import com.dtstack.flinkx.metrics.MetricRegistry; +import org.apache.flink.metrics.CharacterFilter; + +/** + * copy from https://github.com/apache/flink + */ +public class GenericMetricGroup extends AbstractMetricGroup> { + + /** The name of this group. */ + private String name; + + public GenericMetricGroup(MetricRegistry registry, AbstractMetricGroup parent, String name) { + super(registry, makeScopeComponents(parent, name), parent); + this.name = name; + } + + // ------------------------------------------------------------------------ + + private static String[] makeScopeComponents(AbstractMetricGroup parent, String name) { + if (parent != null) { + String[] parentComponents = parent.getScopeComponents(); + if (parentComponents != null && parentComponents.length > 0) { + String[] parts = new String[parentComponents.length + 1]; + System.arraycopy(parentComponents, 0, parts, 0, parentComponents.length); + parts[parts.length - 1] = name; + return parts; + } + } + return new String[] { name }; + } + + @Override + protected String getGroupName(CharacterFilter filter) { + return filter.filterCharacters(name); + } +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineInputMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineInputMetricGroup.java index a94f5e4177..b269811e6d 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineInputMetricGroup.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineInputMetricGroup.java @@ -20,16 +20,14 @@ import com.dtstack.flinkx.constants.Metrics; +import com.dtstack.flinkx.metrics.MetricRegistry; import org.apache.flink.metrics.CharacterFilter; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.MeterView; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.runtime.metrics.MetricNames; -import org.apache.flink.runtime.metrics.MetricRegistry; -import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; -import org.apache.flink.runtime.metrics.groups.ComponentMetricGroup; -import org.apache.flink.runtime.metrics.scope.ScopeFormat; +import com.dtstack.flinkx.metrics.scope.ScopeFormat; import java.util.ArrayList; import java.util.List; @@ -105,11 +103,6 @@ protected String getGroupName(CharacterFilter filter) { return "pipeline"; } - @Override - protected QueryScopeInfo createQueryServiceMetricInfo(CharacterFilter filter) { - return null; - } - // ============================================================================================ // Metric Reuse // ============================================================================================ diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineOutputMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineOutputMetricGroup.java index b8aad9bfb5..6c609971b3 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineOutputMetricGroup.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineOutputMetricGroup.java @@ -20,16 +20,14 @@ import com.dtstack.flinkx.constants.Metrics; +import com.dtstack.flinkx.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.metrics.CharacterFilter; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.MeterView; import org.apache.flink.metrics.SimpleCounter; -import org.apache.flink.runtime.metrics.MetricRegistry; -import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; -import org.apache.flink.runtime.metrics.groups.ComponentMetricGroup; -import org.apache.flink.runtime.metrics.scope.ScopeFormat; +import com.dtstack.flinkx.metrics.scope.ScopeFormat; import java.util.ArrayList; import java.util.List; @@ -109,11 +107,6 @@ protected String getGroupName(CharacterFilter filter) { return "pipeline"; } - @Override - protected QueryScopeInfo createQueryServiceMetricInfo(CharacterFilter filter) { - return null; - } - // ============================================================================================ // Metric Reuse // ============================================================================================ diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ProxyMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ProxyMetricGroup.java new file mode 100644 index 0000000000..17a5e59e0a --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ProxyMetricGroup.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.metrics.groups; + + +import com.dtstack.flinkx.metrics.base.MetricGroup; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Meter; + +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * + * copy from https://github.com/apache/flink + * + * Metric group which forwards all registration calls to its parent metric group. + * + * @param

Type of the parent metric group + */ +public class ProxyMetricGroup

implements MetricGroup { + + protected final P parentMetricGroup; + + public ProxyMetricGroup(P parentMetricGroup) { + this.parentMetricGroup = checkNotNull(parentMetricGroup); + } + + @Override + public final Counter counter(int name) { + return parentMetricGroup.counter(name); + } + + @Override + public final Counter counter(String name) { + return parentMetricGroup.counter(name); + } + + @Override + public final C counter(int name, C counter) { + return parentMetricGroup.counter(name, counter); + } + + @Override + public final C counter(String name, C counter) { + return parentMetricGroup.counter(name, counter); + } + + @Override + public final > G gauge(int name, G gauge) { + return parentMetricGroup.gauge(name, gauge); + } + + @Override + public final > G gauge(String name, G gauge) { + return parentMetricGroup.gauge(name, gauge); + } + + @Override + public M meter(String name, M meter) { + return parentMetricGroup.meter(name, meter); + } + + @Override + public M meter(int name, M meter) { + return parentMetricGroup.meter(name, meter); + } + + @Override + public final MetricGroup addGroup(int name) { + return parentMetricGroup.addGroup(name); + } + + @Override + public final MetricGroup addGroup(String name) { + return parentMetricGroup.addGroup(name); + } + + @Override + public String[] getScopeComponents() { + return parentMetricGroup.getScopeComponents(); + } + + @Override + public Map getAllVariables() { + return parentMetricGroup.getAllVariables(); + } + + @Override + public String getMetricIdentifier(String metricName) { + return parentMetricGroup.getMetricIdentifier(metricName); + } + +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/AbstractPrometheusReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/AbstractPrometheusReporter.java index 3a0dd2653e..5e361ccf2b 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/AbstractPrometheusReporter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/AbstractPrometheusReporter.java @@ -18,6 +18,10 @@ package com.dtstack.flinkx.metrics.promethues; +import com.dtstack.flinkx.metrics.base.MetricGroup; +import com.dtstack.flinkx.metrics.base.reporter.MetricReporter; +import com.dtstack.flinkx.metrics.groups.AbstractMetricGroup; +import com.dtstack.flinkx.metrics.groups.FrontMetricGroup; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; @@ -25,10 +29,6 @@ import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricConfig; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.metrics.reporter.MetricReporter; -import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; -import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; import io.prometheus.client.Collector; import io.prometheus.client.CollectorRegistry; import org.apache.flink.metrics.CharacterFilter; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporter.java index 9fb982b07a..0a488ec145 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporter.java @@ -20,7 +20,6 @@ import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricConfig; -import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.reporter.Scheduled; import io.prometheus.client.CollectorRegistry; import io.prometheus.client.exporter.PushGateway; @@ -35,7 +34,7 @@ import static com.dtstack.flinkx.metrics.promethues.PrometheusPushGatewayReporterOptions.RANDOM_JOB_NAME_SUFFIX; /** - * {@link MetricReporter} that exports {@link Metric Metrics} via Prometheus {@link PushGateway}. + * {@link com.dtstack.flinkx.metrics.base.reporter.MetricReporter} that exports {@link Metric Metrics} via Prometheus {@link PushGateway}. */ public class PrometheusPushGatewayReporter extends AbstractPrometheusReporter implements Scheduled { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusReporter.java index 67b6c05b8e..ca8f5596fb 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusReporter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusReporter.java @@ -20,7 +20,6 @@ import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricConfig; -import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.util.NetUtils; import org.apache.flink.util.Preconditions; import io.prometheus.client.exporter.HTTPServer; @@ -29,7 +28,7 @@ import java.util.Iterator; /** - * {@link MetricReporter} that exports {@link Metric Metrics} via Prometheus. + * {@link com.dtstack.flinkx.metrics.base.reporter.MetricReporter} that exports {@link Metric Metrics} via Prometheus. */ public class PrometheusReporter extends AbstractPrometheusReporter { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/PipelineScopeFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/PipelineScopeFormat.java index 883bf93c7b..8e7b8d0f25 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/PipelineScopeFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/PipelineScopeFormat.java @@ -19,8 +19,6 @@ package com.dtstack.flinkx.metrics.scope; -import org.apache.flink.runtime.metrics.scope.ScopeFormat; - public class PipelineScopeFormat extends ScopeFormat { public PipelineScopeFormat(String format) { diff --git a/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormat.java similarity index 99% rename from flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormat.java rename to flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormat.java index ac37356ac2..b5a574fd81 100644 --- a/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormat.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.metrics.scope; +package com.dtstack.flinkx.metrics.scope; import org.apache.flink.metrics.CharacterFilter; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormats.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormats.java new file mode 100644 index 0000000000..7ba20bdebe --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormats.java @@ -0,0 +1,57 @@ +/* + * 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.metrics.scope; + +/** + * A container for component scope formats. + */ +public final class ScopeFormats { + + private final PipelineScopeFormat pipelineScopeFormat; + + // ------------------------------------------------------------------------ + + /** + * Creates all scope formats, based on the given scope format strings. + */ + private ScopeFormats() { + this.pipelineScopeFormat = new PipelineScopeFormat(".flinkx..."); + } + + // ------------------------------------------------------------------------ + // Accessors + // ------------------------------------------------------------------------ + + public PipelineScopeFormat getPipelineScopeFormat() { + return pipelineScopeFormat; + } + + // ------------------------------------------------------------------------ + // Parsing from Config + // ------------------------------------------------------------------------ + + /** + * Creates the scope formats as defined in the given configuration. + * + * @return The ScopeFormats parsed from the configuration + */ + public static ScopeFormats fromDefault() { + return new ScopeFormats(); + } +} diff --git a/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java b/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java deleted file mode 100644 index 630211837d..0000000000 --- a/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java +++ /dev/null @@ -1,436 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.metrics; - -import akka.actor.ActorRef; -import akka.actor.ActorSystem; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.time.Time; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Metric; -import org.apache.flink.metrics.MetricConfig; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.metrics.View; -import org.apache.flink.metrics.reporter.MetricReporter; -import org.apache.flink.metrics.reporter.Scheduled; -import org.apache.flink.runtime.akka.ActorUtils; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.metrics.dump.MetricQueryService; -import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; -import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; -import org.apache.flink.runtime.metrics.scope.ScopeFormats; -import org.apache.flink.runtime.util.ExecutorThreadFactory; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.ExecutorUtils; -import org.apache.flink.util.FlinkException; -import org.apache.flink.util.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.TimerTask; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - -/** - * A MetricRegistry keeps track of all registered {@link Metric Metrics}. It serves as the - * connection between {@link MetricGroup MetricGroups} and {@link MetricReporter MetricReporters}. - */ -public class MetricRegistryImpl implements MetricRegistry { - static final Logger LOG = LoggerFactory.getLogger(MetricRegistryImpl.class); - - private final Object lock = new Object(); - - private final List reporters; - private final ScheduledExecutorService executor; - - private final ScopeFormats scopeFormats = ScopeFormats.fromConfig(new Configuration()); - private final char globalDelimiter; - private final List delimiters; - - private final CompletableFuture terminationFuture; - - @Nullable - private ActorRef queryService; - - @Nullable - private String metricQueryServicePath; - - private ViewUpdater viewUpdater; - - private boolean isShutdown; - - public MetricRegistryImpl createMetricRegistry() { - MetricRegistryConfiguration config = MetricRegistryConfiguration.fromConfiguration(new Configuration()); - return new MetricRegistryImpl(config); - } - - /** - * Creates a new MetricRegistry and starts the configured reporter. - */ - public MetricRegistryImpl(MetricRegistryConfiguration config) { - - this.globalDelimiter = config.getDelimiter(); - this.delimiters = new ArrayList<>(10); - this.terminationFuture = new CompletableFuture<>(); - this.isShutdown = false; - - // second, instantiate any custom configured reporters - this.reporters = new ArrayList<>(4); - - List> reporterConfigurations = config.getReporterConfigurations(); - - this.executor = Executors.newSingleThreadScheduledExecutor(new ExecutorThreadFactory("Flink-MetricRegistry")); - - this.queryService = null; - this.metricQueryServicePath = null; - - if (reporterConfigurations.isEmpty()) { - // no reporters defined - // by default, don't report anything - LOG.info("No metrics reporter configured, no metrics will be exposed/reported."); - } else { - // we have some reporters so - for (Tuple2 reporterConfiguration: reporterConfigurations) { - String namedReporter = reporterConfiguration.f0; - Configuration reporterConfig = reporterConfiguration.f1; - - final String className = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, null); - if (className == null) { - LOG.error("No reporter class set for reporter " + namedReporter + ". Metrics might not be exposed/reported."); - continue; - } - - try { - String configuredPeriod = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_INTERVAL_SUFFIX, null); - TimeUnit timeunit = TimeUnit.SECONDS; - long period = 10; - - if (configuredPeriod != null) { - try { - String[] interval = configuredPeriod.split(" "); - period = Long.parseLong(interval[0]); - timeunit = TimeUnit.valueOf(interval[1]); - } - catch (Exception e) { - LOG.error("Cannot parse report interval from config: " + configuredPeriod + - " - please use values like '10 SECONDS' or '500 MILLISECONDS'. " + - "Using default reporting interval."); - } - } - - Class reporterClass = Class.forName(className); - MetricReporter reporterInstance = (MetricReporter) reporterClass.newInstance(); - - MetricConfig metricConfig = new MetricConfig(); - reporterConfig.addAllToProperties(metricConfig); - LOG.info("Configuring {} with {}.", namedReporter, metricConfig); - reporterInstance.open(metricConfig); - - if (reporterInstance instanceof Scheduled) { - LOG.info("Periodically reporting metrics in intervals of {} {} for reporter {} of type {}.", period, timeunit.name(), namedReporter, className); - - executor.scheduleWithFixedDelay( - new MetricRegistryImpl.ReporterTask((Scheduled) reporterInstance), period, period, timeunit); - } else { - LOG.info("Reporting metrics for reporter {} of type {}.", namedReporter, className); - } - reporters.add(reporterInstance); - - String delimiterForReporter = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, String.valueOf(globalDelimiter)); - if (delimiterForReporter.length() != 1) { - LOG.warn("Failed to parse delimiter '{}' for reporter '{}', using global delimiter '{}'.", delimiterForReporter, namedReporter, globalDelimiter); - delimiterForReporter = String.valueOf(globalDelimiter); - } - this.delimiters.add(delimiterForReporter.charAt(0)); - } - catch (Throwable t) { - LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", namedReporter, t); - } - } - } - } - - /** - * Initializes the MetricQueryService. - * - * @param actorSystem ActorSystem to create the MetricQueryService on - * @param resourceID resource ID used to disambiguate the actor name - */ - public void startQueryService(ActorSystem actorSystem, ResourceID resourceID) { - synchronized (lock) { - Preconditions.checkState(!isShutdown(), "The metric registry has already been shut down."); - - try { - queryService = MetricQueryService.startMetricQueryService(actorSystem, resourceID); - metricQueryServicePath = AkkaUtils.getAkkaURL(actorSystem, queryService); - } catch (Exception e) { - LOG.warn("Could not start MetricDumpActor. No metrics will be submitted to the WebInterface.", e); - } - } - } - - /** - * Returns the address under which the {@link MetricQueryService} is reachable. - * - * @return address of the metric query service - */ - @Override - @Nullable - public String getMetricQueryServicePath() { - return metricQueryServicePath; - } - - @Override - public char getDelimiter() { - return this.globalDelimiter; - } - - @Override - public char getDelimiter(int reporterIndex) { - try { - return delimiters.get(reporterIndex); - } catch (IndexOutOfBoundsException e) { - LOG.warn("Delimiter for reporter index {} not found, returning global delimiter.", reporterIndex); - return this.globalDelimiter; - } - } - - @Override - public int getNumberReporters() { - return reporters.size(); - } - - @VisibleForTesting - public List getReporters() { - return reporters; - } - - /** - * Returns whether this registry has been shutdown. - * - * @return true, if this registry was shutdown, otherwise false - */ - public boolean isShutdown() { - synchronized (lock) { - return isShutdown; - } - } - - /** - * Shuts down this registry and the associated {@link MetricReporter}. - * - *

NOTE: This operation is asynchronous and returns a future which is completed - * once the shutdown operation has been completed. - * - * @return Future which is completed once the {@link MetricRegistryImpl} - * is shut down. - */ - public CompletableFuture shutdown() { - synchronized (lock) { - if (isShutdown) { - return terminationFuture; - } else { - isShutdown = true; - final Collection> terminationFutures = new ArrayList<>(3); - final Time gracePeriod = Time.seconds(1L); - - if (queryService != null) { - final CompletableFuture queryServiceTerminationFuture = ActorUtils.nonBlockingShutDown( - gracePeriod.toMilliseconds(), - TimeUnit.MILLISECONDS, - queryService); - - terminationFutures.add(queryServiceTerminationFuture); - } - - Throwable throwable = null; - for (MetricReporter reporter : reporters) { - try { - reporter.close(); - } catch (Throwable t) { - throwable = ExceptionUtils.firstOrSuppressed(t, throwable); - } - } - reporters.clear(); - - if (throwable != null) { - terminationFutures.add( - FutureUtils.completedExceptionally( - new FlinkException("Could not shut down the metric reporters properly.", throwable))); - } - - final CompletableFuture executorShutdownFuture = ExecutorUtils.nonBlockingShutdown( - gracePeriod.toMilliseconds(), - TimeUnit.MILLISECONDS, - executor); - - terminationFutures.add(executorShutdownFuture); - - FutureUtils - .completeAll(terminationFutures) - .whenComplete( - (Void ignored, Throwable error) -> { - if (error != null) { - terminationFuture.completeExceptionally(error); - } else { - terminationFuture.complete(null); - } - }); - - return terminationFuture; - } - } - } - - @Override - public ScopeFormats getScopeFormats() { - return scopeFormats; - } - - // ------------------------------------------------------------------------ - // Metrics (de)registration - // ------------------------------------------------------------------------ - - @Override - public void register(Metric metric, String metricName, AbstractMetricGroup group) { - synchronized (lock) { - if (isShutdown()) { - LOG.warn("Cannot register metric, because the MetricRegistry has already been shut down."); - } else { - if (reporters != null) { - for (int i = 0; i < reporters.size(); i++) { - MetricReporter reporter = reporters.get(i); - try { - if (reporter != null) { - FrontMetricGroup front = new FrontMetricGroup>(i, group); - reporter.notifyOfAddedMetric(metric, metricName, front); - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - } - } - try { - if (queryService != null) { - MetricQueryService.notifyOfAddedMetric(queryService, metric, metricName, group); - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - try { - if (metric instanceof View) { - if (viewUpdater == null) { - viewUpdater = new ViewUpdater(executor); - } - viewUpdater.notifyOfAddedView((View) metric); - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - } - } - } - - @Override - public void unregister(Metric metric, String metricName, AbstractMetricGroup group) { - synchronized (lock) { - if (isShutdown()) { - LOG.warn("Cannot unregister metric, because the MetricRegistry has already been shut down."); - } else { - if (reporters != null) { - for (int i = 0; i < reporters.size(); i++) { - try { - MetricReporter reporter = reporters.get(i); - if (reporter != null) { - FrontMetricGroup front = new FrontMetricGroup>(i, group); - reporter.notifyOfRemovedMetric(metric, metricName, front); - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - } - } - try { - if (queryService != null) { - MetricQueryService.notifyOfRemovedMetric(queryService, metric); - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - try { - if (metric instanceof View) { - if (viewUpdater != null) { - viewUpdater.notifyOfRemovedView((View) metric); - } - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - } - } - } - - // ------------------------------------------------------------------------ - - @VisibleForTesting - @Nullable - public ActorRef getQueryService() { - return queryService; - } - - // ------------------------------------------------------------------------ - - /** - * This task is explicitly a static class, so that it does not hold any references to the enclosing - * MetricsRegistry instance. - * - *

This is a subtle difference, but very important: With this static class, the enclosing class instance - * may become garbage-collectible, whereas with an anonymous inner class, the timer thread - * (which is a GC root) will hold a reference via the timer task and its enclosing instance pointer. - * Making the MetricsRegistry garbage collectible makes the java.util.Timer garbage collectible, - * which acts as a fail-safe to stop the timer thread and prevents resource leaks. - */ - private static final class ReporterTask extends TimerTask { - - private final Scheduled reporter; - - private ReporterTask(Scheduled reporter) { - this.reporter = reporter; - } - - @Override - public void run() { - try { - reporter.report(); - } catch (Throwable t) { - LOG.warn("Error while reporting metrics", t); - } - } - } -} diff --git a/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormats.java b/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormats.java deleted file mode 100644 index a395881956..0000000000 --- a/flinkx-core/src/main/java/org/apache/flink/runtime/metrics/scope/ScopeFormats.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.metrics.scope; - -import com.dtstack.flinkx.metrics.scope.PipelineScopeFormat; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.MetricOptions; - -/** - * A container for component scope formats. - */ -public final class ScopeFormats { - - private final JobManagerScopeFormat jobManagerFormat; - private final JobManagerJobScopeFormat jobManagerJobFormat; - private final TaskManagerScopeFormat taskManagerFormat; - private final TaskManagerJobScopeFormat taskManagerJobFormat; - private final TaskScopeFormat taskFormat; - private final OperatorScopeFormat operatorFormat; - private final PipelineScopeFormat pipelineScopeFormat; - - // ------------------------------------------------------------------------ - - /** - * Creates all scope formats, based on the given scope format strings. - */ - private ScopeFormats( - String jobManagerFormat, - String jobManagerJobFormat, - String taskManagerFormat, - String taskManagerJobFormat, - String taskFormat, - String operatorFormat) { - this.jobManagerFormat = new JobManagerScopeFormat(jobManagerFormat); - this.jobManagerJobFormat = new JobManagerJobScopeFormat(jobManagerJobFormat, this.jobManagerFormat); - this.taskManagerFormat = new TaskManagerScopeFormat(taskManagerFormat); - this.taskManagerJobFormat = new TaskManagerJobScopeFormat(taskManagerJobFormat, this.taskManagerFormat); - this.taskFormat = new TaskScopeFormat(taskFormat, this.taskManagerJobFormat); - this.operatorFormat = new OperatorScopeFormat(operatorFormat, this.taskFormat); - this.pipelineScopeFormat = new PipelineScopeFormat(".flinkx..."); - } - - // ------------------------------------------------------------------------ - // Accessors - // ------------------------------------------------------------------------ - - public JobManagerScopeFormat getJobManagerFormat() { - return this.jobManagerFormat; - } - - public TaskManagerScopeFormat getTaskManagerFormat() { - return this.taskManagerFormat; - } - - public TaskManagerJobScopeFormat getTaskManagerJobFormat() { - return this.taskManagerJobFormat; - } - - public JobManagerJobScopeFormat getJobManagerJobFormat() { - return this.jobManagerJobFormat; - } - - public TaskScopeFormat getTaskFormat() { - return this.taskFormat; - } - - public OperatorScopeFormat getOperatorFormat() { - return this.operatorFormat; - } - - public PipelineScopeFormat getPipelineScopeFormat() { - return pipelineScopeFormat; - } - - // ------------------------------------------------------------------------ - // Parsing from Config - // ------------------------------------------------------------------------ - - /** - * Creates the scope formats as defined in the given configuration. - * - * @param config The configuration that defines the formats - * @return The ScopeFormats parsed from the configuration - */ - public static ScopeFormats fromConfig(Configuration config) { - String jmFormat = config.getString(MetricOptions.SCOPE_NAMING_JM); - String jmJobFormat = config.getString(MetricOptions.SCOPE_NAMING_JM_JOB); - String tmFormat = config.getString(MetricOptions.SCOPE_NAMING_TM); - String tmJobFormat = config.getString(MetricOptions.SCOPE_NAMING_TM_JOB); - String taskFormat = config.getString(MetricOptions.SCOPE_NAMING_TASK); - String operatorFormat = config.getString(MetricOptions.SCOPE_NAMING_OPERATOR); - - return new ScopeFormats(jmFormat, jmJobFormat, tmFormat, tmJobFormat, taskFormat, operatorFormat); - } -} From 9a3f20ba39a1695e9159d8e6c0ac8b88a9c15940 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 18 Mar 2019 10:56:19 +0800 Subject: [PATCH 19/96] Not convert type to string when write to ES --- .../src/main/java/com/dtstack/flinkx/es/EsUtil.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java index 906b13cfe2..10625da5cf 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java @@ -138,8 +138,7 @@ public static Map rowToJsonMap(Row row, List fields, Lis String key = parts[parts.length - 1]; Object col = row.getField(i); if(col != null) { - Object value = StringUtil.col2string(col, types.get(i)); - currMap.put(key, value); + currMap.put(key, col); } } From 2eedb655fa0d8fd17ddeae19c7d7b9da441dad42 Mon Sep 17 00:00:00 2001 From: toutian Date: Mon, 18 Mar 2019 11:19:09 +0800 Subject: [PATCH 20/96] comment --- .../flinkx/metrics/base/MetricGroup.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/MetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/MetricGroup.java index ebb45be8da..23abb95925 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/MetricGroup.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/MetricGroup.java @@ -31,7 +31,7 @@ * * A MetricGroup is a named container for {@link org.apache.flink.metrics.Metric Metrics} and further metric subgroups. * - *

Instances of this class can be used to register new metrics with jLogstash and to create a nested + *

Instances of this class can be used to register new metrics with Flinkx and to create a nested * hierarchy based on the group names. * *

A MetricGroup is uniquely identified by it's place in the hierarchy and name. @@ -43,7 +43,7 @@ public interface MetricGroup { // ------------------------------------------------------------------------ /** - * Creates and registers a new {@link Counter} with jLogstash. + * Creates and registers a new {@link Counter} with Flinkx. * * @param name name of the counter * @return the created counter @@ -51,7 +51,7 @@ public interface MetricGroup { Counter counter(int name); /** - * Creates and registers a new {@link Counter} with jLogstash. + * Creates and registers a new {@link Counter} with Flinkx. * * @param name name of the counter * @return the created counter @@ -59,7 +59,7 @@ public interface MetricGroup { Counter counter(String name); /** - * Registers a {@link Counter} with jLogstash. + * Registers a {@link Counter} with Flinkx. * * @param name name of the counter * @param counter counter to register @@ -69,7 +69,7 @@ public interface MetricGroup { C counter(int name, C counter); /** - * Registers a {@link Counter} with jLogstash. + * Registers a {@link Counter} with Flinkx. * * @param name name of the counter * @param counter counter to register @@ -79,7 +79,7 @@ public interface MetricGroup { C counter(String name, C counter); /** - * Registers a new {@link Gauge} with jLogstash. + * Registers a new {@link Gauge} with Flinkx. * * @param name name of the gauge * @param gauge gauge to register @@ -89,7 +89,7 @@ public interface MetricGroup { > G gauge(int name, G gauge); /** - * Registers a new {@link Gauge} with jLogstash. + * Registers a new {@link Gauge} with Flinkx. * * @param name name of the gauge * @param gauge gauge to register @@ -100,7 +100,7 @@ public interface MetricGroup { /** - * Registers a new {@link Meter} with Flink. + * Registers a new {@link Meter} with Flinkx. * * @param name name of the meter * @param meter meter to register @@ -110,7 +110,7 @@ public interface MetricGroup { M meter(String name, M meter); /** - * Registers a new {@link Meter} with Flink. + * Registers a new {@link Meter} with Flinkx. * * @param name name of the meter * @param meter meter to register From 08901dcacba15771659b9ebc75d7feb32aeb1443 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 18 Mar 2019 11:23:17 +0800 Subject: [PATCH 21/96] Convert string type to the specified type --- .../src/main/java/com/dtstack/flinkx/es/EsUtil.java | 4 ++++ .../com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java index 10625da5cf..d64e60ada7 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java @@ -138,6 +138,10 @@ public static Map rowToJsonMap(Row row, List fields, Lis String key = parts[parts.length - 1]; Object col = row.getField(i); if(col != null) { + if (col instanceof String){ + col = StringUtil.string2col(String.valueOf(col), types.get(i), null); + } + currMap.put(key, col); } diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java index c81618090f..d8c1a64ea4 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java @@ -204,7 +204,7 @@ public Row nextRecordInternal(Row row) throws IOException { val = metaColumn.getValue(); } - if(val instanceof String){ + if(val instanceof String || val instanceof org.apache.hadoop.io.Text){ val = HdfsUtil.string2col(String.valueOf(val),metaColumn.getType(),metaColumn.getTimeFormat()); } From 984bfd4e90125d108de48f5de6729f0fb34d3fd5 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 18 Mar 2019 11:59:34 +0800 Subject: [PATCH 22/96] Convert type to the specified type --- .../src/main/java/com/dtstack/flinkx/es/EsUtil.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java index d64e60ada7..ee542c64b0 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java @@ -138,10 +138,7 @@ public static Map rowToJsonMap(Row row, List fields, Lis String key = parts[parts.length - 1]; Object col = row.getField(i); if(col != null) { - if (col instanceof String){ - col = StringUtil.string2col(String.valueOf(col), types.get(i), null); - } - + col = StringUtil.string2col(String.valueOf(col), types.get(i), null); currMap.put(key, col); } From d8e878e96ec075b3047b074e0deb83aa6e31c0bd Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 18 Mar 2019 14:08:14 +0800 Subject: [PATCH 23/96] =?UTF-8?q?=E8=BF=94=E5=9B=9E=E5=80=BC=E5=8C=BA?= =?UTF-8?q?=E5=88=86null=E5=92=8C=E7=A9=BA=E5=AD=97=E7=AC=A6=E4=B8=B2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flinkx/util/StringUtil.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java index a61b49a764..dcb59a78a9 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java @@ -67,11 +67,7 @@ public static String convertRegularExpr (String str) { } public static Object string2col(String str, String type, SimpleDateFormat customTimeFormat) { - if(str == null || str.length() == 0){ - return null; - } - - if(type == null){ + if(str == null || str.length() == 0 || type == null){ return str; } From 261e44d45fb0a7dbae26ce4308324244079a90b3 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 18 Mar 2019 14:14:41 +0800 Subject: [PATCH 24/96] =?UTF-8?q?null=E5=80=BC=E4=B9=9F=E5=86=99=E5=85=A5e?= =?UTF-8?q?s?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flinkx/es/EsUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java index ee542c64b0..3c65db2821 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java @@ -139,9 +139,9 @@ public static Map rowToJsonMap(Row row, List fields, Lis Object col = row.getField(i); if(col != null) { col = StringUtil.string2col(String.valueOf(col), types.get(i), null); - currMap.put(key, col); } + currMap.put(key, col); } } catch(Exception ex) { String msg = "EsUtil.rowToJsonMap Writing record error: when converting field[" + i + "] in Row(" + row + ")"; From 2b6a1f0f6f45b7e473eee84e035cf021a3ecba6d Mon Sep 17 00:00:00 2001 From: toutian Date: Mon, 18 Mar 2019 16:34:14 +0800 Subject: [PATCH 25/96] =?UTF-8?q?=E4=BD=BF=E7=94=A8=E5=86=85=E7=BD=AEflink?= =?UTF-8?q?=E5=AE=9E=E7=8E=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-core/pom.xml | 20 - .../flinkx/inputformat/RichInputFormat.java | 6 + .../dtstack/flinkx/metrics/DTMetricNames.java | 48 +++ .../dtstack/flinkx/metrics/InputMetric.java | 58 +++ .../flinkx/metrics/MetricRegistry.java | 73 ---- .../flinkx/metrics/MetricRegistryImpl.java | 310 -------------- .../dtstack/flinkx/metrics/OutputMetric.java | 97 +++++ .../flinkx/metrics/base/MetricGroup.java | 168 -------- .../metrics/base/reporter/MetricReporter.java | 54 --- .../metrics/groups/AbstractMetricGroup.java | 387 ------------------ .../metrics/groups/ComponentMetricGroup.java | 111 ----- .../metrics/groups/FrontMetricGroup.java | 48 --- .../metrics/groups/GenericMetricGroup.java | 57 --- .../groups/PipelineInputMetricGroup.java | 136 ------ .../groups/PipelineOutputMetricGroup.java | 140 ------- .../metrics/groups/ProxyMetricGroup.java | 112 ----- .../AbstractPrometheusReporter.java | 315 -------------- .../PrometheusPushGatewayReporter.java | 90 ---- .../PrometheusPushGatewayReporterOptions.java | 62 --- .../promethues/PrometheusReporter.java | 79 ---- .../metrics/scope/PipelineScopeFormat.java | 38 -- .../flinkx/metrics/scope/ScopeFormat.java | 264 ------------ .../flinkx/metrics/scope/ScopeFormats.java | 57 --- .../flinkx/outputformat/RichOutputFormat.java | 12 + 24 files changed, 221 insertions(+), 2521 deletions(-) create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/DTMetricNames.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistry.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistryImpl.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/OutputMetric.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/MetricGroup.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/reporter/MetricReporter.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/AbstractMetricGroup.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ComponentMetricGroup.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/FrontMetricGroup.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/GenericMetricGroup.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineInputMetricGroup.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineOutputMetricGroup.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ProxyMetricGroup.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/AbstractPrometheusReporter.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporter.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporterOptions.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusReporter.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/PipelineScopeFormat.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormat.java delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormats.java diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index 97d9ca88b6..161f3e2eed 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -16,7 +16,6 @@ 2.10 1.5.4 ${basedir}/../dev - 0.5.0 @@ -102,25 +101,6 @@ 1.2.17 - - - io.prometheus - simpleclient - ${prometheus.version} - - - - io.prometheus - simpleclient_httpserver - ${prometheus.version} - - - - io.prometheus - simpleclient_pushgateway - ${prometheus.version} - - diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java index f7ca98edda..e6cadbddc4 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java @@ -19,6 +19,7 @@ package com.dtstack.flinkx.inputformat; import com.dtstack.flinkx.constants.Metrics; +import com.dtstack.flinkx.metrics.InputMetric; import com.dtstack.flinkx.reader.ByteRateLimiter; import org.apache.commons.lang.StringUtils; import org.apache.flink.api.common.accumulators.LongCounter; @@ -50,6 +51,7 @@ public abstract class RichInputFormat extends org.apache.flink.api.common.io.Ric protected long bytes; protected ByteRateLimiter byteRateLimiter; + protected InputMetric inputMetric; protected abstract void openInternal(InputSplit inputSplit) throws IOException; @@ -59,6 +61,9 @@ public void open(InputSplit inputSplit) throws IOException { if (vars != null && vars.get(Metrics.JOB_NAME) != null) { jobName = vars.get(Metrics.JOB_NAME); } + + inputMetric = new InputMetric(getRuntimeContext()); + numReadCounter = getRuntimeContext().getLongCounter(Metrics.NUM_READS); openInternal(inputSplit); @@ -74,6 +79,7 @@ public void open(InputSplit inputSplit) throws IOException { @Override public Row nextRecord(Row row) throws IOException { numReadCounter.add(1); + inputMetric.getNumRecordsIn().inc(); if(byteRateLimiter != null) { byteRateLimiter.acquire(); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/DTMetricNames.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/DTMetricNames.java new file mode 100644 index 0000000000..6ab3179abe --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/DTMetricNames.java @@ -0,0 +1,48 @@ +/* + * 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.metrics; + +/** + * Collection of dtstack metric names. + */ +public class DTMetricNames { + + private DTMetricNames() { + } + + private static final String SUFFIX_RATE = "PerSecond"; + + public static final String IO_NUM_RECORDS_IN = "numRecordsIn"; + public static final String IO_NUM_RECORDS_OUT = "numRecordsOut"; + public static final String IO_NUM_RECORDS_IN_RATE = IO_NUM_RECORDS_IN + SUFFIX_RATE; + public static final String IO_NUM_RECORDS_OUT_RATE = IO_NUM_RECORDS_OUT + SUFFIX_RATE; + + public static final String IO_NUM_BYTES_IN = "numBytesIn"; + public static final String IO_NUM_BYTES_OUT = "numBytesOut"; + public static final String IO_NUM_BYTES_IN_RATE = IO_NUM_BYTES_IN + SUFFIX_RATE; + public static final String IO_NUM_BYTES_OUT_RATE = IO_NUM_BYTES_OUT + SUFFIX_RATE; + + public static final String NUM_ERRORS = "nErrors"; + public static final String NUM_NULL_ERRORS = "nullErrors"; + public static final String NUM_DUPLICATE_ERRORS = "duplicateErrors"; + public static final String NUM_CONVERSION_ERRORS = "conversionErrors"; + public static final String NUM_OTHER_ERRORS = "otherErrors"; + public static final String NUM_WRITES = "numWrite"; + +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java new file mode 100644 index 0000000000..17a05d6149 --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java @@ -0,0 +1,58 @@ +package com.dtstack.flinkx.metrics; + +import com.dtstack.flinkx.inputformat.RichInputFormat; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; + +/** + * company: www.dtstack.com + * author: toutian + * create: 2019/3/18 + */ +public class InputMetric { + + protected transient Counter numRecordsIn; + protected transient Meter numRecordsInRate; + + protected transient Counter numBytesIn; + protected transient Meter numInBytesRate; + + private transient RuntimeContext runtimeContext; + + public InputMetric(RuntimeContext runtimeContext) { + this.runtimeContext = runtimeContext; + + initMetric(); + } + + public void initMetric() { + + numRecordsIn = getRuntimeContext().getMetricGroup().counter(DTMetricNames.IO_NUM_RECORDS_IN); + numRecordsInRate = getRuntimeContext().getMetricGroup().meter(DTMetricNames.IO_NUM_RECORDS_IN_RATE, new MeterView(numRecordsIn, 20)); + + numBytesIn = getRuntimeContext().getMetricGroup().counter(DTMetricNames.IO_NUM_BYTES_IN); + numInBytesRate = getRuntimeContext().getMetricGroup().meter(DTMetricNames.IO_NUM_BYTES_IN_RATE, new MeterView(numBytesIn, 20)); + } + + public Counter getNumRecordsIn() { + return numRecordsIn; + } + + public Meter getNumRecordsInRate() { + return numRecordsInRate; + } + + public Counter getNumBytesIn() { + return numBytesIn; + } + + public Meter getNumInBytesRate() { + return numInBytesRate; + } + + public RuntimeContext getRuntimeContext() { + return runtimeContext; + } +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistry.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistry.java deleted file mode 100644 index ba8112445e..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistry.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.metrics; - -import com.dtstack.flinkx.metrics.groups.AbstractMetricGroup; -import com.dtstack.flinkx.metrics.scope.ScopeFormats; -import org.apache.flink.metrics.Metric; - -/** - * copy from https://github.com/apache/flink - * - * Interface for a metric registry. - */ -public interface MetricRegistry { - - /** - * Returns the global delimiter. - * - * @return global delimiter - */ - char getDelimiter(); - - /** - * Returns the configured delimiter for the reporter with the given index. - * - * @param index index of the reporter whose delimiter should be used - * @return configured reporter delimiter, or global delimiter if index is invalid - */ - char getDelimiter(int index); - - /** - * Returns the number of registered reporters. - */ - int getNumberReporters(); - - /** - * Registers a new {@link Metric} with this registry. - * - * @param metric the metric that was added - * @param metricName the name of the metric - * @param group the group that contains the metric - */ - void register(Metric metric, String metricName, AbstractMetricGroup group); - - /** - * Un-registers the given {@link Metric} with this registry. - * - * @param metric the metric that should be removed - * @param metricName the name of the metric - * @param group the group that contains the metric - */ - void unregister(Metric metric, String metricName, AbstractMetricGroup group); - - - ScopeFormats getScopeFormats(); -} - diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistryImpl.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistryImpl.java deleted file mode 100644 index c804dac7db..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/MetricRegistryImpl.java +++ /dev/null @@ -1,310 +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.metrics; - -import com.dtstack.flinkx.metrics.base.reporter.MetricReporter; -import com.dtstack.flinkx.metrics.groups.AbstractMetricGroup; -import com.dtstack.flinkx.metrics.groups.FrontMetricGroup; -import com.dtstack.flinkx.metrics.scope.ScopeFormats; -import org.apache.flink.metrics.Metric; -import org.apache.flink.metrics.MetricConfig; -import org.apache.flink.metrics.View; -import org.apache.flink.metrics.reporter.Scheduled; -import org.apache.commons.collections.MapUtils; -import org.apache.flink.runtime.metrics.ViewUpdater; -import org.apache.flink.runtime.util.ExecutorThreadFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.TimerTask; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - -/** - * copy from https://github.com/apache/flink - * - * A MetricRegistry keeps track of all registered {@link Metric Metrics}. It serves as the - * connection between {@link com.dtstack.flinkx.metrics.base.MetricGroup MetricGroups} and {@link MetricReporter MetricReporters}. - */ -public class MetricRegistryImpl implements MetricRegistry { - static final Logger LOG = LoggerFactory.getLogger(MetricRegistryImpl.class); - - private final Object lock = new Object(); - - private List reporters; - private ScheduledExecutorService executor; - - private final ScopeFormats scopeFormats = ScopeFormats.fromDefault(); - private final char globalDelimiter = '.'; - private final List delimiters = new ArrayList<>(); - - private ViewUpdater viewUpdater; - - /** - * Creates a new MetricRegistry and starts the configured reporter. - */ - public MetricRegistryImpl(List reporterPlugins) { - - // instantiate any custom configured reporters - this.reporters = new ArrayList<>(); - this.executor = Executors.newSingleThreadScheduledExecutor(new ExecutorThreadFactory("flinkx-MetricRegistry")); - - - if (reporterPlugins.isEmpty()) { - // no reporters defined - // by default, don't report anything - LOG.info("No metrics reporter configured, no metrics will be exposed/reported."); - } else { - // we have some reporters so - for (Map reporterPlugin: reporterPlugins) { - Iterator>> reportIt = reporterPlugin.entrySet().iterator(); - while (reportIt.hasNext()) { - Map.Entry> reportEntry = reportIt.next(); - String namedReporter = reportEntry.getKey(); - Map reporterConfig = reportEntry.getValue(); - final String className = MapUtils.getString(reporterConfig,"class", null); - if (className == null) { - LOG.error("No reporter class set for reporter " + namedReporter + ". Metrics might not be exposed/reported."); - continue; - } - - try { - String configuredPeriod = MapUtils.getString(reporterConfig,"interval", null); - TimeUnit timeunit = TimeUnit.SECONDS; - long period = 10; - - if (configuredPeriod != null) { - try { - String[] interval = configuredPeriod.split(" "); - period = Long.parseLong(interval[0]); - timeunit = TimeUnit.valueOf(interval[1]); - } catch (Exception e) { - LOG.error("Cannot parse report interval from config: " + configuredPeriod + - " - please use values like '10 SECONDS' or '500 MILLISECONDS'. " + - "Using default reporting interval."); - } - } - - Class reporterClass = Class.forName(className); - MetricReporter reporterInstance = (MetricReporter) reporterClass.newInstance(); - - MetricConfig metricConfig = new MetricConfig(); - addAllToProperties(reporterConfig, metricConfig); - LOG.info("Configuring {} with {}.", reporterClass.getSimpleName(), metricConfig); - reporterInstance.open(metricConfig); - - if (reporterInstance instanceof Scheduled) { - LOG.info("Periodically reporting metrics in intervals of {} {} for reporter {} of type {}.", period, timeunit.name(), namedReporter, className); - - executor.scheduleWithFixedDelay( - new MetricRegistryImpl.ReporterTask((Scheduled) reporterInstance), period, period, timeunit); - } else { - LOG.info("Reporting metrics for reporter {} of type {}.", namedReporter, className); - } - reporters.add(reporterInstance); - - String delimiterForReporter = MapUtils.getString(reporterConfig,"scope.delimiter", String.valueOf(globalDelimiter)); - if (delimiterForReporter.length() != 1) { - LOG.warn("Failed to parse delimiter '{}' for reporter '{}', using global delimiter '{}'.", delimiterForReporter, namedReporter, globalDelimiter); - delimiterForReporter = String.valueOf(globalDelimiter); - } - this.delimiters.add(delimiterForReporter.charAt(0)); - } catch (Throwable t) { - LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", namedReporter, t); - } - } - } - } - } - - private void addAllToProperties(Map confData, Properties props) { - for (Map.Entry entry : confData.entrySet()) { - props.put(entry.getKey(), entry.getValue()); - } - } - - - @Override - public char getDelimiter() { - return this.globalDelimiter; - } - - @Override - public char getDelimiter(int reporterIndex) { - try { - return delimiters.get(reporterIndex); - } catch (IndexOutOfBoundsException e) { - LOG.warn("Delimiter for reporter index {} not found, returning global delimiter.", reporterIndex); - return this.globalDelimiter; - } - } - - @Override - public int getNumberReporters() { - return reporters.size(); - } - - public List getReporters() { - return reporters; - } - - /** - * Returns whether this registry has been shutdown. - * - * @return true, if this registry was shutdown, otherwise false - */ - public boolean isShutdown() { - synchronized (lock) { - return reporters == null && executor.isShutdown(); - } - } - - /** - * Shuts down this registry and the associated {@link MetricReporter}. - */ - public void shutdown() { - synchronized (lock) { - if (reporters != null) { - for (MetricReporter reporter : reporters) { - try { - reporter.close(); - } catch (Throwable t) { - LOG.warn("Metrics reporter did not shut down cleanly", t); - } - } - reporters = null; - } - shutdownExecutor(); - } - } - - private void shutdownExecutor() { - if (executor != null) { - executor.shutdown(); - - try { - if (!executor.awaitTermination(1L, TimeUnit.SECONDS)) { - executor.shutdownNow(); - } - } catch (InterruptedException e) { - executor.shutdownNow(); - } - } - } - - // ------------------------------------------------------------------------ - // Metrics (de)registration - // ------------------------------------------------------------------------ - - @Override - public void register(Metric metric, String metricName, AbstractMetricGroup group) { - synchronized (lock) { - if (isShutdown()) { - LOG.warn("Cannot register metric, because the MetricRegistry has already been shut down."); - } else { - if (reporters != null) { - for (int i = 0; i < reporters.size(); i++) { - MetricReporter reporter = reporters.get(i); - try { - if (reporter != null) { - FrontMetricGroup front = new FrontMetricGroup>(i, group); - reporter.notifyOfAddedMetric(metric, metricName, front); - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - } - } - try { - if (metric instanceof View) { - if (viewUpdater == null) { - viewUpdater = new ViewUpdater(executor); - } - viewUpdater.notifyOfAddedView((View) metric); - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - } - } - } - - @Override - public void unregister(Metric metric, String metricName, AbstractMetricGroup group) { - synchronized (lock) { - if (isShutdown()) { - LOG.warn("Cannot unregister metric, because the MetricRegistry has already been shut down."); - } else { - if (reporters != null) { - for (int i = 0; i < reporters.size(); i++) { - try { - MetricReporter reporter = reporters.get(i); - if (reporter != null) { - FrontMetricGroup front = new FrontMetricGroup>(i, group); - reporter.notifyOfRemovedMetric(metric, metricName, front); - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - } - } - } - } - } - - @Override - public ScopeFormats getScopeFormats() { - return scopeFormats; - } - - - /** - * This task is explicitly a static class, so that it does not hold any references to the enclosing - * MetricsRegistry instance. - * - *

This is a subtle difference, but very important: With this static class, the enclosing class instance - * may become garbage-collectible, whereas with an anonymous inner class, the timer thread - * (which is a GC root) will hold a reference via the timer task and its enclosing instance pointer. - * Making the MetricsRegistry garbage collectible makes the java.util.Timer garbage collectible, - * which acts as a fail-safe to stop the timer thread and prevents resource leaks. - */ - private static final class ReporterTask extends TimerTask { - - private final Scheduled reporter; - - private ReporterTask(Scheduled reporter) { - this.reporter = reporter; - } - - @Override - public void run() { - try { - reporter.report(); - } catch (Throwable t) { - LOG.warn("Error while reporting metrics", t); - } - } - } -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/OutputMetric.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/OutputMetric.java new file mode 100644 index 0000000000..bfc8e0bbf9 --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/OutputMetric.java @@ -0,0 +1,97 @@ +package com.dtstack.flinkx.metrics; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; + +/** + * company: www.dtstack.com + * author: toutian + * create: 2019/3/18 + */ +public class OutputMetric { + + protected transient Counter numErrors; + protected transient Counter numNullErrors; + protected transient Counter numDuplicateErrors; + protected transient Counter numConversionErrors; + protected transient Counter numOtherErrors; + protected transient Counter numWrite; + + protected transient Counter numRecordsOut; + protected transient Meter numRecordsOutRate; + + protected transient Counter numBytesOut; + protected transient Meter numBytesOutRate; + + private transient RuntimeContext runtimeContext; + + public OutputMetric(RuntimeContext runtimeContext) { + this.runtimeContext = runtimeContext; + + initMetric(); + } + + public void initMetric() { + + numErrors = getRuntimeContext().getMetricGroup().counter(DTMetricNames.NUM_ERRORS); + numNullErrors = getRuntimeContext().getMetricGroup().counter(DTMetricNames.NUM_NULL_ERRORS); + numDuplicateErrors = getRuntimeContext().getMetricGroup().counter(DTMetricNames.NUM_DUPLICATE_ERRORS); + numConversionErrors = getRuntimeContext().getMetricGroup().counter(DTMetricNames.NUM_CONVERSION_ERRORS); + numOtherErrors = getRuntimeContext().getMetricGroup().counter(DTMetricNames.NUM_OTHER_ERRORS); + numWrite = getRuntimeContext().getMetricGroup().counter(DTMetricNames.NUM_WRITES); + + numRecordsOut = getRuntimeContext().getMetricGroup().counter(DTMetricNames.IO_NUM_RECORDS_OUT); + numRecordsOutRate = getRuntimeContext().getMetricGroup().meter(DTMetricNames.IO_NUM_RECORDS_OUT_RATE, new MeterView(numRecordsOut, 20)); + + numBytesOut = getRuntimeContext().getMetricGroup().counter(DTMetricNames.IO_NUM_BYTES_OUT); + numBytesOutRate = getRuntimeContext().getMetricGroup().meter(DTMetricNames.IO_NUM_BYTES_OUT_RATE, new MeterView(numBytesOut, 20)); + } + + public Counter getNumErrors() { + return numErrors; + } + + public Counter getNumNullErrors() { + return numNullErrors; + } + + public Counter getNumDuplicateErrors() { + return numDuplicateErrors; + } + + public Counter getNumConversionErrors() { + return numConversionErrors; + } + + public Counter getNumOtherErrors() { + return numOtherErrors; + } + + public Counter getNumWrite() { + return numWrite; + } + + public Counter getNumRecordsOut() { + return numRecordsOut; + } + + public Meter getNumRecordsOutRate() { + return numRecordsOutRate; + } + + public Counter getNumBytesOut() { + return numBytesOut; + } + + public Meter getNumBytesOutRate() { + return numBytesOutRate; + } + + public RuntimeContext getRuntimeContext() { + return runtimeContext; + } + + +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/MetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/MetricGroup.java deleted file mode 100644 index 23abb95925..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/MetricGroup.java +++ /dev/null @@ -1,168 +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.metrics.base; - - -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.metrics.Meter; - -import java.util.Map; - -/** - * - * copy from https://github.com/apache/flink - * - * A MetricGroup is a named container for {@link org.apache.flink.metrics.Metric Metrics} and further metric subgroups. - * - *

Instances of this class can be used to register new metrics with Flinkx and to create a nested - * hierarchy based on the group names. - * - *

A MetricGroup is uniquely identified by it's place in the hierarchy and name. - */ -public interface MetricGroup { - - // ------------------------------------------------------------------------ - // Metrics - // ------------------------------------------------------------------------ - - /** - * Creates and registers a new {@link Counter} with Flinkx. - * - * @param name name of the counter - * @return the created counter - */ - Counter counter(int name); - - /** - * Creates and registers a new {@link Counter} with Flinkx. - * - * @param name name of the counter - * @return the created counter - */ - Counter counter(String name); - - /** - * Registers a {@link Counter} with Flinkx. - * - * @param name name of the counter - * @param counter counter to register - * @param counter type - * @return the given counter - */ - C counter(int name, C counter); - - /** - * Registers a {@link Counter} with Flinkx. - * - * @param name name of the counter - * @param counter counter to register - * @param counter type - * @return the given counter - */ - C counter(String name, C counter); - - /** - * Registers a new {@link Gauge} with Flinkx. - * - * @param name name of the gauge - * @param gauge gauge to register - * @param return type of the gauge - * @return the given gauge - */ - > G gauge(int name, G gauge); - - /** - * Registers a new {@link Gauge} with Flinkx. - * - * @param name name of the gauge - * @param gauge gauge to register - * @param return type of the gauge - * @return the given gauge - */ - > G gauge(String name, G gauge); - - - /** - * Registers a new {@link Meter} with Flinkx. - * - * @param name name of the meter - * @param meter meter to register - * @param meter type - * @return the registered meter - */ - M meter(String name, M meter); - - /** - * Registers a new {@link Meter} with Flinkx. - * - * @param name name of the meter - * @param meter meter to register - * @param meter type - * @return the registered meter - */ - M meter(int name, M meter); - - - // ------------------------------------------------------------------------ - // Groups - // ------------------------------------------------------------------------ - - /** - * Creates a new MetricGroup and adds it to this groups sub-groups. - * - * @param name name of the group - * @return the created group - */ - MetricGroup addGroup(int name); - - /** - * Creates a new MetricGroup and adds it to this groups sub-groups. - * - * @param name name of the group - * @return the created group - */ - MetricGroup addGroup(String name); - - /** - * Gets the scope as an array of the scope components, for example - * {@code ["host-7", "taskmanager-2", "window_word_count", "my-mapper"]}. - * - * @see #getMetricIdentifier(String) - */ - String[] getScopeComponents(); - - - /** - * Returns a map of all variables and their associated value, for example - * {@code {""="host-7", ""="taskmanager-2"}}. - * - * @return map of all variables and their associated value - */ - Map getAllVariables(); - - /** - * Returns the fully qualified metric name, for example - * {@code "host-7.taskmanager-2.window_word_count.my-mapper.metricName"}. - * - * @param metricName metric name - * @return fully qualified metric name - */ - String getMetricIdentifier(String metricName); -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/reporter/MetricReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/reporter/MetricReporter.java deleted file mode 100644 index c37a794df6..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/base/reporter/MetricReporter.java +++ /dev/null @@ -1,54 +0,0 @@ -package com.dtstack.flinkx.metrics.base.reporter; - - -import com.dtstack.flinkx.metrics.base.MetricGroup; -import org.apache.flink.metrics.Metric; -import org.apache.flink.metrics.MetricConfig; - -/** - * copy from https://github.com/apache/flink - * - * Reporters are used to export {@link Metric Metrics} to an external backend. - * - *

Reporters are instantiated via reflection and must be public, non-abstract, and have a - * public no-argument constructor. - */ -public interface MetricReporter { - - /** - * Configures this reporter. Since reporters are instantiated generically and hence parameter-less, - * this method is the place where the reporters set their basic fields based on configuration values. - * - *

This method is always called first on a newly instantiated reporter. - * - * @param config A properties object that contains all parameters set for this reporter. - */ - void open(MetricConfig config); - - /** - * Closes this reporter. Should be used to close channels, streams and release resources. - */ - void close(); - - // ------------------------------------------------------------------------ - // adding / removing metrics - // ------------------------------------------------------------------------ - - /** - * Called when a new {@link Metric} was added. - * - * @param metric the metric that was added - * @param metricName the name of the metric - * @param group the group that contains the metric - */ - void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup group); - - /** - * Called when a {@link Metric} was should be removed. - * - * @param metric the metric that should be removed - * @param metricName the name of the metric - * @param group the group that contains the metric - */ - void notifyOfRemovedMetric(Metric metric, String metricName, MetricGroup group); -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/AbstractMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/AbstractMetricGroup.java deleted file mode 100644 index 6645952681..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/AbstractMetricGroup.java +++ /dev/null @@ -1,387 +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.metrics.groups; - -import com.dtstack.flinkx.metrics.MetricRegistry; -import com.dtstack.flinkx.metrics.base.MetricGroup; -import com.dtstack.flinkx.metrics.scope.ScopeFormat; -import org.apache.flink.metrics.CharacterFilter; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.Metric; -import org.apache.flink.metrics.SimpleCounter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * copy from https://github.com/apache/flink - * - * Abstract {@link MetricGroup} that contains key functionality for adding metrics and groups. - * - *

IMPORTANT IMPLEMENTATION NOTE - * - *

This class uses locks for adding and removing metrics objects. This is done to - * prevent resource leaks in the presence of concurrently closing a group and adding - * metrics and subgroups. - * Since closing groups recursively closes the subgroups, the lock acquisition order must - * be strictly from parent group to subgroup. If at any point, a subgroup holds its group - * lock and calls a parent method that also acquires the lock, it will create a deadlock - * condition. - * - *

An AbstractMetricGroup can be {@link #close() closed}. Upon closing, the group de-register all metrics - * from any metrics reporter and any internal maps. Note that even closed metrics groups - * return Counters, Gauges, etc to the code, to prevent exceptions in the monitored code. - * These metrics simply do not get reported any more, when created on a closed group. - * - * @param The type of the parent MetricGroup - */ -public abstract class AbstractMetricGroup> implements MetricGroup { - - protected static final Logger LOG = LoggerFactory.getLogger(MetricGroup.class); - - // ------------------------------------------------------------------------ - - /** The parent group containing this group. */ - protected final A parent; - - /** The map containing all variables and their associated values, lazily computed. */ - protected volatile Map variables; - - /** The registry that this metrics group belongs to. */ - protected final MetricRegistry registry; - - /** All metrics that are directly contained in this group. */ - private final Map metrics = new HashMap<>(); - - /** All metric subgroups of this group. */ - private final Map groups = new HashMap<>(); - - /** The metrics scope represented by this group. - * For example ["host-7", "taskmanager-2", "window_word_count", "my-mapper" ]. */ - private final String[] scopeComponents; - - /** Array containing the metrics scope represented by this group for each reporter, as a concatenated string, lazily computed. - * For example: "host-7.taskmanager-2.window_word_count.my-mapper" */ - private final String[] scopeStrings; - - /** The logical metrics scope represented by this group, as a concatenated string, lazily computed. - * For example: "taskmanager.job.task" */ - private String logicalScopeString; - - /** Flag indicating whether this group has been closed. */ - private volatile boolean closed; - - // ------------------------------------------------------------------------ - - public AbstractMetricGroup(MetricRegistry registry, String[] scope, A parent) { - this.registry = checkNotNull(registry); - this.scopeComponents = checkNotNull(scope); - this.parent = parent; - this.scopeStrings = new String[registry.getNumberReporters()]; - } - - public Map getAllVariables() { - if (variables == null) { // avoid synchronization for common case - synchronized (this) { - if (variables == null) { - if (parent != null) { - variables = parent.getAllVariables(); - } else { // this case should only be true for mock groups - variables = new HashMap<>(); - } - } - } - } - return variables; - } - - /** - * Returns the logical scope of this group, for example - * {@code "taskmanager.job.task"}. - * - * @param filter character filter which is applied to the scope components - * @return logical scope - */ - public String getLogicalScope(CharacterFilter filter) { - return getLogicalScope(filter, registry.getDelimiter()); - } - - /** - * Returns the logical scope of this group, for example - * {@code "taskmanager.job.task"}. - * - * @param filter character filter which is applied to the scope components - * @return logical scope - */ - public String getLogicalScope(CharacterFilter filter, char delimiter) { - if (logicalScopeString == null) { - if (parent == null) { - logicalScopeString = getGroupName(filter); - } else { - logicalScopeString = parent.getLogicalScope(filter, delimiter) + delimiter + getGroupName(filter); - } - } - return logicalScopeString; - } - - /** - * Returns the name for this group, meaning what kind of entity it represents, for example "inputs". - * - * @return logical name for this group - */ - protected abstract String getGroupName(CharacterFilter filter); - - /** - * Gets the scope as an array of the scope components, for example - * {@code ["host-7", "taskmanager-2", "window_word_count", "my-mapper"]}. - * - * @see #getMetricIdentifier(String) - */ - public String[] getScopeComponents() { - return scopeComponents; - } - - /** - * Returns the fully qualified metric name, for example - * {@code "host-7.taskmanager-2.window_word_count.my-mapper.metricName"}. - * - * @param metricName metric name - * @return fully qualified metric name - */ - public String getMetricIdentifier(String metricName) { - return getMetricIdentifier(metricName, null); - } - - - /** - * Returns the fully qualified metric name, for example - * {@code "host-7.taskmanager-2.window_word_count.my-mapper.metricName"}. - * - * @param metricName metric name - * @param filter character filter which is applied to the scope components if not null. - * @return fully qualified metric name - */ - public String getMetricIdentifier(String metricName, CharacterFilter filter) { - return getMetricIdentifier(metricName, filter, -1); - } - - /** - * Returns the fully qualified metric name using the configured delimiter for the reporter with the given index, for example - * {@code "host-7.taskmanager-2.window_word_count.my-mapper.metricName"}. - * - * @param metricName metric name - * @param filter character filter which is applied to the scope components if not null. - * @param reporterIndex index of the reporter whose delimiter should be used - * @return fully qualified metric name - */ - public String getMetricIdentifier(String metricName, CharacterFilter filter, int reporterIndex) { - if (scopeStrings.length == 0 || (reporterIndex < 0 || reporterIndex >= scopeStrings.length)) { - char delimiter = registry.getDelimiter(); - String newScopeString; - if (filter != null) { - newScopeString = ScopeFormat.concat(filter, delimiter, scopeComponents); - metricName = filter.filterCharacters(metricName); - } else { - newScopeString = ScopeFormat.concat(delimiter, scopeComponents); - } - return newScopeString + delimiter + metricName; - } else { - char delimiter = registry.getDelimiter(reporterIndex); - if (scopeStrings[reporterIndex] == null) { - if (filter != null) { - scopeStrings[reporterIndex] = ScopeFormat.concat(filter, delimiter, scopeComponents); - } else { - scopeStrings[reporterIndex] = ScopeFormat.concat(delimiter, scopeComponents); - } - } - if (filter != null) { - metricName = filter.filterCharacters(metricName); - } - return scopeStrings[reporterIndex] + delimiter + metricName; - } - } - - // ------------------------------------------------------------------------ - // Closing - // ------------------------------------------------------------------------ - - public void close() { - synchronized (this) { - if (!closed) { - closed = true; - - // close all subgroups - for (AbstractMetricGroup group : groups.values()) { - group.close(); - } - groups.clear(); - - // un-register all directly contained metrics - for (Map.Entry metric : metrics.entrySet()) { - registry.unregister(metric.getValue(), metric.getKey(), this); - } - metrics.clear(); - } - } - } - - public final boolean isClosed() { - return closed; - } - - // ----------------------------------------------------------------------------------------------------------------- - // Metrics - // ----------------------------------------------------------------------------------------------------------------- - - @Override - public Counter counter(int name) { - return counter(String.valueOf(name)); - } - - @Override - public Counter counter(String name) { - return counter(name, new SimpleCounter()); - } - - @Override - public C counter(int name, C counter) { - return counter(String.valueOf(name), counter); - } - - @Override - public C counter(String name, C counter) { - addMetric(name, counter); - return counter; - } - - @Override - public > G gauge(int name, G gauge) { - return gauge(String.valueOf(name), gauge); - } - - @Override - public > G gauge(String name, G gauge) { - addMetric(name, gauge); - return gauge; - } - - @Override - public M meter(int name, M meter) { - return meter(String.valueOf(name), meter); - } - - @Override - public M meter(String name, M meter) { - addMetric(name, meter); - return meter; - } - - /** - * Adds the given metric to the group and registers it at the registry, if the group - * is not yet closed, and if no metric with the same name has been registered before. - * - * @param name the name to register the metric under - * @param metric the metric to register - */ - protected void addMetric(String name, Metric metric) { - if (metric == null) { - LOG.warn("Ignoring attempted registration of a metric due to being null for name {}.", name); - return; - } - // add the metric only if the group is still open - synchronized (this) { - if (!closed) { - // immediately put without a 'contains' check to optimize the common case (no collision) - // collisions are resolved later - Metric prior = metrics.put(name, metric); - - // check for collisions with other metric names - if (prior == null) { - // no other metric with this name yet - - if (groups.containsKey(name)) { - // we warn here, rather than failing, because metrics are tools that should not fail the - // program when used incorrectly - LOG.warn("Name collision: Adding a metric with the same name as a metric subgroup: '" + - name + "'. Metric might not get properly reported. " + Arrays.toString(scopeComponents)); - } - - registry.register(metric, name, this); - } - else { - // we had a collision. put back the original value - metrics.put(name, prior); - - // we warn here, rather than failing, because metrics are tools that should not fail the - // program when used incorrectly - LOG.warn("Name collision: Group already contains a Metric with the name '" + - name + "'. Metric will not be reported." + Arrays.toString(scopeComponents)); - } - } - } - } - - // ------------------------------------------------------------------------ - // Groups - // ------------------------------------------------------------------------ - - @Override - public MetricGroup addGroup(int name) { - return addGroup(String.valueOf(name)); - } - - @Override - public MetricGroup addGroup(String name) { - synchronized (this) { - if (!closed) { - // adding a group with the same name as a metric creates problems in many reporters/dashboards - // we warn here, rather than failing, because metrics are tools that should not fail the - // program when used incorrectly - if (metrics.containsKey(name)) { - LOG.warn("Name collision: Adding a metric subgroup with the same name as an existing metric: '" + - name + "'. Metric might not get properly reported. " + Arrays.toString(scopeComponents)); - } - - AbstractMetricGroup newGroup = new GenericMetricGroup(registry, this, name); - AbstractMetricGroup prior = groups.put(name, newGroup); - if (prior == null) { - // no prior group with that name - return newGroup; - } else { - // had a prior group with that name, add the prior group back - groups.put(name, prior); - return prior; - } - } - else { - // return a non-registered group that is immediately closed already - GenericMetricGroup closedGroup = new GenericMetricGroup(registry, this, name); - closedGroup.close(); - return closedGroup; - } - } - } -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ComponentMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ComponentMetricGroup.java deleted file mode 100644 index e42dae093f..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ComponentMetricGroup.java +++ /dev/null @@ -1,111 +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.metrics.groups; - - -import com.dtstack.flinkx.metrics.MetricRegistry; - -import java.util.HashMap; -import java.util.Map; - -/** - * - * copy from https://github.com/apache/flink - * - * Abstract {@link com.dtstack.flinkx.metrics.base.MetricGroup} for system components (e.g., - * TaskManager, Job, Task, Operator). - * - *

Usually, the scope of metrics is simply the hierarchy of the containing groups. For example - * the Metric {@code "MyMetric"} in group {@code "B"} nested in group {@code "A"} would have a - * fully scoped name of {@code "A.B.MyMetric"}, with {@code "A.B"} being the Metric's scope. - * - *

Component groups, however, have configurable scopes. This allow users to include or exclude - * certain identifiers from the scope. The scope for metrics belonging to the "Task" - * group could for example include the task attempt number (more fine grained identification), or - * exclude it (for continuity of the namespace across failure and recovery). - * - * @param

The type of the parent MetricGroup. - */ -public abstract class ComponentMetricGroup

> extends AbstractMetricGroup

{ - - /** - * Creates a new ComponentMetricGroup. - * - * @param registry registry to register new metrics with - * @param scope the scope of the group - */ - public ComponentMetricGroup(MetricRegistry registry, String[] scope, P parent) { - super(registry, scope, parent); - } - - @Override - public Map getAllVariables() { - if (variables == null) { // avoid synchronization for common case - synchronized (this) { - if (variables == null) { - Map tmpVariables = new HashMap<>(); - putVariables(tmpVariables); - if (parent != null) { // not true for Job-/TaskManagerMetricGroup - tmpVariables.putAll(parent.getAllVariables()); - } - variables = tmpVariables; - } - } - } - return variables; - } - - /** - * Enters all variables specific to this ComponentMetricGroup and their associated values into the map. - * - * @param variables map to enter variables and their values into - */ - protected abstract void putVariables(Map variables); - - /** - * Closes the component group by removing and closing all metrics and subgroups - * (inherited from {@link AbstractMetricGroup}), plus closing and removing all dedicated - * component subgroups. - */ - @Override - public void close() { - synchronized (this) { - if (!isClosed()) { - // remove all metrics and generic subgroups - super.close(); - - // remove and close all subcomponent metrics - for (ComponentMetricGroup group : subComponents()) { - group.close(); - } - } - } - } - - // ------------------------------------------------------------------------ - // Component Metric Group Specifics - // ------------------------------------------------------------------------ - - /** - * Gets all component metric groups that are contained in this component metric group. - * - * @return All component metric groups that are contained in this component metric group. - */ - protected abstract Iterable subComponents(); -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/FrontMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/FrontMetricGroup.java deleted file mode 100644 index 710c4e7818..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/FrontMetricGroup.java +++ /dev/null @@ -1,48 +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.metrics.groups; - - -import org.apache.flink.metrics.CharacterFilter; - -/** - * copy from https://github.com/apache/flink - */ -public class FrontMetricGroup

> extends ProxyMetricGroup

{ - - protected int reporterIndex; - - public FrontMetricGroup(int reporterIndex, P reference) { - super(reference); - this.reporterIndex = reporterIndex; - } - - @Override - public String getMetricIdentifier(String metricName) { - return parentMetricGroup.getMetricIdentifier(metricName); - } - - public String getLogicalScope(CharacterFilter filter) { - return parentMetricGroup.getLogicalScope(filter); - } - - public String getLogicalScope(CharacterFilter filter, char delimiter) { - return parentMetricGroup.getLogicalScope(filter, delimiter); - } -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/GenericMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/GenericMetricGroup.java deleted file mode 100644 index 5cffd60c47..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/GenericMetricGroup.java +++ /dev/null @@ -1,57 +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.metrics.groups; - - -import com.dtstack.flinkx.metrics.MetricRegistry; -import org.apache.flink.metrics.CharacterFilter; - -/** - * copy from https://github.com/apache/flink - */ -public class GenericMetricGroup extends AbstractMetricGroup> { - - /** The name of this group. */ - private String name; - - public GenericMetricGroup(MetricRegistry registry, AbstractMetricGroup parent, String name) { - super(registry, makeScopeComponents(parent, name), parent); - this.name = name; - } - - // ------------------------------------------------------------------------ - - private static String[] makeScopeComponents(AbstractMetricGroup parent, String name) { - if (parent != null) { - String[] parentComponents = parent.getScopeComponents(); - if (parentComponents != null && parentComponents.length > 0) { - String[] parts = new String[parentComponents.length + 1]; - System.arraycopy(parentComponents, 0, parts, 0, parentComponents.length); - parts[parts.length - 1] = name; - return parts; - } - } - return new String[] { name }; - } - - @Override - protected String getGroupName(CharacterFilter filter) { - return filter.filterCharacters(name); - } -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineInputMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineInputMetricGroup.java deleted file mode 100644 index b269811e6d..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineInputMetricGroup.java +++ /dev/null @@ -1,136 +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.metrics.groups; - - -import com.dtstack.flinkx.constants.Metrics; -import com.dtstack.flinkx.metrics.MetricRegistry; -import org.apache.flink.metrics.CharacterFilter; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; -import org.apache.flink.metrics.SimpleCounter; -import org.apache.flink.runtime.metrics.MetricNames; -import com.dtstack.flinkx.metrics.scope.ScopeFormat; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -/** - * Metric group that contains shareable pre-defined IO-related metrics. The metrics registration is - * forwarded to the parent task metric group. - */ -public class PipelineInputMetricGroup> extends ComponentMetricGroup { - - private final Counter numBytesInLocal; - private final Meter numBytesInRateLocal; - - private final SumCounter numRead; - - - private final String hostname; - private final String pluginType; - private final String pluginName; - private final String jobName; - - public PipelineInputMetricGroup(MetricRegistry registry, - String hostname, - String pluginType, - String pluginName, - String jobName) { - super(registry, registry.getScopeFormats().getPipelineScopeFormat().formatScope(hostname, pluginType, pluginName, jobName), null); - - this.hostname = hostname; - this.pluginType = pluginType; - this.pluginName = pluginName; - this.jobName = jobName; - - this.numBytesInLocal = counter(MetricNames.IO_NUM_BYTES_IN_LOCAL); - this.numBytesInRateLocal = meter(MetricNames.IO_NUM_BYTES_IN_LOCAL_RATE, new MeterView(numBytesInLocal, 60)); - - this.numRead = (SumCounter) counter(Metrics.NUM_READS, new SumCounter()); - } - - // ============================================================================================ - // Getters - // ============================================================================================ - - - public Counter getNumBytesInLocal() { - return numBytesInLocal; - } - - public Meter getNumBytesInRateLocal() { - return numBytesInRateLocal; - } - - public SumCounter getNumRead() { - return numRead; - } - - @Override - protected void putVariables(Map variables) { - variables.put(ScopeFormat.SCOPE_HOST, hostname); - variables.put(ScopeFormat.SCOPE_PLUGINE_TYPE, pluginType); - variables.put(ScopeFormat.SCOPE_PLUGINE_NAME, pluginName); - variables.put(ScopeFormat.SCOPE_JOB_NAME, jobName); - } - - @Override - protected Iterable subComponents() { - return null; - } - - @Override - protected String getGroupName(CharacterFilter filter) { - return "pipeline"; - } - - // ============================================================================================ - // Metric Reuse - // ============================================================================================ - public void reuseRecordsInputCounter(Counter numRecordsInCounter) { - this.numRead.addCounter(numRecordsInCounter); - } - - /** - * A {@link SimpleCounter} that can contain other {@link Counter}s. A call to {@link SumCounter#getCount()} returns - * the sum of this counters and all contained counters. - */ - private static class SumCounter extends SimpleCounter { - private final List internalCounters = new ArrayList<>(); - - SumCounter() { - } - - public void addCounter(Counter toAdd) { - internalCounters.add(toAdd); - } - - @Override - public long getCount() { - long sum = super.getCount(); - for (Counter counter : internalCounters) { - sum += counter.getCount(); - } - return sum; - } - } -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineOutputMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineOutputMetricGroup.java deleted file mode 100644 index 6c609971b3..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/PipelineOutputMetricGroup.java +++ /dev/null @@ -1,140 +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.metrics.groups; - - -import com.dtstack.flinkx.constants.Metrics; -import com.dtstack.flinkx.metrics.MetricRegistry; -import org.apache.flink.runtime.metrics.MetricNames; -import org.apache.flink.metrics.CharacterFilter; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; -import org.apache.flink.metrics.SimpleCounter; -import com.dtstack.flinkx.metrics.scope.ScopeFormat; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -/** - * Metric group that contains shareable pre-defined IO-related metrics. The metrics registration is - * forwarded to the parent task metric group. - */ -public class PipelineOutputMetricGroup> extends ComponentMetricGroup { - - private final Counter numBytesOut; - private final SumCounter numRecordsOut; - - private final SumCounter numErrors; - private final SumCounter numNullErrors; - private final SumCounter numDuplicateErrors; - private final SumCounter numConversionErrors; - private final SumCounter numOtherErrors; - private final SumCounter numWrite; - - private final Meter numBytesOutRate; - private final Meter numRecordsOutRate; - - private final String hostname; - private final String pluginType; - private final String pluginName; - private final String jobName; - - public PipelineOutputMetricGroup(MetricRegistry registry, - String hostname, - String pluginType, - String pluginName, - String jobName) { - super(registry, registry.getScopeFormats().getPipelineScopeFormat().formatScope(hostname, pluginType, pluginName, jobName), null); - - this.hostname = hostname; - this.pluginType = pluginType; - this.pluginName = pluginName; - this.jobName = jobName; - - - this.numBytesOut = counter(MetricNames.IO_NUM_BYTES_OUT); - this.numBytesOutRate = meter(MetricNames.IO_NUM_BYTES_OUT_RATE, new MeterView(numBytesOut, 60)); - this.numRecordsOut = (SumCounter) counter(MetricNames.IO_NUM_RECORDS_OUT, new SumCounter()); - this.numRecordsOutRate = meter(MetricNames.IO_NUM_RECORDS_OUT_RATE, new MeterView(numRecordsOut, 60)); - - this.numErrors = (SumCounter) counter(Metrics.NUM_ERRORS, new SumCounter()); - this.numNullErrors = (SumCounter) counter(Metrics.NUM_NULL_ERRORS, new SumCounter()); - this.numDuplicateErrors = (SumCounter) counter(Metrics.NUM_DUPLICATE_ERRORS, new SumCounter()); - this.numConversionErrors = (SumCounter) counter(Metrics.NUM_CONVERSION_ERRORS, new SumCounter()); - this.numOtherErrors = (SumCounter) counter(Metrics.NUM_OTHER_ERRORS, new SumCounter()); - this.numWrite = (SumCounter) counter(Metrics.NUM_WRITES, new SumCounter()); - } - - // ============================================================================================ - // Getters - // ============================================================================================ - - - - @Override - protected void putVariables(Map variables) { - variables.put(ScopeFormat.SCOPE_HOST, hostname); - variables.put(ScopeFormat.SCOPE_PLUGINE_TYPE, pluginType); - variables.put(ScopeFormat.SCOPE_PLUGINE_NAME, pluginName); - variables.put(ScopeFormat.SCOPE_JOB_NAME, jobName); - } - - @Override - protected Iterable subComponents() { - return null; - } - - @Override - protected String getGroupName(CharacterFilter filter) { - return "pipeline"; - } - - // ============================================================================================ - // Metric Reuse - // ============================================================================================ - public void reuseRecordsOutputCounter(Counter numRecordsOutCounter) { - this.numRecordsOut.addCounter(numRecordsOutCounter); - } - - /** - * A {@link SimpleCounter} that can contain other {@link Counter}s. A call to {@link SumCounter#getCount()} returns - * the sum of this counters and all contained counters. - */ - private static class SumCounter extends SimpleCounter { - private final List internalCounters = new ArrayList<>(); - - SumCounter() { - } - - public void addCounter(Counter toAdd) { - internalCounters.add(toAdd); - } - - @Override - public long getCount() { - long sum = super.getCount(); - for (Counter counter : internalCounters) { - sum += counter.getCount(); - } - return sum; - } - } -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ProxyMetricGroup.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ProxyMetricGroup.java deleted file mode 100644 index 17a5e59e0a..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/groups/ProxyMetricGroup.java +++ /dev/null @@ -1,112 +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.metrics.groups; - - -import com.dtstack.flinkx.metrics.base.MetricGroup; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.metrics.Meter; - -import java.util.Map; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * - * copy from https://github.com/apache/flink - * - * Metric group which forwards all registration calls to its parent metric group. - * - * @param

Type of the parent metric group - */ -public class ProxyMetricGroup

implements MetricGroup { - - protected final P parentMetricGroup; - - public ProxyMetricGroup(P parentMetricGroup) { - this.parentMetricGroup = checkNotNull(parentMetricGroup); - } - - @Override - public final Counter counter(int name) { - return parentMetricGroup.counter(name); - } - - @Override - public final Counter counter(String name) { - return parentMetricGroup.counter(name); - } - - @Override - public final C counter(int name, C counter) { - return parentMetricGroup.counter(name, counter); - } - - @Override - public final C counter(String name, C counter) { - return parentMetricGroup.counter(name, counter); - } - - @Override - public final > G gauge(int name, G gauge) { - return parentMetricGroup.gauge(name, gauge); - } - - @Override - public final > G gauge(String name, G gauge) { - return parentMetricGroup.gauge(name, gauge); - } - - @Override - public M meter(String name, M meter) { - return parentMetricGroup.meter(name, meter); - } - - @Override - public M meter(int name, M meter) { - return parentMetricGroup.meter(name, meter); - } - - @Override - public final MetricGroup addGroup(int name) { - return parentMetricGroup.addGroup(name); - } - - @Override - public final MetricGroup addGroup(String name) { - return parentMetricGroup.addGroup(name); - } - - @Override - public String[] getScopeComponents() { - return parentMetricGroup.getScopeComponents(); - } - - @Override - public Map getAllVariables() { - return parentMetricGroup.getAllVariables(); - } - - @Override - public String getMetricIdentifier(String metricName) { - return parentMetricGroup.getMetricIdentifier(metricName); - } - -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/AbstractPrometheusReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/AbstractPrometheusReporter.java deleted file mode 100644 index 5e361ccf2b..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/AbstractPrometheusReporter.java +++ /dev/null @@ -1,315 +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.metrics.promethues; - -import com.dtstack.flinkx.metrics.base.MetricGroup; -import com.dtstack.flinkx.metrics.base.reporter.MetricReporter; -import com.dtstack.flinkx.metrics.groups.AbstractMetricGroup; -import com.dtstack.flinkx.metrics.groups.FrontMetricGroup; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.metrics.Histogram; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.Metric; -import org.apache.flink.metrics.MetricConfig; -import io.prometheus.client.Collector; -import io.prometheus.client.CollectorRegistry; -import org.apache.flink.metrics.CharacterFilter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.AbstractMap; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.regex.Pattern; - -import static com.dtstack.flinkx.metrics.promethues.PrometheusPushGatewayReporterOptions.FILTER_LABEL_VALUE_CHARACTER; - -/** - * base prometheus reporter for prometheus metrics. - */ -public abstract class AbstractPrometheusReporter implements MetricReporter { - - protected final Logger log = LoggerFactory.getLogger(getClass()); - - private static final Pattern UNALLOWED_CHAR_PATTERN = Pattern.compile("[^a-zA-Z0-9:_]"); - private static final CharacterFilter CHARACTER_FILTER = new CharacterFilter() { - @Override - public String filterCharacters(String input) { - return replaceInvalidChars(input); - } - }; - - private static final char SCOPE_SEPARATOR = '_'; - private static final String SCOPE_PREFIX = "flinkx" + SCOPE_SEPARATOR; - - private final Map> collectorsWithCountByMetricName = new HashMap<>(); - - static String replaceInvalidChars(final String input) { - // https://prometheus.io/docs/instrumenting/writing_exporters/ - // Only [a-zA-Z0-9:_] are valid in metric names, any other characters should be sanitized to an underscore. - return UNALLOWED_CHAR_PATTERN.matcher(input).replaceAll("_"); - } - - private CharacterFilter labelValueCharactersFilter = CHARACTER_FILTER; - - @Override - public void open(MetricConfig config) { - boolean filterLabelValueCharacters = config.getBoolean( - FILTER_LABEL_VALUE_CHARACTER.key(), FILTER_LABEL_VALUE_CHARACTER.defaultValue()); - - if (!filterLabelValueCharacters) { - labelValueCharactersFilter = input -> input; - } - } - - @Override - public void close() { - CollectorRegistry.defaultRegistry.clear(); - } - - @Override - public void notifyOfAddedMetric(final Metric metric, final String metricName, final MetricGroup group) { - - List dimensionKeys = new LinkedList<>(); - List dimensionValues = new LinkedList<>(); - for (final Map.Entry dimension : group.getAllVariables().entrySet()) { - final String key = dimension.getKey(); - dimensionKeys.add(CHARACTER_FILTER.filterCharacters(key.substring(1, key.length() - 1))); - dimensionValues.add(labelValueCharactersFilter.filterCharacters(dimension.getValue())); - } - - final String scopedMetricName = getScopedName(metricName, group); - final String helpString = metricName + " (scope: " + getLogicalScope(group) + ")"; - - final Collector collector; - Integer count = 0; - - synchronized (this) { - if (collectorsWithCountByMetricName.containsKey(scopedMetricName)) { - final AbstractMap.SimpleImmutableEntry collectorWithCount = collectorsWithCountByMetricName.get(scopedMetricName); - collector = collectorWithCount.getKey(); - count = collectorWithCount.getValue(); - } else { - collector = createCollector(metric, dimensionKeys, dimensionValues, scopedMetricName, helpString); - try { - collector.register(); - } catch (Exception e) { - log.warn("There was a problem registering metric {}.", metricName, e); - } - } - addMetric(metric, dimensionValues, collector); - collectorsWithCountByMetricName.put(scopedMetricName, new AbstractMap.SimpleImmutableEntry<>(collector, count + 1)); - } - } - - private static String getScopedName(String metricName, MetricGroup group) { - return SCOPE_PREFIX + getLogicalScope(group) + SCOPE_SEPARATOR + CHARACTER_FILTER.filterCharacters(metricName); - } - - private Collector createCollector(Metric metric, List dimensionKeys, List dimensionValues, String scopedMetricName, String helpString) { - Collector collector; - if (metric instanceof Gauge || metric instanceof Counter || metric instanceof Meter) { - collector = io.prometheus.client.Gauge - .build() - .name(scopedMetricName) - .help(helpString) - .labelNames(toArray(dimensionKeys)) - .create(); - } else if (metric instanceof Histogram) { - collector = new HistogramSummaryProxy((Histogram) metric, scopedMetricName, helpString, dimensionKeys, dimensionValues); - } else { - log.warn("Cannot create collector for unknown metric type: {}. This indicates that the metric type is not supported by this reporter.", - metric.getClass().getName()); - collector = null; - } - return collector; - } - - private void addMetric(Metric metric, List dimensionValues, Collector collector) { - if (metric instanceof Gauge) { - ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((Gauge) metric), toArray(dimensionValues)); - } else if (metric instanceof Counter) { - ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((Counter) metric), toArray(dimensionValues)); - } else if (metric instanceof Meter) { - ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((Meter) metric), toArray(dimensionValues)); - } else if (metric instanceof Histogram) { - ((HistogramSummaryProxy) collector).addChild((Histogram) metric, dimensionValues); - } else { - log.warn("Cannot add unknown metric type: {}. This indicates that the metric type is not supported by this reporter.", - metric.getClass().getName()); - } - } - - private void removeMetric(Metric metric, List dimensionValues, Collector collector) { - if (metric instanceof Gauge) { - ((io.prometheus.client.Gauge) collector).remove(toArray(dimensionValues)); - } else if (metric instanceof Counter) { - ((io.prometheus.client.Gauge) collector).remove(toArray(dimensionValues)); - } else if (metric instanceof Meter) { - ((io.prometheus.client.Gauge) collector).remove(toArray(dimensionValues)); - } else if (metric instanceof Histogram) { - ((HistogramSummaryProxy) collector).remove(dimensionValues); - } else { - log.warn("Cannot remove unknown metric type: {}. This indicates that the metric type is not supported by this reporter.", - metric.getClass().getName()); - } - } - - @Override - public void notifyOfRemovedMetric(final Metric metric, final String metricName, final MetricGroup group) { - - List dimensionValues = new LinkedList<>(); - for (final Map.Entry dimension : group.getAllVariables().entrySet()) { - dimensionValues.add(labelValueCharactersFilter.filterCharacters(dimension.getValue())); - } - - final String scopedMetricName = getScopedName(metricName, group); - synchronized (this) { - final AbstractMap.SimpleImmutableEntry collectorWithCount = collectorsWithCountByMetricName.get(scopedMetricName); - final Integer count = collectorWithCount.getValue(); - final Collector collector = collectorWithCount.getKey(); - - removeMetric(metric, dimensionValues, collector); - - if (count == 1) { - try { - CollectorRegistry.defaultRegistry.unregister(collector); - } catch (Exception e) { - log.warn("There was a problem unregistering metric {}.", scopedMetricName, e); - } - collectorsWithCountByMetricName.remove(scopedMetricName); - } else { - collectorsWithCountByMetricName.put(scopedMetricName, new AbstractMap.SimpleImmutableEntry<>(collector, count - 1)); - } - } - } - - @SuppressWarnings("unchecked") - private static String getLogicalScope(MetricGroup group) { - return ((FrontMetricGroup>) group).getLogicalScope(CHARACTER_FILTER, SCOPE_SEPARATOR); - } - - io.prometheus.client.Gauge.Child gaugeFrom(Gauge gauge) { - return new io.prometheus.client.Gauge.Child() { - @Override - public double get() { - final Object value = gauge.getValue(); - if (value == null) { - log.debug("Gauge {} is null-valued, defaulting to 0.", gauge); - return 0; - } - if (value instanceof Double) { - return (double) value; - } - if (value instanceof Number) { - return ((Number) value).doubleValue(); - } - if (value instanceof Boolean) { - return ((Boolean) value) ? 1 : 0; - } - log.debug("Invalid type for Gauge {}: {}, only number types and booleans are supported by this reporter.", - gauge, value.getClass().getName()); - return 0; - } - }; - } - - private static io.prometheus.client.Gauge.Child gaugeFrom(Counter counter) { - return new io.prometheus.client.Gauge.Child() { - @Override - public double get() { - return (double) counter.getCount(); - } - }; - } - - private static io.prometheus.client.Gauge.Child gaugeFrom(Meter meter) { - return new io.prometheus.client.Gauge.Child() { - @Override - public double get() { - return meter.getRate(); - } - }; - } - - @VisibleForTesting - static class HistogramSummaryProxy extends Collector { - static final List QUANTILES = Arrays.asList(.5, .75, .95, .98, .99, .999); - - private final String metricName; - private final String helpString; - private final List labelNamesWithQuantile; - - private final Map, Histogram> histogramsByLabelValues = new HashMap<>(); - - HistogramSummaryProxy(final Histogram histogram, final String metricName, final String helpString, final List labelNames, final List labelValues) { - this.metricName = metricName; - this.helpString = helpString; - this.labelNamesWithQuantile = addToList(labelNames, "quantile"); - histogramsByLabelValues.put(labelValues, histogram); - } - - @Override - public List collect() { - // We cannot use SummaryMetricFamily because it is impossible to get a sum of all values (at least for Dropwizard histograms, - // whose snapshot's values array only holds a sample of recent values). - - List samples = new LinkedList<>(); - for (Map.Entry, Histogram> labelValuesToHistogram : histogramsByLabelValues.entrySet()) { - addSamples(labelValuesToHistogram.getKey(), labelValuesToHistogram.getValue(), samples); - } - return Collections.singletonList(new MetricFamilySamples(metricName, Type.SUMMARY, helpString, samples)); - } - - void addChild(final Histogram histogram, final List labelValues) { - histogramsByLabelValues.put(labelValues, histogram); - } - - void remove(final List labelValues) { - histogramsByLabelValues.remove(labelValues); - } - - private void addSamples(final List labelValues, final Histogram histogram, final List samples) { - samples.add(new MetricFamilySamples.Sample(metricName + "_count", - labelNamesWithQuantile.subList(0, labelNamesWithQuantile.size() - 1), labelValues, histogram.getCount())); - for (final Double quantile : QUANTILES) { - samples.add(new MetricFamilySamples.Sample(metricName, labelNamesWithQuantile, - addToList(labelValues, quantile.toString()), - histogram.getStatistics().getQuantile(quantile))); - } - } - } - - private static List addToList(List list, String element) { - final List result = new ArrayList<>(list); - result.add(element); - return result; - } - - private static String[] toArray(List list) { - return list.toArray(new String[list.size()]); - } -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporter.java deleted file mode 100644 index 0a488ec145..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporter.java +++ /dev/null @@ -1,90 +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.metrics.promethues; - -import org.apache.flink.metrics.Metric; -import org.apache.flink.metrics.MetricConfig; -import org.apache.flink.metrics.reporter.Scheduled; -import io.prometheus.client.CollectorRegistry; -import io.prometheus.client.exporter.PushGateway; -import org.apache.flink.util.AbstractID; - -import java.io.IOException; - -import static com.dtstack.flinkx.metrics.promethues.PrometheusPushGatewayReporterOptions.DELETE_ON_SHUTDOWN; -import static com.dtstack.flinkx.metrics.promethues.PrometheusPushGatewayReporterOptions.HOST; -import static com.dtstack.flinkx.metrics.promethues.PrometheusPushGatewayReporterOptions.JOB_NAME; -import static com.dtstack.flinkx.metrics.promethues.PrometheusPushGatewayReporterOptions.PORT; -import static com.dtstack.flinkx.metrics.promethues.PrometheusPushGatewayReporterOptions.RANDOM_JOB_NAME_SUFFIX; - -/** - * {@link com.dtstack.flinkx.metrics.base.reporter.MetricReporter} that exports {@link Metric Metrics} via Prometheus {@link PushGateway}. - */ -public class PrometheusPushGatewayReporter extends AbstractPrometheusReporter implements Scheduled { - - private PushGateway pushGateway; - private String jobName; - private boolean deleteOnShutdown; - - @Override - public void open(MetricConfig config) { - super.open(config); - - String host = config.getString(HOST.key(), HOST.defaultValue()); - int port = config.getInteger(PORT.key(), PORT.defaultValue()); - String configuredJobName = config.getString(JOB_NAME.key(), JOB_NAME.defaultValue()); - boolean randomSuffix = config.getBoolean(RANDOM_JOB_NAME_SUFFIX.key(), RANDOM_JOB_NAME_SUFFIX.defaultValue()); - deleteOnShutdown = config.getBoolean(DELETE_ON_SHUTDOWN.key(), DELETE_ON_SHUTDOWN.defaultValue()); - - if (host == null || host.isEmpty() || port < 1) { - throw new IllegalArgumentException("Invalid host/port configuration. Host: " + host + " Port: " + port); - } - - if (randomSuffix) { - this.jobName = configuredJobName + new AbstractID(); - } else { - this.jobName = configuredJobName; - } - - - pushGateway = new PushGateway(host + ':' + port); - log.info("Configured PrometheusPushGatewayReporter with {host:{}, port:{}, jobName: {}, deleteOnShutdown:{}}", host, port, jobName, deleteOnShutdown); - } - - @Override - public void report() { - try { - pushGateway.push(CollectorRegistry.defaultRegistry, jobName); - } catch (Exception e) { - log.warn("Failed to push metrics to PushGateway with jobName {}.", jobName, e); - } - } - - @Override - public void close() { - if (deleteOnShutdown && pushGateway != null) { - try { - pushGateway.delete(jobName); - } catch (IOException e) { - log.warn("Failed to delete metrics from PushGateway with jobName {}.", jobName, e); - } - } - super.close(); - } -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporterOptions.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporterOptions.java deleted file mode 100644 index b6c2f9b132..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusPushGatewayReporterOptions.java +++ /dev/null @@ -1,62 +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.metrics.promethues; - -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; - -/** - * Config options for the {@link PrometheusPushGatewayReporter}. - */ -public class PrometheusPushGatewayReporterOptions { - - public static final ConfigOption HOST = ConfigOptions - .key("host") - .noDefaultValue() - .withDescription("The PushGateway server host."); - - public static final ConfigOption PORT = ConfigOptions - .key("port") - .defaultValue(-1) - .withDescription("The PushGateway server port."); - - public static final ConfigOption JOB_NAME = ConfigOptions - .key("jobName") - .defaultValue("") - .withDescription("The job name under which metrics will be pushed"); - - public static final ConfigOption RANDOM_JOB_NAME_SUFFIX = ConfigOptions - .key("randomJobNameSuffix") - .defaultValue(true) - .withDescription("Specifies whether a random suffix should be appended to the job name."); - - public static final ConfigOption DELETE_ON_SHUTDOWN = ConfigOptions - .key("deleteOnShutdown") - .defaultValue(true) - .withDescription("Specifies whether to delete metrics from the PushGateway on shutdown."); - - public static final ConfigOption FILTER_LABEL_VALUE_CHARACTER = ConfigOptions - .key("filterLabelValueCharacters") - .defaultValue(true) - .withDescription("Specifies whether to filter label value characters." + - " If enabled, all characters not matching [a-zA-Z0-9:_] will be removed," + - " otherwise no characters will be removed." + - " Before disabling this option please ensure that your" + - " label values meet the Prometheus requirements(https://prometheus.io/docs/concepts/data_model/#metric-names-and-label)s."); -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusReporter.java deleted file mode 100644 index ca8f5596fb..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/promethues/PrometheusReporter.java +++ /dev/null @@ -1,79 +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.metrics.promethues; - -import org.apache.flink.metrics.Metric; -import org.apache.flink.metrics.MetricConfig; -import org.apache.flink.util.NetUtils; -import org.apache.flink.util.Preconditions; -import io.prometheus.client.exporter.HTTPServer; - -import java.io.IOException; -import java.util.Iterator; - -/** - * {@link com.dtstack.flinkx.metrics.base.reporter.MetricReporter} that exports {@link Metric Metrics} via Prometheus. - */ -public class PrometheusReporter extends AbstractPrometheusReporter { - - static final String ARG_PORT = "port"; - private static final String DEFAULT_PORT = "9249"; - - private HTTPServer httpServer; - private int port; - - int getPort() { - Preconditions.checkState(httpServer != null, "Server has not been initialized."); - return port; - } - - @Override - public void open(MetricConfig config) { - super.open(config); - - String portsConfig = config.getString(ARG_PORT, DEFAULT_PORT); - Iterator ports = NetUtils.getPortRangeFromString(portsConfig); - - while (ports.hasNext()) { - int port = ports.next(); - try { - // internally accesses CollectorRegistry.defaultRegistry - httpServer = new HTTPServer(port); - this.port = port; - log.info("Started PrometheusReporter HTTP server on port {}.", port); - break; - } catch (IOException ioe) { //assume port conflict - log.debug("Could not start PrometheusReporter HTTP server on port {}.", port, ioe); - } - } - if (httpServer == null) { - throw new RuntimeException("Could not start PrometheusReporter HTTP server on any configured port. Ports: " + portsConfig); - } - } - - @Override - public void close() { - if (httpServer != null) { - httpServer.stop(); - } - - super.close(); - } - -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/PipelineScopeFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/PipelineScopeFormat.java deleted file mode 100644 index 8e7b8d0f25..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/PipelineScopeFormat.java +++ /dev/null @@ -1,38 +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.metrics.scope; - - -public class PipelineScopeFormat extends ScopeFormat { - - public PipelineScopeFormat(String format) { - super(format, null, new String[]{ - SCOPE_HOST, - SCOPE_PLUGINE_TYPE, - SCOPE_PLUGINE_NAME, - SCOPE_JOB_NAME - }); - } - - public String[] formatScope(String hostname, String pluginType, String pluginName, String jobName) { - final String[] template = copyTemplate(); - final String[] values = {hostname, pluginType, pluginName, jobName}; - return bindVariables(template, values); - } -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormat.java deleted file mode 100644 index b5a574fd81..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormat.java +++ /dev/null @@ -1,264 +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.metrics.scope; - -import org.apache.flink.metrics.CharacterFilter; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * This class represents the format after which the "scope" (or namespace) of the various - * component metric groups is built. Component metric groups are for example - * "TaskManager", "Task", or "Operator". - * - *

User defined scope formats allow users to include or exclude - * certain identifiers from the scope. The scope for metrics belonging to the "Task" - * group could for example include the task attempt number (more fine grained identification), or - * exclude it (continuity of the namespace across failure and recovery). - */ -public abstract class ScopeFormat { - - private static CharacterFilter defaultFilter = new CharacterFilter() { - @Override - public String filterCharacters(String input) { - return input; - } - }; - - // ------------------------------------------------------------------------ - // Scope Format Special Characters - // ------------------------------------------------------------------------ - - /** - * If the scope format starts with this character, then the parent components scope - * format will be used as a prefix. - * - *

For example, if the TaskManager's job format is {@code "*."}, and the - * TaskManager format is {@code ""}, then the job's metrics - * will have {@code "."} as their scope. - */ - public static final String SCOPE_INHERIT_PARENT = "*"; - - public static final String SCOPE_SEPARATOR = "."; - - private static final String SCOPE_VARIABLE_PREFIX = "<"; - private static final String SCOPE_VARIABLE_SUFFIX = ">"; - - // ------------------------------------------------------------------------ - // Scope Variables - // ------------------------------------------------------------------------ - - public static final String SCOPE_HOST = asVariable("host"); - - // ----- Task Manager ---- - - public static final String SCOPE_TASKMANAGER_ID = asVariable("tm_id"); - - // ----- Job ----- - - public static final String SCOPE_JOB_ID = asVariable("job_id"); - public static final String SCOPE_JOB_NAME = asVariable("job_name"); - - // ----- Task ---- - - public static final String SCOPE_TASK_VERTEX_ID = asVariable("task_id"); - public static final String SCOPE_TASK_NAME = asVariable("task_name"); - public static final String SCOPE_TASK_ATTEMPT_ID = asVariable("task_attempt_id"); - public static final String SCOPE_TASK_ATTEMPT_NUM = asVariable("task_attempt_num"); - public static final String SCOPE_TASK_SUBTASK_INDEX = asVariable("subtask_index"); - - - public static final String SCOPE_PLUGINE_TYPE = asVariable("plugin_type"); - - public static final String SCOPE_PLUGINE_NAME = asVariable("plugin_name"); - - // ----- Operator ---- - - public static final String SCOPE_OPERATOR_ID = asVariable("operator_id"); - public static final String SCOPE_OPERATOR_NAME = asVariable("operator_name"); - - - // ------------------------------------------------------------------------ - // Scope Format Base - // ------------------------------------------------------------------------ - - /** The scope format. */ - private final String format; - - /** The format, split into components. */ - private final String[] template; - - private final int[] templatePos; - - private final int[] valuePos; - - // ------------------------------------------------------------------------ - - protected ScopeFormat(String format, ScopeFormat parent, String[] variables) { - checkNotNull(format, "format is null"); - - final String[] rawComponents = format.split("\\" + SCOPE_SEPARATOR); - - // compute the template array - final boolean parentAsPrefix = rawComponents.length > 0 && rawComponents[0].equals(SCOPE_INHERIT_PARENT); - if (parentAsPrefix) { - if (parent == null) { - throw new IllegalArgumentException("Component scope format requires parent prefix (starts with '" - + SCOPE_INHERIT_PARENT + "'), but this component has no parent (is root component)."); - } - - this.format = format.length() > 2 ? format.substring(2) : ""; - - String[] parentTemplate = parent.template; - int parentLen = parentTemplate.length; - - this.template = new String[parentLen + rawComponents.length - 1]; - System.arraycopy(parentTemplate, 0, this.template, 0, parentLen); - System.arraycopy(rawComponents, 1, this.template, parentLen, rawComponents.length - 1); - } - else { - this.format = format.isEmpty() ? "" : format; - this.template = rawComponents; - } - - // --- compute the replacement matrix --- - // a bit of clumsy Java collections code ;-) - - HashMap varToValuePos = arrayToMap(variables); - List templatePos = new ArrayList<>(); - List valuePos = new ArrayList<>(); - - for (int i = 0; i < template.length; i++) { - final String component = template[i]; - - // check if that is a variable - if (component != null && component.length() >= 3 && - component.charAt(0) == '<' && component.charAt(component.length() - 1) == '>') { - - // this is a variable - Integer replacementPos = varToValuePos.get(component); - if (replacementPos != null) { - templatePos.add(i); - valuePos.add(replacementPos); - } - } - } - - this.templatePos = integerListToArray(templatePos); - this.valuePos = integerListToArray(valuePos); - } - - // ------------------------------------------------------------------------ - - public String format() { - return format; - } - - protected final String[] copyTemplate() { - String[] copy = new String[template.length]; - System.arraycopy(template, 0, copy, 0, template.length); - return copy; - } - - protected final String[] bindVariables(String[] template, String[] values) { - final int len = templatePos.length; - for (int i = 0; i < len; i++) { - template[templatePos[i]] = values[valuePos[i]]; - } - return template; - } - - // ------------------------------------------------------------------------ - - @Override - public String toString() { - return "ScopeFormat '" + format + '\''; - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - /** - * Formats the given string to resemble a scope variable. - * - * @param scope The string to format - * @return The formatted string - */ - public static String asVariable(String scope) { - return SCOPE_VARIABLE_PREFIX + scope + SCOPE_VARIABLE_SUFFIX; - } - - public static String concat(String... components) { - return concat(defaultFilter, '.', components); - } - - public static String concat(CharacterFilter filter, String... components) { - return concat(filter, '.', components); - } - - public static String concat(Character delimiter, String... components) { - return concat(defaultFilter, delimiter, components); - } - - /** - * Concatenates the given component names separated by the delimiter character. Additionally - * the character filter is applied to all component names. - * - * @param filter Character filter to be applied to the component names - * @param delimiter Delimiter to separate component names - * @param components Array of component names - * @return The concatenated component name - */ - public static String concat(CharacterFilter filter, Character delimiter, String... components) { - StringBuilder sb = new StringBuilder(); - sb.append(filter.filterCharacters(components[0])); - for (int x = 1; x < components.length; x++) { - sb.append(delimiter); - sb.append(filter.filterCharacters(components[x])); - } - return sb.toString(); - } - - protected static String valueOrNull(Object value) { - return (value == null || (value instanceof String && ((String) value).isEmpty())) ? - "null" : value.toString(); - } - - protected static HashMap arrayToMap(String[] array) { - HashMap map = new HashMap<>(array.length); - for (int i = 0; i < array.length; i++) { - map.put(array[i], i); - } - return map; - } - - private static int[] integerListToArray(List list) { - int[] array = new int[list.size()]; - int pos = 0; - for (Integer i : list) { - array[pos++] = i; - } - return array; - } -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormats.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormats.java deleted file mode 100644 index 7ba20bdebe..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/scope/ScopeFormats.java +++ /dev/null @@ -1,57 +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.metrics.scope; - -/** - * A container for component scope formats. - */ -public final class ScopeFormats { - - private final PipelineScopeFormat pipelineScopeFormat; - - // ------------------------------------------------------------------------ - - /** - * Creates all scope formats, based on the given scope format strings. - */ - private ScopeFormats() { - this.pipelineScopeFormat = new PipelineScopeFormat(".flinkx..."); - } - - // ------------------------------------------------------------------------ - // Accessors - // ------------------------------------------------------------------------ - - public PipelineScopeFormat getPipelineScopeFormat() { - return pipelineScopeFormat; - } - - // ------------------------------------------------------------------------ - // Parsing from Config - // ------------------------------------------------------------------------ - - /** - * Creates the scope formats as defined in the given configuration. - * - * @return The ScopeFormats parsed from the configuration - */ - public static ScopeFormats fromDefault() { - return new ScopeFormats(); - } -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java index 42bffa1336..084083293d 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java @@ -23,6 +23,7 @@ import com.dtstack.flinkx.latch.Latch; import com.dtstack.flinkx.latch.LocalLatch; import com.dtstack.flinkx.latch.MetricLatch; +import com.dtstack.flinkx.metrics.OutputMetric; import com.dtstack.flinkx.writer.DirtyDataManager; import com.dtstack.flinkx.writer.ErrorLimiter; import org.apache.commons.lang.StringUtils; @@ -111,6 +112,8 @@ public abstract class RichOutputFormat extends org.apache.flink.api.common.io.Ri protected String jobId; + protected OutputMetric outputMetric; + public DirtyDataManager getDirtyDataManager() { return dirtyDataManager; } @@ -163,6 +166,8 @@ public void open(int taskNumber, int numTasks) throws IOException { context = (StreamingRuntimeContext) getRuntimeContext(); this.numTasks = numTasks; + outputMetric = new OutputMetric(context); + //错误记录数 errCounter = context.getIntCounter(Metrics.NUM_ERRORS); nullErrCounter = context.getIntCounter(Metrics.NUM_NULL_ERRORS); @@ -236,8 +241,11 @@ protected void writeSingleRecord(Row row) { // 总记录数加1 numWriteCounter.add(1); + outputMetric.getNumWrite().inc(); + outputMetric.getNumRecordsOut(); } catch(WriteRecordException e) { errCounter.add(1); + outputMetric.getNumErrors().inc(); String errMsg = e.getMessage(); int pos = e.getColIndex(); @@ -254,12 +262,16 @@ protected void writeSingleRecord(Row row) { String errorType = dirtyDataManager.writeData(row, e); if (ERR_NULL_POINTER.equals(errorType)){ nullErrCounter.add(1); + outputMetric.getNumNullErrors().inc(); } else if(ERR_FORMAT_TRANSFORM.equals(errorType)){ conversionErrCounter.add(1); + outputMetric.getNumConversionErrors().inc(); } else if(ERR_PRIMARY_CONFLICT.equals(errorType)){ duplicateErrCounter.add(1); + outputMetric.getNumDuplicateErrors().inc(); } else { otherErrCounter.add(1); + outputMetric.getNumOtherErrors().inc(); } } From 5b31f777dbb07c8c89874eac88f7dcba0356253e Mon Sep 17 00:00:00 2001 From: toutian Date: Mon, 18 Mar 2019 17:24:25 +0800 Subject: [PATCH 26/96] metric --- .../flinkx/inputformat/RichInputFormat.java | 4 +- .../dtstack/flinkx/metrics/DTMetricNames.java | 48 ------------ .../dtstack/flinkx/metrics/InputMetric.java | 57 +++++++------- .../dtstack/flinkx/metrics/OutputMetric.java | 78 ++++++++----------- .../flinkx/outputformat/RichOutputFormat.java | 3 +- 5 files changed, 62 insertions(+), 128 deletions(-) delete mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/DTMetricNames.java diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java index e6cadbddc4..91bc9b79b1 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java @@ -51,7 +51,7 @@ public abstract class RichInputFormat extends org.apache.flink.api.common.io.Ric protected long bytes; protected ByteRateLimiter byteRateLimiter; - protected InputMetric inputMetric; + protected transient InputMetric inputMetric; protected abstract void openInternal(InputSplit inputSplit) throws IOException; @@ -79,7 +79,7 @@ public void open(InputSplit inputSplit) throws IOException { @Override public Row nextRecord(Row row) throws IOException { numReadCounter.add(1); - inputMetric.getNumRecordsIn().inc(); + inputMetric.getNumRead().inc(); if(byteRateLimiter != null) { byteRateLimiter.acquire(); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/DTMetricNames.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/DTMetricNames.java deleted file mode 100644 index 6ab3179abe..0000000000 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/DTMetricNames.java +++ /dev/null @@ -1,48 +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.metrics; - -/** - * Collection of dtstack metric names. - */ -public class DTMetricNames { - - private DTMetricNames() { - } - - private static final String SUFFIX_RATE = "PerSecond"; - - public static final String IO_NUM_RECORDS_IN = "numRecordsIn"; - public static final String IO_NUM_RECORDS_OUT = "numRecordsOut"; - public static final String IO_NUM_RECORDS_IN_RATE = IO_NUM_RECORDS_IN + SUFFIX_RATE; - public static final String IO_NUM_RECORDS_OUT_RATE = IO_NUM_RECORDS_OUT + SUFFIX_RATE; - - public static final String IO_NUM_BYTES_IN = "numBytesIn"; - public static final String IO_NUM_BYTES_OUT = "numBytesOut"; - public static final String IO_NUM_BYTES_IN_RATE = IO_NUM_BYTES_IN + SUFFIX_RATE; - public static final String IO_NUM_BYTES_OUT_RATE = IO_NUM_BYTES_OUT + SUFFIX_RATE; - - public static final String NUM_ERRORS = "nErrors"; - public static final String NUM_NULL_ERRORS = "nullErrors"; - public static final String NUM_DUPLICATE_ERRORS = "duplicateErrors"; - public static final String NUM_CONVERSION_ERRORS = "conversionErrors"; - public static final String NUM_OTHER_ERRORS = "otherErrors"; - public static final String NUM_WRITES = "numWrite"; - -} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java index 17a05d6149..76b52937dd 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java @@ -1,10 +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.flinkx.metrics; -import com.dtstack.flinkx.inputformat.RichInputFormat; +import com.dtstack.flinkx.constants.Metrics; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; /** * company: www.dtstack.com @@ -13,13 +29,9 @@ */ public class InputMetric { - protected transient Counter numRecordsIn; - protected transient Meter numRecordsInRate; + private Counter numRead; - protected transient Counter numBytesIn; - protected transient Meter numInBytesRate; - - private transient RuntimeContext runtimeContext; + private RuntimeContext runtimeContext; public InputMetric(RuntimeContext runtimeContext) { this.runtimeContext = runtimeContext; @@ -27,32 +39,15 @@ public InputMetric(RuntimeContext runtimeContext) { initMetric(); } - public void initMetric() { - - numRecordsIn = getRuntimeContext().getMetricGroup().counter(DTMetricNames.IO_NUM_RECORDS_IN); - numRecordsInRate = getRuntimeContext().getMetricGroup().meter(DTMetricNames.IO_NUM_RECORDS_IN_RATE, new MeterView(numRecordsIn, 20)); - - numBytesIn = getRuntimeContext().getMetricGroup().counter(DTMetricNames.IO_NUM_BYTES_IN); - numInBytesRate = getRuntimeContext().getMetricGroup().meter(DTMetricNames.IO_NUM_BYTES_IN_RATE, new MeterView(numBytesIn, 20)); - } - - public Counter getNumRecordsIn() { - return numRecordsIn; - } - - public Meter getNumRecordsInRate() { - return numRecordsInRate; - } - - public Counter getNumBytesIn() { - return numBytesIn; + private void initMetric() { + numRead = getRuntimeContext().getMetricGroup().counter(Metrics.NUM_READS); } - public Meter getNumInBytesRate() { - return numInBytesRate; + public Counter getNumRead() { + return numRead; } - public RuntimeContext getRuntimeContext() { + private RuntimeContext getRuntimeContext() { return runtimeContext; } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/OutputMetric.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/OutputMetric.java index bfc8e0bbf9..06afb6c0cb 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/OutputMetric.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/OutputMetric.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.flinkx.metrics; +import com.dtstack.flinkx.constants.Metrics; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; /** * company: www.dtstack.com @@ -12,18 +29,12 @@ */ public class OutputMetric { - protected transient Counter numErrors; - protected transient Counter numNullErrors; - protected transient Counter numDuplicateErrors; - protected transient Counter numConversionErrors; - protected transient Counter numOtherErrors; - protected transient Counter numWrite; - - protected transient Counter numRecordsOut; - protected transient Meter numRecordsOutRate; - - protected transient Counter numBytesOut; - protected transient Meter numBytesOutRate; + private Counter numErrors; + private Counter numNullErrors; + private Counter numDuplicateErrors; + private Counter numConversionErrors; + private Counter numOtherErrors; + private Counter numWrite; private transient RuntimeContext runtimeContext; @@ -33,20 +44,13 @@ public OutputMetric(RuntimeContext runtimeContext) { initMetric(); } - public void initMetric() { - - numErrors = getRuntimeContext().getMetricGroup().counter(DTMetricNames.NUM_ERRORS); - numNullErrors = getRuntimeContext().getMetricGroup().counter(DTMetricNames.NUM_NULL_ERRORS); - numDuplicateErrors = getRuntimeContext().getMetricGroup().counter(DTMetricNames.NUM_DUPLICATE_ERRORS); - numConversionErrors = getRuntimeContext().getMetricGroup().counter(DTMetricNames.NUM_CONVERSION_ERRORS); - numOtherErrors = getRuntimeContext().getMetricGroup().counter(DTMetricNames.NUM_OTHER_ERRORS); - numWrite = getRuntimeContext().getMetricGroup().counter(DTMetricNames.NUM_WRITES); - - numRecordsOut = getRuntimeContext().getMetricGroup().counter(DTMetricNames.IO_NUM_RECORDS_OUT); - numRecordsOutRate = getRuntimeContext().getMetricGroup().meter(DTMetricNames.IO_NUM_RECORDS_OUT_RATE, new MeterView(numRecordsOut, 20)); - - numBytesOut = getRuntimeContext().getMetricGroup().counter(DTMetricNames.IO_NUM_BYTES_OUT); - numBytesOutRate = getRuntimeContext().getMetricGroup().meter(DTMetricNames.IO_NUM_BYTES_OUT_RATE, new MeterView(numBytesOut, 20)); + private void initMetric() { + numErrors = getRuntimeContext().getMetricGroup().counter(Metrics.NUM_ERRORS); + numNullErrors = getRuntimeContext().getMetricGroup().counter(Metrics.NUM_NULL_ERRORS); + numDuplicateErrors = getRuntimeContext().getMetricGroup().counter(Metrics.NUM_DUPLICATE_ERRORS); + numConversionErrors = getRuntimeContext().getMetricGroup().counter(Metrics.NUM_CONVERSION_ERRORS); + numOtherErrors = getRuntimeContext().getMetricGroup().counter(Metrics.NUM_OTHER_ERRORS); + numWrite = getRuntimeContext().getMetricGroup().counter(Metrics.NUM_WRITES); } public Counter getNumErrors() { @@ -73,23 +77,7 @@ public Counter getNumWrite() { return numWrite; } - public Counter getNumRecordsOut() { - return numRecordsOut; - } - - public Meter getNumRecordsOutRate() { - return numRecordsOutRate; - } - - public Counter getNumBytesOut() { - return numBytesOut; - } - - public Meter getNumBytesOutRate() { - return numBytesOutRate; - } - - public RuntimeContext getRuntimeContext() { + private RuntimeContext getRuntimeContext() { return runtimeContext; } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java index 084083293d..866b79125d 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java @@ -112,7 +112,7 @@ public abstract class RichOutputFormat extends org.apache.flink.api.common.io.Ri protected String jobId; - protected OutputMetric outputMetric; + protected transient OutputMetric outputMetric; public DirtyDataManager getDirtyDataManager() { return dirtyDataManager; @@ -242,7 +242,6 @@ protected void writeSingleRecord(Row row) { // 总记录数加1 numWriteCounter.add(1); outputMetric.getNumWrite().inc(); - outputMetric.getNumRecordsOut(); } catch(WriteRecordException e) { errCounter.add(1); outputMetric.getNumErrors().inc(); From e766f4c49cb713930b6302565bfdfa088e87e4a2 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 19 Mar 2019 11:00:58 +0800 Subject: [PATCH 27/96] =?UTF-8?q?=E6=A3=80=E6=9F=A5connection=E6=98=AF?= =?UTF-8?q?=E5=90=A6=E5=85=B3=E9=97=AD?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/hbase/reader/HbaseInputFormat.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java index f5060b6597..06d2bb1b1b 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java @@ -70,11 +70,13 @@ public class HbaseInputFormat extends RichInputFormat { private transient ResultScanner resultScanner; private transient Result next; + private org.apache.hadoop.conf.Configuration hConfiguration; + @Override public void configure(Configuration configuration) { LOG.info("HbaseOutputFormat configure start"); - org.apache.hadoop.conf.Configuration hConfiguration = new org.apache.hadoop.conf.Configuration(); + hConfiguration = new org.apache.hadoop.conf.Configuration(); Validate.isTrue(hbaseConfig != null && hbaseConfig.size() !=0, "hbaseConfig不能为空Map结构!"); for (Map.Entry entry : hbaseConfig.entrySet()) { @@ -220,6 +222,16 @@ public void openInternal(InputSplit inputSplit) throws IOException { HbaseInputSplit hbaseInputSplit = (HbaseInputSplit) inputSplit; byte[] startRow = Bytes.toBytesBinary(hbaseInputSplit.getStartkey()); byte[] stopRow = Bytes.toBytesBinary(hbaseInputSplit.getEndKey()); + + if(null == connection || connection.isClosed()){ + try { + connection = ConnectionFactory.createConnection(hConfiguration); + } catch (Exception e) { + HbaseHelper.closeConnection(connection); + throw new IllegalArgumentException(e); + } + } + table = connection.getTable(TableName.valueOf(tableName)); scan = new Scan(); scan.setStartRow(startRow); From 9912f6c5ad743ec8444ed2f35ca4e66e3875a1c4 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 19 Mar 2019 11:12:29 +0800 Subject: [PATCH 28/96] =?UTF-8?q?=E5=BC=82=E5=B8=B8=E5=85=B3=E9=97=AD?= =?UTF-8?q?=E8=BF=9E=E6=8E=A5?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java index 06d2bb1b1b..a47a3a5296 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java @@ -86,8 +86,9 @@ public void configure(Configuration configuration) { try { connection = ConnectionFactory.createConnection(hConfiguration); } catch (Exception e) { - HbaseHelper.closeConnection(connection); throw new IllegalArgumentException(e); + } finally { + HbaseHelper.closeConnection(connection); } LOG.info("HbaseOutputFormat configure end"); @@ -227,8 +228,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { try { connection = ConnectionFactory.createConnection(hConfiguration); } catch (Exception e) { - HbaseHelper.closeConnection(connection); throw new IllegalArgumentException(e); + } finally { + HbaseHelper.closeConnection(connection); } } From 93a11de23b1fe2390fb24b48a0102e9ca57ea4fd Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 19 Mar 2019 11:24:00 +0800 Subject: [PATCH 29/96] =?UTF-8?q?Revert=20"=E5=BC=82=E5=B8=B8=E5=85=B3?= =?UTF-8?q?=E9=97=AD=E8=BF=9E=E6=8E=A5"?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This reverts commit 9912f6c --- .../com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java index a47a3a5296..06d2bb1b1b 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java @@ -86,9 +86,8 @@ public void configure(Configuration configuration) { try { connection = ConnectionFactory.createConnection(hConfiguration); } catch (Exception e) { - throw new IllegalArgumentException(e); - } finally { HbaseHelper.closeConnection(connection); + throw new IllegalArgumentException(e); } LOG.info("HbaseOutputFormat configure end"); @@ -228,9 +227,8 @@ public void openInternal(InputSplit inputSplit) throws IOException { try { connection = ConnectionFactory.createConnection(hConfiguration); } catch (Exception e) { - throw new IllegalArgumentException(e); - } finally { HbaseHelper.closeConnection(connection); + throw new IllegalArgumentException(e); } } From 7a6c710e1a8eeac2caf991ca59a4be057a89b4b7 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 19 Mar 2019 11:41:41 +0800 Subject: [PATCH 30/96] =?UTF-8?q?=E5=8E=BB=E6=8E=89=E9=87=8D=E5=A4=8D?= =?UTF-8?q?=E5=BC=80=E5=90=AF=E7=9A=84ByteLimiter?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java | 5 ----- .../com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java | 5 ----- 2 files changed, 10 deletions(-) diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java index db22d524a6..377989213f 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java @@ -132,11 +132,6 @@ public void openInternal(InputSplit split) throws IOException { br.setFromLine(0); } br.setCharsetName(charsetName); - - if(StringUtils.isNotBlank(monitorUrls) && this.bytes > 0) { - this.byteRateLimiter = new ByteRateLimiter(getRuntimeContext(), monitorUrls, bytes, 1); - this.byteRateLimiter.start(); - } } @Override diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java index f5060b6597..706bf700bc 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java @@ -227,11 +227,6 @@ public void openInternal(InputSplit inputSplit) throws IOException { scan.setCaching(scanCacheSize); scan.setBatch(scanBatchSize); resultScanner = table.getScanner(scan); - - if(StringUtils.isNotBlank(monitorUrls) && this.bytes > 0) { - this.byteRateLimiter = new ByteRateLimiter(getRuntimeContext(), monitorUrls, bytes, 1); - this.byteRateLimiter.start(); - } } @Override From 4535842f98089ecc2e7a89eda5708a8a751fa167 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 19 Mar 2019 16:04:26 +0800 Subject: [PATCH 31/96] =?UTF-8?q?=E6=94=AF=E6=8C=81=E8=87=AA=E5=AE=9A?= =?UTF-8?q?=E4=B9=89sql=E6=9F=A5=E8=AF=A2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flinkx/db2/Db2DatabaseMeta.java | 5 +++ .../flinkx/mysql/MySqlDatabaseMeta.java | 5 +++ .../flinkx/oracle/OracleDatabaseMeta.java | 5 +++ .../postgresql/PostgresqlDatabaseMeta.java | 5 +++ .../dtstack/flinkx/rdb/DatabaseInterface.java | 2 + .../flinkx/rdb/datareader/JdbcConfigKeys.java | 2 + .../flinkx/rdb/datareader/JdbcDataReader.java | 12 ++++- .../rdb/inputformat/JdbcInputFormat.java | 22 +++++++-- .../inputformat/JdbcInputFormatBuilder.java | 4 ++ .../com/dtstack/flinkx/rdb/util/DBUtil.java | 45 ++++++++++++++----- .../sqlserver/SqlServerDatabaseMeta.java | 5 +++ 11 files changed, 97 insertions(+), 15 deletions(-) diff --git a/flinkx-db2/flinkx-db2-core/src/main/java/com/dtstack/flinkx/db2/Db2DatabaseMeta.java b/flinkx-db2/flinkx-db2-core/src/main/java/com/dtstack/flinkx/db2/Db2DatabaseMeta.java index 76f63cd37e..12178d0a16 100644 --- a/flinkx-db2/flinkx-db2-core/src/main/java/com/dtstack/flinkx/db2/Db2DatabaseMeta.java +++ b/flinkx-db2/flinkx-db2-core/src/main/java/com/dtstack/flinkx/db2/Db2DatabaseMeta.java @@ -85,6 +85,11 @@ public String getSplitFilter(String columnName) { return String.format("mod(%s,${N}) = ${M}", getStartQuote() + columnName + getEndQuote()); } + @Override + public String getSplitFilterWithTmpTable(String tmpTable, String columnName){ + return String.format("mod(%s.%s,${N}) = ${M}", tmpTable, getStartQuote() + columnName + getEndQuote()); + } + @Override public EDatabaseType getDatabaseType() { return EDatabaseType.DB2; diff --git a/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java b/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java index 48517f2f24..67c2b2bc66 100644 --- a/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java +++ b/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java @@ -114,6 +114,11 @@ public String getSplitFilter(String columnName) { return String.format("%s mod ${N} = ${M}", getStartQuote() + columnName + getEndQuote()); } + @Override + public String getSplitFilterWithTmpTable(String tmpTable, String columnName){ + return String.format("%s.%s mod ${N} = ${M}", tmpTable, getStartQuote() + columnName + getEndQuote()); + } + @Override public String getMultiInsertStatement(List column, String table, int batchSize) { return "INSERT INTO " + quoteTable(table) diff --git a/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java b/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java index dcd8d25bec..4fecbab2e4 100644 --- a/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java +++ b/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java @@ -76,6 +76,11 @@ public String getSplitFilter(String columnName) { return String.format("mod(%s, ${N}) = ${M}", getStartQuote() + columnName + getEndQuote()); } + @Override + public String getSplitFilterWithTmpTable(String tmpTable, String columnName) { + return String.format("mod(%s.%s, ${N}) = ${M}", tmpTable, getStartQuote() + columnName + getEndQuote()); + } + @Override protected String makeMultipleValues(int nCols, int batchSize) { String value = makeValues(nCols); diff --git a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java index f482c491eb..7b9586af5f 100644 --- a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java +++ b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java @@ -156,6 +156,11 @@ public String getSplitFilter(String columnName) { return String.format(" mod(%s,${N}) = ${M}", getStartQuote() + columnName + getEndQuote()); } + @Override + public String getSplitFilterWithTmpTable(String tmpTable, String columnName) { + return String.format(" mod(%s.%s,${N}) = ${M}", tmpTable, getStartQuote() + columnName + getEndQuote()); + } + @Override public int getFetchSize(){ return 1000; diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/DatabaseInterface.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/DatabaseInterface.java index d2ff546f20..08a5d10b89 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/DatabaseInterface.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/DatabaseInterface.java @@ -67,6 +67,8 @@ public interface DatabaseInterface { String getSplitFilter(String columnName); + String getSplitFilterWithTmpTable(String tmpTable, String columnName); + int getFetchSize(); int getQueryTimeout(); diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java index ca3180e2fe..3779b26c41 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java @@ -44,5 +44,7 @@ public class JdbcConfigKeys { public static final String KEY_START_LOCATION = "startLocation"; + public static final String KEY_CUSTOM_SQL = "customSql"; + public static final String KEY_REALTIME_INCRE_SYNC = "realTimeIncreSync"; } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java index 2b2d233052..a0d0ffd562 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java @@ -72,6 +72,8 @@ public class JdbcDataReader extends DataReader { protected boolean realTimeIncreSync; + protected String customSql; + public void setDatabaseInterface(DatabaseInterface databaseInterface) { this.databaseInterface = databaseInterface; } @@ -97,6 +99,7 @@ public JdbcDataReader(DataTransferConfig config, StreamExecutionEnvironment env) splitKey = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_SPLIK_KEY); increColumn = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_INCRE_COLUMN); startLocation = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_START_LOCATION,null); + customSql = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_CUSTOM_SQL,null); realTimeIncreSync = readerConfig.getParameter().getBooleanVal(JdbcConfigKeys.KEY_REALTIME_INCRE_SYNC,true); realTimeIncreSync = StringUtils.isNotEmpty(increColumn); @@ -125,9 +128,16 @@ public DataStream readData() { builder.setSplitKey(splitKey); builder.setNumPartitions(numPartitions); builder.setRealTimeIncreSync(realTimeIncreSync); + builder.setCustomSql(customSql); boolean isSplitByKey = numPartitions > 1 && StringUtils.isNotEmpty(splitKey); - String query = DBUtil.getQuerySql(databaseInterface, table, metaColumns, splitKey, where, isSplitByKey, realTimeIncreSync); + + String query; + if (StringUtils.isNotEmpty(customSql)){ + query = DBUtil.buildQuerySqlWithCustomSql(databaseInterface, customSql, isSplitByKey, splitKey, realTimeIncreSync); + } else { + query = DBUtil.getQuerySql(databaseInterface, table, metaColumns, splitKey, where, isSplitByKey, realTimeIncreSync); + } builder.setQuery(query); RichInputFormat format = builder.finish(); diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 8e2fef9189..06a387673b 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -112,6 +112,8 @@ public class JdbcInputFormat extends RichInputFormat { protected int numPartitions; + protected String customSql; + protected StringAccumulator tableColAccumulator; protected StringAccumulator maxValueAccumulator; @@ -311,6 +313,10 @@ private void getMaxValue(InputSplit inputSplit){ maxValueAccumulator.add(maxValue); getRuntimeContext().addAccumulator(Metrics.MAX_VALUE, maxValueAccumulator); } else { + if(StringUtils.isEmpty(monitorUrls)){ + return; + } + Map vars = getRuntimeContext().getMetricGroup().getAllVariables(); String jobId = vars.get(""); @@ -403,7 +409,7 @@ private String buildQuerySql(InputSplit inputSplit){ if (realTimeIncreSync){ String incrementFilter = DBUtil.buildIncrementFilter(databaseInterface, increColType, increCol, - jdbcInputSplit.getStartLocation(), jdbcInputSplit.getEndLocation()); + jdbcInputSplit.getStartLocation(), jdbcInputSplit.getEndLocation(), customSql); if(StringUtils.isNotEmpty(incrementFilter)){ incrementFilter = " and " + incrementFilter; @@ -426,9 +432,17 @@ private String getMaxValueFromDb() { try { long startTime = System.currentTimeMillis(); - String queryMaxValueSql = String.format("select max(%s) as max_value from %s", - databaseInterface.quoteColumn(increCol), databaseInterface.quoteTable(table)); - String startSql = DBUtil.buildStartLocationSql(databaseInterface, increColType, increCol, startLocation); + String queryMaxValueSql; + if (StringUtils.isNotEmpty(customSql)){ + queryMaxValueSql = String.format("select max(%s.%s) as max_value from ( %s ) %s", DBUtil.TEMPORARY_TABLE_NAME, + databaseInterface.quoteColumn(increCol), customSql, DBUtil.TEMPORARY_TABLE_NAME); + } else { + queryMaxValueSql = String.format("select max(%s) as max_value from %s", + databaseInterface.quoteColumn(increCol), databaseInterface.quoteTable(table)); + } + + String startSql = DBUtil.buildStartLocationSql(databaseInterface, increColType, + databaseInterface.quoteColumn(increCol), startLocation); if(StringUtils.isNotEmpty(startSql)){ queryMaxValueSql += " where " + startSql; } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java index 240c3116d9..8af8eaa7c6 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java @@ -111,6 +111,10 @@ public void setNumPartitions(int numPartitions){ format.numPartitions = numPartitions; } + public void setCustomSql(String customSql){ + format.customSql = customSql; + } + @Override protected void checkFormat() { diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java index 5c89bb0173..3b5cdbad99 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java @@ -57,6 +57,10 @@ public class DBUtil { public static final String INCREMENT_FILTER_PLACEHOLDER = "${incrementFilter}"; + public static final String TEMPORARY_TABLE_NAME = "flinkx_tmp"; + + public static final String CUSTOM_SQL_TEMPLATE = "select * from (%s) %s"; + private static Connection getConnectionInternal(String url, String username, String password) throws SQLException { Connection dbConn; synchronized (ClassUtil.lock_str){ @@ -170,7 +174,7 @@ public static void closeDBResources(ResultSet rs, Statement stmt, public static void commit(Connection conn){ try { - if (!conn.getAutoCommit() && !conn.isClosed()){ + if (!conn.isClosed() && !conn.getAutoCommit()){ LOG.info("Start commit connection"); conn.commit(); LOG.info("Commit connection successful"); @@ -362,15 +366,19 @@ public static Object clobToString(Object obj) throws Exception{ } public static String buildIncrementFilter(DatabaseInterface databaseInterface,String increColType,String increCol, - String startLocation,String endLocation){ + String startLocation,String endLocation, String customSql){ StringBuilder filter = new StringBuilder(); - String startFilter = buildStartLocationSql(databaseInterface,increColType,increCol,startLocation); + if (StringUtils.isNotEmpty(customSql)){ + increCol = String.format("%s.%s", TEMPORARY_TABLE_NAME, databaseInterface.quoteColumn(increCol)); + } + + String startFilter = buildStartLocationSql(databaseInterface, increColType, increCol, startLocation); if (StringUtils.isNotEmpty(startFilter)){ filter.append(startFilter); } - String endFilter = buildEndLocationSql(databaseInterface,increColType,increCol,endLocation); + String endFilter = buildEndLocationSql(databaseInterface, increColType, increCol, endLocation); if (StringUtils.isNotEmpty(endFilter)){ if (filter.length() > 0){ filter.append(" and ").append(endFilter); @@ -400,12 +408,12 @@ public static String buildEndLocationSql(DatabaseInterface databaseInterface,Str endTimeStr = String.format("'%s'",endTimeStr); } - endLocationSql = databaseInterface.quoteColumn(increCol) + " < " + endTimeStr; + endLocationSql = increCol + " < " + endTimeStr; } else if(ColumnType.isNumberType(increColType)){ - endLocationSql = databaseInterface.quoteColumn(increCol) + " < " + endLocation; + endLocationSql = increCol + " < " + endLocation; } else { endTimeStr = String.format("'%s'",endLocation); - endLocationSql = databaseInterface.quoteColumn(increCol) + " < " + endTimeStr; + endLocationSql = increCol + " < " + endTimeStr; } return endLocationSql; @@ -429,12 +437,12 @@ public static String buildStartLocationSql(DatabaseInterface databaseInterface,S startTimeStr = String.format("'%s'",startTimeStr); } - startLocationSql = databaseInterface.quoteColumn(increCol) + " >= " + startTimeStr; + startLocationSql = increCol + " >= " + startTimeStr; } else if(ColumnType.isNumberType(increColType)){ - startLocationSql = databaseInterface.quoteColumn(increCol) + " >= " + startLocation; + startLocationSql = increCol + " >= " + startLocation; } else { startTimeStr = String.format("'%s'",startLocation); - startLocationSql = databaseInterface.quoteColumn(increCol) + " >= " + startTimeStr; + startLocationSql = increCol + " >= " + startTimeStr; } return startLocationSql; @@ -523,6 +531,23 @@ public static long getMillis(long startLocation){ return millisSecond; } + public static String buildQuerySqlWithCustomSql(DatabaseInterface databaseInterface,String customSql, + boolean isSplitByKey,String splitKey,boolean realTimeIncreSync){ + StringBuilder querySql = new StringBuilder(); + querySql.append(String.format(CUSTOM_SQL_TEMPLATE, customSql, TEMPORARY_TABLE_NAME)); + querySql.append(" WHERE 1=1 "); + + if (isSplitByKey){ + querySql.append(" And ").append(databaseInterface.getSplitFilterWithTmpTable(TEMPORARY_TABLE_NAME, splitKey)); + } + + if (realTimeIncreSync){ + querySql.append(" ").append(INCREMENT_FILTER_PLACEHOLDER); + } + + return querySql.toString(); + } + public static String getQuerySql(DatabaseInterface databaseInterface,String table,List metaColumns, String splitKey,String customFilter,boolean isSplitByKey){ return getQuerySql(databaseInterface, table, metaColumns, splitKey, customFilter, isSplitByKey, false); diff --git a/flinkx-sqlserver/flinkx-sqlserver-core/src/main/java/com/dtstack/flinkx/sqlserver/SqlServerDatabaseMeta.java b/flinkx-sqlserver/flinkx-sqlserver-core/src/main/java/com/dtstack/flinkx/sqlserver/SqlServerDatabaseMeta.java index d8b6c5aea2..7e6425ac56 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-core/src/main/java/com/dtstack/flinkx/sqlserver/SqlServerDatabaseMeta.java +++ b/flinkx-sqlserver/flinkx-sqlserver-core/src/main/java/com/dtstack/flinkx/sqlserver/SqlServerDatabaseMeta.java @@ -64,6 +64,11 @@ public String getSplitFilter(String columnName) { return String.format("%s %% ${N} = ${M}", getStartQuote() + columnName + getEndQuote()); } + @Override + public String getSplitFilterWithTmpTable(String tmpTable, String columnName) { + return String.format("%s.%s %% ${N} = ${M}", tmpTable, getStartQuote() + columnName + getEndQuote()); + } + @Override protected String makeMultipleValues(int nCols, int batchSize) { String value = makeValues(nCols); From 844a0dfeefe9c708a266dd15fdd1072b3472e334 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Tue, 19 Mar 2019 16:34:13 +0800 Subject: [PATCH 32/96] add telnet retry --- .../com/dtstack/flinkx/util/TelnetUtil.java | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/TelnetUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/TelnetUtil.java index 51226c83cf..9ab95fcd52 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/TelnetUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/TelnetUtil.java @@ -2,6 +2,7 @@ import org.apache.commons.net.telnet.TelnetClient; +import java.util.concurrent.Callable; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -18,7 +19,22 @@ public static void telnet(String ip,int port) { client.setConnectTimeout(3000); client.connect(ip,port); } catch (Exception e){ - throw new RuntimeException("Unable connect to : " + ip + ":" + port); + TelnetClient finalClient = client; + boolean flag = false; + try { + flag = RetryUtil.executeWithRetry(new Callable() { + @Override + public Boolean call() throws Exception { + finalClient.connect(ip,port); + return finalClient.isAvailable(); + } + }, 4,2000,false); + } catch (Exception e1) { + e1.printStackTrace(); + } + if (!flag){ + throw new RuntimeException("Unable connect to : " + ip + ":" + port); + } } finally { try { if (client != null){ From edd45de8e4778231966fd9811e1173e4a798da53 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Tue, 19 Mar 2019 17:32:48 +0800 Subject: [PATCH 33/96] add telnet retry --- .../com/dtstack/flinkx/util/TelnetUtil.java | 50 +++++++++---------- 1 file changed, 23 insertions(+), 27 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/TelnetUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/TelnetUtil.java index 9ab95fcd52..e0ee5e0edd 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/TelnetUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/TelnetUtil.java @@ -13,36 +13,32 @@ public class TelnetUtil { private static final String PORT_KEY = "port"; public static void telnet(String ip,int port) { - TelnetClient client = null; - try{ - client = new TelnetClient(); - client.setConnectTimeout(3000); - client.connect(ip,port); - } catch (Exception e){ - TelnetClient finalClient = client; - boolean flag = false; - try { - flag = RetryUtil.executeWithRetry(new Callable() { - @Override - public Boolean call() throws Exception { - finalClient.connect(ip,port); - return finalClient.isAvailable(); + try { + RetryUtil.executeWithRetry(new Callable() { + @Override + public Boolean call() throws Exception { + TelnetClient client = null; + try{ + client = new TelnetClient(); + client.setConnectTimeout(3000); + client.connect(ip,port); + } catch (Exception e){ + throw new RuntimeException("Unable connect to : " + ip + ":" + port); + } finally { + try { + if (client != null){ + client.disconnect(); + } + } catch (Exception ignore){ + } } - }, 4,2000,false); - } catch (Exception e1) { - e1.printStackTrace(); - } - if (!flag){ - throw new RuntimeException("Unable connect to : " + ip + ":" + port); - } - } finally { - try { - if (client != null){ - client.disconnect(); + return null; } - } catch (Exception ignore){ - } + }, 3,1000,false); + } catch (Exception e) { + e.printStackTrace(); } + } public static void telnet(String url) { From 84ead164be66481f02ac2c537b119cfd7f5e8820 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 19 Mar 2019 19:24:02 +0800 Subject: [PATCH 34/96] =?UTF-8?q?es=E6=89=B9=E9=87=8F=E5=86=99=E5=85=A5bug?= =?UTF-8?q?=E5=92=8Chdfs=20orc=E8=AF=BB=E5=8F=96=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flinkx/es/writer/EsOutputFormat.java | 2 +- .../com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java b/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java index 28013af32f..6a757c8798 100644 --- a/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java +++ b/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java @@ -91,8 +91,8 @@ protected void writeMultipleRecordsInternal() throws Exception { IndexRequest request = StringUtils.isBlank(id) ? new IndexRequest(index, type) : new IndexRequest(index, type, id); request = request.source(EsUtil.rowToJsonMap(row, columnNames, columnTypes)); bulkRequest.add(request); - client.bulk(bulkRequest); } + client.bulk(bulkRequest); } @Override diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java index d8c1a64ea4..4a0ac66695 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java @@ -208,10 +208,6 @@ public Row nextRecordInternal(Row row) throws IOException { val = HdfsUtil.string2col(String.valueOf(val),metaColumn.getType(),metaColumn.getTimeFormat()); } - if (val != null) { - val = HdfsUtil.getWritableValue(val); - } - row.setField(i,val); } } From d36d04b00646228af2cdd08307973f6a41e7e2c9 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 19 Mar 2019 20:50:46 +0800 Subject: [PATCH 35/96] =?UTF-8?q?hbaseConfig=E7=B1=BB=E5=BA=8F=E5=88=97?= =?UTF-8?q?=E5=8C=96=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/hbase/reader/HbaseInputFormat.java | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java index 06d2bb1b1b..fa63048622 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java @@ -70,21 +70,12 @@ public class HbaseInputFormat extends RichInputFormat { private transient ResultScanner resultScanner; private transient Result next; - private org.apache.hadoop.conf.Configuration hConfiguration; - @Override public void configure(Configuration configuration) { LOG.info("HbaseOutputFormat configure start"); - hConfiguration = new org.apache.hadoop.conf.Configuration(); - Validate.isTrue(hbaseConfig != null && hbaseConfig.size() !=0, "hbaseConfig不能为空Map结构!"); - - for (Map.Entry entry : hbaseConfig.entrySet()) { - hConfiguration.set(entry.getKey(), entry.getValue()); - } - try { - connection = ConnectionFactory.createConnection(hConfiguration); + connection = ConnectionFactory.createConnection(getConfig()); } catch (Exception e) { HbaseHelper.closeConnection(connection); throw new IllegalArgumentException(e); @@ -93,6 +84,17 @@ public void configure(Configuration configuration) { LOG.info("HbaseOutputFormat configure end"); } + public org.apache.hadoop.conf.Configuration getConfig(){ + org.apache.hadoop.conf.Configuration hConfiguration = new org.apache.hadoop.conf.Configuration(); + Validate.isTrue(hbaseConfig != null && hbaseConfig.size() !=0, "hbaseConfig不能为空Map结构!"); + + for (Map.Entry entry : hbaseConfig.entrySet()) { + hConfiguration.set(entry.getKey(), entry.getValue()); + } + + return hConfiguration; + } + @Override public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOException { return null; @@ -225,7 +227,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { if(null == connection || connection.isClosed()){ try { - connection = ConnectionFactory.createConnection(hConfiguration); + connection = ConnectionFactory.createConnection(getConfig()); } catch (Exception e) { HbaseHelper.closeConnection(connection); throw new IllegalArgumentException(e); @@ -276,7 +278,6 @@ public Row nextRecordInternal(Row row) throws IOException { String family = arr[0].trim(); String qualifier = arr[1].trim(); bytes = next.getValue(family.getBytes(), qualifier.getBytes()); - //col = String.valueOf(bytes); } col = convertBytesToAssignType(columnType, bytes, columnFormat); } From 3fdcedec6c79addb086fac0961b7325d52564278 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 20 Mar 2019 10:14:12 +0800 Subject: [PATCH 36/96] =?UTF-8?q?=E5=87=8F=E5=B0=91odps=E7=9A=84=E9=87=8D?= =?UTF-8?q?=E8=AF=95=E6=AC=A1=E6=95=B0=E4=B8=BA3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java index dd977ca179..a6f458c1ee 100644 --- a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java +++ b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java @@ -54,7 +54,7 @@ public class OdpsUtil { private static final Logger LOG = LoggerFactory.getLogger(OdpsUtil.class); - public static int MAX_RETRY_TIME = 10; + public static int MAX_RETRY_TIME = 3; public static Odps initOdps(Map odpsConfig) { String odpsServer = odpsConfig.get(OdpsConfigKeys.KEY_ODPS_SERVER); From 1a72f6b890916c97625640e76dc05e0e98a4419c Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 20 Mar 2019 21:19:05 +0800 Subject: [PATCH 37/96] =?UTF-8?q?=E4=BF=AE=E5=A4=8Dmongodb=E7=9B=B8?= =?UTF-8?q?=E5=85=B3bug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/mongodb/MongodbConfigKeys.java | 14 ++- .../dtstack/flinkx/mongodb/MongodbUtil.java | 107 +++++++++--------- .../mongodb/reader/MongodbInputFormat.java | 65 ++++++----- .../reader/MongodbInputFormatBuilder.java | 9 ++ .../flinkx/mongodb/reader/MongodbReader.java | 16 ++- .../mongodb/writer/MongodbOutputFormat.java | 26 ++--- .../writer/MongodbOutputFormatBuilder.java | 6 + .../flinkx/mongodb/writer/MongodbWriter.java | 14 ++- 8 files changed, 159 insertions(+), 98 deletions(-) diff --git a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfigKeys.java b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfigKeys.java index 3e3b462399..1f5617b915 100644 --- a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfigKeys.java +++ b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfigKeys.java @@ -38,13 +38,21 @@ public class MongodbConfigKeys { public final static String KEY_FILTER = "filter"; + public final static String KEY_FETCH_SIZE = "fetchSize"; + public final static String KEY_MODE = "writeMode"; public final static String KEY_REPLACE_KEY = "replaceKey"; - public final static String KEY_NAME = "name"; + public final static String KEY_MONGODB_CONFIG = "mongodbConfig"; + + public final static String KEY_CONNECTIONS_PERHOST = "connectionsPerHost"; + + public final static String KEY_THREADS_FOR_CONNECTION_MULTIPLIER = "threadsForConnectionMultiplier"; + + public final static String KEY_CONNECTION_TIMEOUT = "connectionTimeout"; - public final static String KEY_TYPE = "type"; + public final static String KEY_MAX_WAIT_TIME = "maxWaitTime"; - public final static String KEY_SPLITTER = "splitter"; + public final static String KEY_SOCKET_TIMEOUT = "socketTimeout"; } diff --git a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java index a8d10ba727..2ca8502fca 100644 --- a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java +++ b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java @@ -25,8 +25,8 @@ import com.dtstack.flinkx.util.TelnetUtil; import com.google.common.collect.Lists; import com.mongodb.*; -import com.mongodb.client.MongoCollection; -import com.mongodb.client.MongoDatabase; +import com.mongodb.client.MongoCursor; +import org.apache.commons.collections.MapUtils; import org.apache.commons.lang.StringUtils; import org.apache.flink.types.Row; import org.bson.Document; @@ -60,69 +60,58 @@ public class MongodbUtil { private static final Integer DEFAULT_PORT = 27017; - private static final Integer ONE_SECOND = 1000; + private static final Integer DEFAULT_CONNECTIONS_PER_HOST = 100; - private static final Integer CONNECTIONS_PER_HOST = 100; + private static final Integer DEFAULT_THREADS_FOR_CONNECTION_MULTIPLIER = 100; - private static final Integer THREADS_FOR_CONNECTION_MULTIPLIER = 100; + private static final Integer DEFAULT_CONNECT_TIMEOUT = 10 * 1000; - private static final Integer CONNECT_TIMEOUT = 10 * ONE_SECOND; + private static final Integer DEFAULT_MAX_WAIT_TIME = 5 * 1000; - private static final Integer MAX_WAIT_TIME = 5 * ONE_SECOND; - - private static final Integer SOCKET_TIMEOUT = 0; - - private static MongoClient mongoClient; + private static final Integer DEFAULT_SOCKET_TIMEOUT = 0; /** * Get mongo client - * @param config + * @param mongodbConfig * @return MongoClient */ - public static MongoClient getMongoClient(Map config){ + public static MongoClient getMongoClient(Map mongodbConfig){ + MongoClient mongoClient; try{ - if(mongoClient == null){ - MongoClientOptions options = getOption(); - List serverAddress = getServerAddress(config.get(KEY_HOST_PORTS)); - String username = config.get(KEY_USERNAME); - String password = config.get(KEY_PASSWORD); - String database = config.get(KEY_DATABASE); - - if(StringUtils.isEmpty(username)){ - mongoClient = new MongoClient(serverAddress,options); - } else { - MongoCredential credential = MongoCredential.createScramSha1Credential(username, database, password.toCharArray()); - List credentials = Lists.newArrayList(); - credentials.add(credential); - - mongoClient = new MongoClient(serverAddress,credentials,options); - } - - - LOG.info("mongo客户端获取成功"); + MongoClientOptions options = getOption(mongodbConfig); + List serverAddress = getServerAddress(MapUtils.getString(mongodbConfig, KEY_HOST_PORTS)); + String username = MapUtils.getString(mongodbConfig, KEY_USERNAME); + String password = MapUtils.getString(mongodbConfig, KEY_PASSWORD); + String database = MapUtils.getString(mongodbConfig, KEY_DATABASE); + + if(StringUtils.isEmpty(username)){ + mongoClient = new MongoClient(serverAddress,options); + } else { + MongoCredential credential = MongoCredential.createScramSha1Credential(username, database, password.toCharArray()); + List credentials = Lists.newArrayList(); + credentials.add(credential); + + mongoClient = new MongoClient(serverAddress,credentials,options); } + + LOG.info("Get mongodb client successful"); return mongoClient; }catch (Exception e){ throw new RuntimeException(e); } } - public static MongoDatabase getDatabase(Map config,String database){ - MongoClient client = getMongoClient(config); - return mongoClient.getDatabase(database); - } - - public static MongoCollection getCollection(Map config,String database, String collection){ - MongoClient client = getMongoClient(config); - MongoDatabase db = client.getDatabase(database); - - return db.getCollection(collection); - } + public static void close(MongoClient mongoClient, MongoCursor cursor){ + if (cursor != null){ + LOG.info("Start close mongodb cursor"); + cursor.close(); + LOG.info("Close mongodb cursor successfully"); + } - public static void close(){ if (mongoClient != null){ + LOG.info("Start close mongodb client"); mongoClient.close(); - mongoClient = null; + LOG.info("Close mongodb client successfully"); } } @@ -181,13 +170,29 @@ private static List getServerAddress(String hostPorts) { return addresses; } - private static MongoClientOptions getOption(){ + private static MongoClientOptions getOption(Map mongodbConfig){ MongoClientOptions.Builder build = new MongoClientOptions.Builder(); - build.connectionsPerHost(CONNECTIONS_PER_HOST); - build.threadsAllowedToBlockForConnectionMultiplier(THREADS_FOR_CONNECTION_MULTIPLIER); - build.connectTimeout(CONNECT_TIMEOUT); - build.maxWaitTime(MAX_WAIT_TIME); - build.socketTimeout(SOCKET_TIMEOUT); + + int connectionsPerHost = MapUtils.getIntValue(mongodbConfig, KEY_CONNECTIONS_PERHOST, DEFAULT_CONNECTIONS_PER_HOST); + LOG.info("Mongodb config -- connectionsPerHost:" + connectionsPerHost); + build.connectionsPerHost(connectionsPerHost); + + int threadsForConnectionMultiplier = MapUtils.getIntValue(mongodbConfig, KEY_THREADS_FOR_CONNECTION_MULTIPLIER, DEFAULT_THREADS_FOR_CONNECTION_MULTIPLIER); + LOG.info("Mongodb config -- threadsForConnectionMultiplier:" + threadsForConnectionMultiplier); + build.threadsAllowedToBlockForConnectionMultiplier(threadsForConnectionMultiplier); + + int connectionTimeout = MapUtils.getIntValue(mongodbConfig, KEY_CONNECTION_TIMEOUT, DEFAULT_CONNECT_TIMEOUT); + LOG.info("Mongodb config -- connectionTimeout:" + connectionTimeout); + build.connectTimeout(connectionTimeout); + + int maxWaitTime = MapUtils.getIntValue(mongodbConfig, KEY_MAX_WAIT_TIME, DEFAULT_MAX_WAIT_TIME); + LOG.info("Mongodb config -- maxWaitTime:" + maxWaitTime); + build.maxWaitTime(maxWaitTime); + + int socketTimeout = MapUtils.getIntValue(mongodbConfig, KEY_SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT); + LOG.info("Mongodb config -- socketTimeout:" + socketTimeout); + build.maxWaitTime(socketTimeout); + build.writeConcern(WriteConcern.UNACKNOWLEDGED); return build.build(); } diff --git a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java index 50847d4f79..2a1beeed2f 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java +++ b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java @@ -23,9 +23,11 @@ import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.StringUtil; import com.mongodb.BasicDBObject; +import com.mongodb.MongoClient; import com.mongodb.client.FindIterable; import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoCursor; +import com.mongodb.client.MongoDatabase; import org.apache.commons.lang.StringUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.InputSplit; @@ -36,8 +38,6 @@ import java.io.IOException; import java.util.*; -import static com.dtstack.flinkx.mongodb.MongodbConfigKeys.*; - /** * Read plugin for reading static data * @@ -60,21 +60,18 @@ public class MongodbInputFormat extends RichInputFormat { protected String filterJson; - private Bson filter; + protected Map mongodbConfig; - private transient MongoCollection collection; + protected int fetchSize; + + private Bson filter; private transient MongoCursor cursor; + private transient MongoClient client; + @Override public void configure(Configuration parameters) { - Map config = new HashMap<>(4); - config.put(KEY_HOST_PORTS,hostPorts); - config.put(KEY_USERNAME,username); - config.put(KEY_PASSWORD,password); - config.put(KEY_DATABASE,database); - - collection = MongodbUtil.getCollection(config,database,collectionName); buildFilter(); } @@ -83,13 +80,19 @@ protected void openInternal(InputSplit inputSplit) throws IOException { MongodbInputSplit split = (MongodbInputSplit) inputSplit; FindIterable findIterable; + client = MongodbUtil.getMongoClient(mongodbConfig); + MongoDatabase db = client.getDatabase(database); + MongoCollection collection = db.getCollection(collectionName); + if(filter == null){ findIterable = collection.find(); } else { findIterable = collection.find(filter); } - findIterable = findIterable.skip(split.getSkip()).limit(split.getLimit()); + findIterable = findIterable.skip(split.getSkip()) + .limit(split.getLimit()) + .batchSize(fetchSize); cursor = findIterable.iterator(); } @@ -130,29 +133,37 @@ public Row nextRecordInternal(Row row) throws IOException { @Override protected void closeInternal() throws IOException { - if (cursor != null){ - cursor.close(); - MongodbUtil.close(); - } + MongodbUtil.close(client, cursor); } @Override public InputSplit[] createInputSplits(int minNumSplits) throws IOException { ArrayList splits = new ArrayList<>(); - long docNum = filter == null ? collection.count() : collection.count(filter); - if(docNum <= minNumSplits){ - splits.add(new MongodbInputSplit(0,(int)docNum)); - return splits.toArray(new MongodbInputSplit[splits.size()]); - } + MongoClient client = null; + try { + client = MongodbUtil.getMongoClient(mongodbConfig); + MongoDatabase db = client.getDatabase(database); + MongoCollection collection = db.getCollection(collectionName); - long size = Math.floorDiv(docNum,(long)minNumSplits); - for (int i = 0; i < minNumSplits; i++) { - splits.add(new MongodbInputSplit((int)(i * size), (int)size)); - } + long docNum = filter == null ? collection.count() : collection.count(filter); + if(docNum <= minNumSplits){ + splits.add(new MongodbInputSplit(0,(int)docNum)); + return splits.toArray(new MongodbInputSplit[splits.size()]); + } - if(size * minNumSplits < docNum){ - splits.add(new MongodbInputSplit((int)(size * minNumSplits), (int)(docNum - size * minNumSplits))); + long size = Math.floorDiv(docNum,(long)minNumSplits); + for (int i = 0; i < minNumSplits; i++) { + splits.add(new MongodbInputSplit((int)(i * size), (int)size)); + } + + if(size * minNumSplits < docNum){ + splits.add(new MongodbInputSplit((int)(size * minNumSplits), (int)(docNum - size * minNumSplits))); + } + } catch (Exception e){ + LOG.error("{}", e); + } finally { + MongodbUtil.close(client, null); } return splits.toArray(new MongodbInputSplit[splits.size()]); diff --git a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java index 2c73cedb7b..b97fcd2116 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java +++ b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java @@ -22,6 +22,7 @@ import com.dtstack.flinkx.reader.MetaColumn; import java.util.List; +import java.util.Map; /** * The builder for mongodb reader plugin @@ -61,6 +62,14 @@ public void setMetaColumns(List metaColumns){ format.metaColumns = metaColumns; } + public void setMongodbConfig(Map mongodbConfig){ + format.mongodbConfig = mongodbConfig; + } + + public void setFetchSize(int fetchSize){ + format.fetchSize = fetchSize; + } + public void setFilter(String filter){ format.filterJson = filter; } diff --git a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbReader.java b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbReader.java index b781d01127..0cec127760 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbReader.java +++ b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbReader.java @@ -26,7 +26,9 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static com.dtstack.flinkx.mongodb.MongodbConfigKeys.*; @@ -52,6 +54,10 @@ public class MongodbReader extends DataReader { protected String filter; + protected Map mongodbConfig; + + protected int fetchSize; + public MongodbReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); @@ -62,13 +68,19 @@ public MongodbReader(DataTransferConfig config, StreamExecutionEnvironment env) database = readerConfig.getParameter().getStringVal(KEY_DATABASE); collection = readerConfig.getParameter().getStringVal(KEY_COLLECTION); filter = readerConfig.getParameter().getStringVal(KEY_FILTER); + fetchSize = readerConfig.getParameter().getIntVal(KEY_FETCH_SIZE, 0); metaColumns = MetaColumn.getMetaColumns(readerConfig.getParameter().getColumn()); + + mongodbConfig = (Map)readerConfig.getParameter().getVal(KEY_MONGODB_CONFIG, new HashMap<>()); + mongodbConfig.put(KEY_HOST_PORTS, hostPorts); + mongodbConfig.put(KEY_USERNAME, username); + mongodbConfig.put(KEY_PASSWORD, password); + mongodbConfig.put(KEY_DATABASE, database); } @Override public DataStream readData() { MongodbInputFormatBuilder builder = new MongodbInputFormatBuilder(); - builder.setHostPorts(hostPorts); builder.setUsername(username); builder.setPassword(password); @@ -76,6 +88,8 @@ public DataStream readData() { builder.setCollection(collection); builder.setFilter(filter); builder.setMetaColumns(metaColumns); + builder.setMongodbConfig(mongodbConfig); + builder.setFetchSize(fetchSize); builder.setMonitorUrls(monitorUrls); builder.setBytes(bytes); diff --git a/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormat.java b/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormat.java index 6d2859902e..9fb3fd6273 100644 --- a/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormat.java +++ b/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormat.java @@ -23,7 +23,9 @@ import com.dtstack.flinkx.outputformat.RichOutputFormat; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.writer.WriteMode; +import com.mongodb.MongoClient; import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; import org.apache.commons.lang.StringUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; @@ -31,12 +33,9 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.mongodb.MongodbConfigKeys.*; - /** * OutputFormat for mongodb writer plugin * @@ -59,26 +58,24 @@ public class MongodbOutputFormat extends RichOutputFormat { protected String replaceKey; - protected String mode = WriteMode.INSERT.getMode(); + protected String mode; private transient MongoCollection collection; + private transient MongoClient client; + + protected Map mongodbConfig; + @Override public void configure(Configuration parameters) { - super.configure(parameters); - Map config = new HashMap<>(4); - config.put(KEY_HOST_PORTS,hostPorts); - config.put(KEY_USERNAME,username); - config.put(KEY_PASSWORD,password); - config.put(KEY_DATABASE,database); - - collection = MongodbUtil.getCollection(config,database,collectionName); } @Override protected void openInternal(int taskNumber, int numTasks) throws IOException { - + client = MongodbUtil.getMongoClient(mongodbConfig); + MongoDatabase db = client.getDatabase(database); + collection = db.getCollection(collectionName); } @Override @@ -119,7 +116,6 @@ protected void writeMultipleRecordsInternal() throws Exception { @Override public void closeInternal() throws IOException { - super.closeInternal(); - MongodbUtil.close(); + MongodbUtil.close(client, null); } } diff --git a/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormatBuilder.java b/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormatBuilder.java index 3790536adf..e382e72816 100644 --- a/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormatBuilder.java +++ b/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormatBuilder.java @@ -22,6 +22,7 @@ import com.dtstack.flinkx.reader.MetaColumn; import java.util.List; +import java.util.Map; /** @@ -70,6 +71,11 @@ public void setReplaceKey(String replaceKey){ format.replaceKey = replaceKey; } + + public void setMongodbConfig(Map mongodbConfig){ + format.mongodbConfig = mongodbConfig; + } + @Override protected void checkFormat() { if(format.hostPorts == null){ diff --git a/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbWriter.java b/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbWriter.java index 0ea3841899..4cf2068457 100644 --- a/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbWriter.java +++ b/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbWriter.java @@ -22,12 +22,15 @@ import com.dtstack.flinkx.config.WriterConfig; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.writer.DataWriter; +import com.dtstack.flinkx.writer.WriteMode; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; import org.apache.flink.types.Row; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static com.dtstack.flinkx.mongodb.MongodbConfigKeys.*; import static com.dtstack.flinkx.mongodb.MongodbConfigKeys.KEY_COLLECTION; @@ -54,6 +57,8 @@ public class MongodbWriter extends DataWriter { protected String replaceKey; + protected Map mongodbConfig; + public MongodbWriter(DataTransferConfig config) { super(config); @@ -63,10 +68,16 @@ public MongodbWriter(DataTransferConfig config) { password = writerConfig.getParameter().getStringVal(KEY_PASSWORD); database = writerConfig.getParameter().getStringVal(KEY_DATABASE); collection = writerConfig.getParameter().getStringVal(KEY_COLLECTION); - mode = writerConfig.getParameter().getStringVal(KEY_MODE); + mode = writerConfig.getParameter().getStringVal(KEY_MODE, WriteMode.INSERT.getMode()); replaceKey = writerConfig.getParameter().getStringVal(KEY_REPLACE_KEY); columns = MetaColumn.getMetaColumns(writerConfig.getParameter().getColumn()); + + mongodbConfig = (Map)writerConfig.getParameter().getVal(KEY_MONGODB_CONFIG, new HashMap<>()); + mongodbConfig.put(KEY_HOST_PORTS, hostPorts); + mongodbConfig.put(KEY_USERNAME, username); + mongodbConfig.put(KEY_PASSWORD, password); + mongodbConfig.put(KEY_DATABASE, database); } @Override @@ -81,6 +92,7 @@ public DataStreamSink writeData(DataStream dataSet) { builder.setMode(mode); builder.setColumns(columns); builder.setReplaceKey(replaceKey); + builder.setMongodbConfig(mongodbConfig); builder.setMonitorUrls(monitorUrls); builder.setErrors(errors); From 149aa2919808716c460cf6dee04ebfa1d7bf7f03 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 20 Mar 2019 21:24:30 +0800 Subject: [PATCH 38/96] =?UTF-8?q?fetchSize=E9=BB=98=E8=AE=A4=E5=80=BC?= =?UTF-8?q?=E8=AE=BE=E4=B8=BA100?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flinkx/mongodb/reader/MongodbReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbReader.java b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbReader.java index 0cec127760..2a32a73aa1 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbReader.java +++ b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbReader.java @@ -68,7 +68,7 @@ public MongodbReader(DataTransferConfig config, StreamExecutionEnvironment env) database = readerConfig.getParameter().getStringVal(KEY_DATABASE); collection = readerConfig.getParameter().getStringVal(KEY_COLLECTION); filter = readerConfig.getParameter().getStringVal(KEY_FILTER); - fetchSize = readerConfig.getParameter().getIntVal(KEY_FETCH_SIZE, 0); + fetchSize = readerConfig.getParameter().getIntVal(KEY_FETCH_SIZE, 100); metaColumns = MetaColumn.getMetaColumns(readerConfig.getParameter().getColumn()); mongodbConfig = (Map)readerConfig.getParameter().getVal(KEY_MONGODB_CONFIG, new HashMap<>()); From 84304862c26906051daa6a3e51ec25e15901c262 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Thu, 21 Mar 2019 10:52:51 +0800 Subject: [PATCH 39/96] =?UTF-8?q?bufferSize=E5=8F=82=E6=95=B0=E6=8F=90?= =?UTF-8?q?=E5=87=BA?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flinkx/odps/OdpsConfigKeys.java | 2 ++ .../main/java/com/dtstack/flinkx/odps/OdpsUtil.java | 2 ++ .../dtstack/flinkx/odps/writer/OdpsOutputFormat.java | 5 +++-- .../flinkx/odps/writer/OdpsOutputFormatBuilder.java | 4 ++++ .../com/dtstack/flinkx/odps/writer/OdpsWriter.java | 10 ++++++++++ 5 files changed, 21 insertions(+), 2 deletions(-) diff --git a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsConfigKeys.java b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsConfigKeys.java index 49b68fbdd8..580996b24b 100755 --- a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsConfigKeys.java +++ b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsConfigKeys.java @@ -54,4 +54,6 @@ public class OdpsConfigKeys { public static final String KEY_MODE = "mode"; + public static final String KEY_BUFFER_SIZE = "bufferSize"; + } diff --git a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java index 0057fdf889..0d8d73f0a3 100644 --- a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java +++ b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java @@ -56,6 +56,8 @@ public class OdpsUtil { public static int MAX_RETRY_TIME = 10; + public static final long BUFFER_SIZE_DEFAULT = 64 * 1024 * 1024; + public static Odps initOdps(Map odpsConfig) { String odpsServer = odpsConfig.get(OdpsConfigKeys.KEY_ODPS_SERVER); if(StringUtils.isBlank(odpsServer)) { diff --git a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java index 94bad7354e..1b134a790c 100644 --- a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java +++ b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java @@ -27,7 +27,6 @@ import com.aliyun.odps.tunnel.io.TunnelBufferedWriter; import com.dtstack.flinkx.common.ColumnType; import com.dtstack.flinkx.exception.WriteRecordException; -import com.dtstack.flinkx.odps.OdpsConfigKeys; import com.dtstack.flinkx.odps.OdpsUtil; import com.dtstack.flinkx.outputformat.RichOutputFormat; import com.dtstack.flinkx.util.DateUtil; @@ -35,7 +34,6 @@ import org.apache.flink.types.Row; import java.io.IOException; import java.math.BigDecimal; -import java.util.HashMap; import java.util.Map; /** @@ -60,6 +58,8 @@ public class OdpsOutputFormat extends RichOutputFormat { protected Map odpsConfig; + protected long bufferSize; + private transient Odps odps; private transient TableTunnel tunnel; @@ -96,6 +96,7 @@ public void openInternal(int taskNumber, int numTasks) throws IOException { session = OdpsUtil.createMasterTunnelUpload(tunnel, projectName, tableName, partition); try { recordWriter = (TunnelBufferedWriter) session.openBufferedWriter(); + recordWriter.setBufferSize(bufferSize); } catch (TunnelException e) { throw new RuntimeException("can not open record writer"); } diff --git a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormatBuilder.java b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormatBuilder.java index 682ff86a93..37f6bd36eb 100644 --- a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormatBuilder.java +++ b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormatBuilder.java @@ -67,6 +67,10 @@ public void setWriteMode(String writeMode) { this.format.writeMode = StringUtils.isBlank(writeMode) ? "APPEND" : writeMode.toUpperCase(); } + public void setBufferSize(long bufferSize){ + format.bufferSize = bufferSize; + } + @Override protected void checkFormat() { diff --git a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsWriter.java b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsWriter.java index b7bdf63f0c..f71d3e39e9 100644 --- a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsWriter.java +++ b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsWriter.java @@ -20,6 +20,7 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.WriterConfig; import com.dtstack.flinkx.odps.OdpsConfigKeys; +import com.dtstack.flinkx.odps.OdpsUtil; import com.dtstack.flinkx.writer.DataWriter; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -50,6 +51,8 @@ public class OdpsWriter extends DataWriter { protected String writeMode; + protected long bufferSize; + public OdpsWriter(DataTransferConfig config) { super(config); WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); @@ -60,6 +63,12 @@ public OdpsWriter(DataTransferConfig config) { projectName = writerConfig.getParameter().getStringVal(OdpsConfigKeys.KEY_PROJECT); writeMode = writerConfig.getParameter().getStringVal(OdpsConfigKeys.KEY_MODE); + bufferSize = writerConfig.getParameter().getLongVal(OdpsConfigKeys.KEY_BUFFER_SIZE, 0); + if (bufferSize == 0){ + bufferSize = OdpsUtil.BUFFER_SIZE_DEFAULT; + } else { + bufferSize = bufferSize * 1024 * 1024; + } List columns = (List) writerConfig.getParameter().getVal(OdpsConfigKeys.KEY_COLUMN_LIST); if(columns != null || columns.size() != 0) { @@ -88,6 +97,7 @@ public DataStreamSink writeData(DataStream dataSet) { builder.setSrcCols(srcCols); builder.setErrorRatio(errorRatio); builder.setErrors(errors); + builder.setBufferSize(bufferSize); OutputFormatSinkFunction sinkFunction = new OutputFormatSinkFunction(builder.finish()); DataStreamSink dataStreamSink = dataSet.addSink(sinkFunction); From 8e523c7fe6febf27e721324725171494883f1907 Mon Sep 17 00:00:00 2001 From: toutian Date: Thu, 21 Mar 2019 17:02:06 +0800 Subject: [PATCH 40/96] add gauge for count --- .../com/dtstack/flinkx/constants/Metrics.java | 5 ++ .../flinkx/inputformat/RichInputFormat.java | 5 +- .../dtstack/flinkx/metrics/InputMetric.java | 19 ++---- .../dtstack/flinkx/metrics/OutputMetric.java | 63 +++++-------------- .../metrics/SimpleAccumulatorGauge.java | 43 +++++++++++++ .../flinkx/outputformat/RichOutputFormat.java | 10 +-- 6 files changed, 74 insertions(+), 71 deletions(-) create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/metrics/SimpleAccumulatorGauge.java diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/Metrics.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/Metrics.java index 964705999d..95b57ba29d 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/Metrics.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/Metrics.java @@ -51,4 +51,9 @@ public class Metrics { public static String START_LOCATION = "startLocation"; public static String TABLE_COL = "tableCol"; + + public static String METRIC_GROUP_KEY_FLINKX = "flinkx"; + public static String METRIC_GROUP_VALUE_INPUT = "input"; + public static String METRIC_GROUP_VALUE_OUTPUT = "output"; + } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java index 91bc9b79b1..e1fdf673c2 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java @@ -62,10 +62,10 @@ public void open(InputSplit inputSplit) throws IOException { jobName = vars.get(Metrics.JOB_NAME); } - inputMetric = new InputMetric(getRuntimeContext()); - numReadCounter = getRuntimeContext().getLongCounter(Metrics.NUM_READS); + inputMetric = new InputMetric(getRuntimeContext(), numReadCounter); + openInternal(inputSplit); if (StringUtils.isNotBlank(this.monitorUrls) && this.bytes > 0) { @@ -79,7 +79,6 @@ public void open(InputSplit inputSplit) throws IOException { @Override public Row nextRecord(Row row) throws IOException { numReadCounter.add(1); - inputMetric.getNumRead().inc(); if(byteRateLimiter != null) { byteRateLimiter.acquire(); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java index 76b52937dd..0f11a718d1 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java @@ -19,32 +19,25 @@ package com.dtstack.flinkx.metrics; import com.dtstack.flinkx.constants.Metrics; +import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.MetricGroup; /** * company: www.dtstack.com - * author: toutian + * @author: toutian * create: 2019/3/18 */ public class InputMetric { - private Counter numRead; - private RuntimeContext runtimeContext; - public InputMetric(RuntimeContext runtimeContext) { + public InputMetric(RuntimeContext runtimeContext, LongCounter numRead) { this.runtimeContext = runtimeContext; - initMetric(); - } - - private void initMetric() { - numRead = getRuntimeContext().getMetricGroup().counter(Metrics.NUM_READS); - } + final MetricGroup flinkxInput = getRuntimeContext().getMetricGroup().addGroup(Metrics.METRIC_GROUP_KEY_FLINKX, Metrics.METRIC_GROUP_VALUE_INPUT); - public Counter getNumRead() { - return numRead; + flinkxInput.gauge(Metrics.NUM_READS, new SimpleAccumulatorGauge(numRead)); } private RuntimeContext getRuntimeContext() { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/OutputMetric.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/OutputMetric.java index 06afb6c0cb..79408ebc15 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/OutputMetric.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/OutputMetric.java @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

+ * * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -19,67 +19,36 @@ package com.dtstack.flinkx.metrics; import com.dtstack.flinkx.constants.Metrics; +import org.apache.flink.api.common.accumulators.IntCounter; +import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.MetricGroup; /** * company: www.dtstack.com - * author: toutian + * + * @author: toutian * create: 2019/3/18 */ public class OutputMetric { - private Counter numErrors; - private Counter numNullErrors; - private Counter numDuplicateErrors; - private Counter numConversionErrors; - private Counter numOtherErrors; - private Counter numWrite; - private transient RuntimeContext runtimeContext; - public OutputMetric(RuntimeContext runtimeContext) { + public OutputMetric(RuntimeContext runtimeContext, IntCounter numErrors, IntCounter numNullErrors, + IntCounter numDuplicateErrors, IntCounter numConversionErrors, IntCounter numOtherErrors, LongCounter numWrite) { this.runtimeContext = runtimeContext; - initMetric(); - } - - private void initMetric() { - numErrors = getRuntimeContext().getMetricGroup().counter(Metrics.NUM_ERRORS); - numNullErrors = getRuntimeContext().getMetricGroup().counter(Metrics.NUM_NULL_ERRORS); - numDuplicateErrors = getRuntimeContext().getMetricGroup().counter(Metrics.NUM_DUPLICATE_ERRORS); - numConversionErrors = getRuntimeContext().getMetricGroup().counter(Metrics.NUM_CONVERSION_ERRORS); - numOtherErrors = getRuntimeContext().getMetricGroup().counter(Metrics.NUM_OTHER_ERRORS); - numWrite = getRuntimeContext().getMetricGroup().counter(Metrics.NUM_WRITES); - } - - public Counter getNumErrors() { - return numErrors; - } - - public Counter getNumNullErrors() { - return numNullErrors; - } - - public Counter getNumDuplicateErrors() { - return numDuplicateErrors; - } - - public Counter getNumConversionErrors() { - return numConversionErrors; - } - - public Counter getNumOtherErrors() { - return numOtherErrors; - } + final MetricGroup flinkxOutput = getRuntimeContext().getMetricGroup().addGroup(Metrics.METRIC_GROUP_KEY_FLINKX, Metrics.METRIC_GROUP_VALUE_OUTPUT); - public Counter getNumWrite() { - return numWrite; + flinkxOutput.gauge(Metrics.NUM_ERRORS, new SimpleAccumulatorGauge(numErrors)); + flinkxOutput.gauge(Metrics.NUM_NULL_ERRORS, new SimpleAccumulatorGauge(numNullErrors)); + flinkxOutput.gauge(Metrics.NUM_DUPLICATE_ERRORS, new SimpleAccumulatorGauge(numDuplicateErrors)); + flinkxOutput.gauge(Metrics.NUM_CONVERSION_ERRORS, new SimpleAccumulatorGauge(numConversionErrors)); + flinkxOutput.gauge(Metrics.NUM_OTHER_ERRORS, new SimpleAccumulatorGauge(numOtherErrors)); + flinkxOutput.gauge(Metrics.NUM_WRITES, new SimpleAccumulatorGauge(numWrite)); } private RuntimeContext getRuntimeContext() { return runtimeContext; } - - } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/SimpleAccumulatorGauge.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/SimpleAccumulatorGauge.java new file mode 100644 index 0000000000..5c52d9fee5 --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/SimpleAccumulatorGauge.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.flinkx.metrics; + +import org.apache.flink.api.common.accumulators.SimpleAccumulator; +import org.apache.flink.metrics.Gauge; + +import java.io.Serializable; + +/** + * company: www.dtstack.com + * @author: toutian + * create: 2019/3/21 + */ +public class SimpleAccumulatorGauge implements Gauge { + + private SimpleAccumulator accumulator; + + public SimpleAccumulatorGauge(SimpleAccumulator accumulator) { + this.accumulator = accumulator; + } + + @Override + public T getValue() { + return accumulator.getLocalValue(); + } +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java index 866b79125d..d25090f8d3 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java @@ -166,8 +166,6 @@ public void open(int taskNumber, int numTasks) throws IOException { context = (StreamingRuntimeContext) getRuntimeContext(); this.numTasks = numTasks; - outputMetric = new OutputMetric(context); - //错误记录数 errCounter = context.getIntCounter(Metrics.NUM_ERRORS); nullErrCounter = context.getIntCounter(Metrics.NUM_NULL_ERRORS); @@ -178,6 +176,8 @@ public void open(int taskNumber, int numTasks) throws IOException { //总记录数 numWriteCounter = context.getLongCounter(Metrics.NUM_WRITES); + outputMetric = new OutputMetric(context, errCounter, nullErrCounter, duplicateErrCounter, conversionErrCounter, otherErrCounter, numWriteCounter); + Map vars = context.getMetricGroup().getAllVariables(); if(vars != null && vars.get(Metrics.JOB_NAME) != null) { @@ -241,10 +241,8 @@ protected void writeSingleRecord(Row row) { // 总记录数加1 numWriteCounter.add(1); - outputMetric.getNumWrite().inc(); } catch(WriteRecordException e) { errCounter.add(1); - outputMetric.getNumErrors().inc(); String errMsg = e.getMessage(); int pos = e.getColIndex(); @@ -261,16 +259,12 @@ protected void writeSingleRecord(Row row) { String errorType = dirtyDataManager.writeData(row, e); if (ERR_NULL_POINTER.equals(errorType)){ nullErrCounter.add(1); - outputMetric.getNumNullErrors().inc(); } else if(ERR_FORMAT_TRANSFORM.equals(errorType)){ conversionErrCounter.add(1); - outputMetric.getNumConversionErrors().inc(); } else if(ERR_PRIMARY_CONFLICT.equals(errorType)){ duplicateErrCounter.add(1); - outputMetric.getNumDuplicateErrors().inc(); } else { otherErrCounter.add(1); - outputMetric.getNumOtherErrors().inc(); } } From 24f6390913930c5caba5a22c8ff6a42356278289 Mon Sep 17 00:00:00 2001 From: toutian Date: Thu, 21 Mar 2019 19:50:52 +0800 Subject: [PATCH 41/96] =?UTF-8?q?=E6=B5=8B=E8=AF=95=20promethues=20?= =?UTF-8?q?=E6=95=B0=E6=8D=AE=E7=9A=84=E5=87=86=E7=A1=AE=E6=80=A7?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-core/.gitignore | 1 + .../com/dtstack/flinkx/metrics/SimpleAccumulatorGauge.java | 7 ++++--- .../dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java | 6 ++++++ 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/flinkx-core/.gitignore b/flinkx-core/.gitignore index ca7ca55c4c..9803fe0b0e 100644 --- a/flinkx-core/.gitignore +++ b/flinkx-core/.gitignore @@ -11,3 +11,4 @@ target .classpath *.eclipse.* *.iml +/dependency-reduced-pom.xml diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/SimpleAccumulatorGauge.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/SimpleAccumulatorGauge.java index 5c52d9fee5..a4d470b66f 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/SimpleAccumulatorGauge.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/SimpleAccumulatorGauge.java @@ -18,21 +18,22 @@ package com.dtstack.flinkx.metrics; -import org.apache.flink.api.common.accumulators.SimpleAccumulator; +import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.metrics.Gauge; import java.io.Serializable; /** * company: www.dtstack.com + * * @author: toutian * create: 2019/3/21 */ public class SimpleAccumulatorGauge implements Gauge { - private SimpleAccumulator accumulator; + private Accumulator accumulator; - public SimpleAccumulatorGauge(SimpleAccumulator accumulator) { + public SimpleAccumulatorGauge(Accumulator accumulator) { this.accumulator = accumulator; } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 5da390da37..ff7273944d 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -21,6 +21,7 @@ import com.dtstack.flinkx.common.ColumnType; import com.dtstack.flinkx.constants.Metrics; import com.dtstack.flinkx.enums.EDatabaseType; +import com.dtstack.flinkx.metrics.SimpleAccumulatorGauge; import com.dtstack.flinkx.rdb.DatabaseInterface; import com.dtstack.flinkx.rdb.type.TypeConverterInterface; import com.dtstack.flinkx.rdb.util.DBUtil; @@ -36,6 +37,7 @@ import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.types.Row; import java.io.IOException; import java.sql.*; @@ -120,16 +122,19 @@ public void configure(Configuration configuration) { private void setMetric(){ Map> accumulatorMap = getRuntimeContext().getAllAccumulators(); + final MetricGroup flinkxOutputMetricGroup = getRuntimeContext().getMetricGroup().addGroup(Metrics.METRIC_GROUP_KEY_FLINKX, Metrics.METRIC_GROUP_VALUE_OUTPUT); if(!accumulatorMap.containsKey(Metrics.TABLE_COL)){ tableColAccumulator = new StringAccumulator(); tableColAccumulator.add(table + "-" + increCol); getRuntimeContext().addAccumulator(Metrics.TABLE_COL,tableColAccumulator); + flinkxOutputMetricGroup.gauge(Metrics.TABLE_COL, new SimpleAccumulatorGauge(tableColAccumulator)); } if(!accumulatorMap.containsKey(Metrics.END_LOCATION)){ endLocationAccumulator = new MaximumAccumulator(); getRuntimeContext().addAccumulator(Metrics.END_LOCATION,endLocationAccumulator); + flinkxOutputMetricGroup.gauge(Metrics.END_LOCATION, new SimpleAccumulatorGauge(endLocationAccumulator)); } if (startLocation != null){ @@ -138,6 +143,7 @@ private void setMetric(){ startLocationAccumulator = new StringAccumulator(); startLocationAccumulator.add(startLocation); getRuntimeContext().addAccumulator(Metrics.START_LOCATION,startLocationAccumulator); + flinkxOutputMetricGroup.gauge(Metrics.START_LOCATION, new SimpleAccumulatorGauge(startLocationAccumulator)); } } From babc96524a3142de23a92759a91f4323f76abcd1 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 22 Mar 2019 11:23:50 +0800 Subject: [PATCH 42/96] =?UTF-8?q?hdfs=20orc=E8=AF=BB=E5=8F=96=E7=B1=BB?= =?UTF-8?q?=E5=9E=8B=E8=BD=AC=E6=8D=A2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java index 4a0ac66695..0edde3ff8e 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java @@ -206,6 +206,8 @@ public Row nextRecordInternal(Row row) throws IOException { if(val instanceof String || val instanceof org.apache.hadoop.io.Text){ val = HdfsUtil.string2col(String.valueOf(val),metaColumn.getType(),metaColumn.getTimeFormat()); + } else if(val != null){ + val = HdfsUtil.getWritableValue(val); } row.setField(i,val); From d0b7b9a874774ed96bc06a398deed6270daaf274 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 22 Mar 2019 11:40:41 +0800 Subject: [PATCH 43/96] =?UTF-8?q?close=E6=97=B6=E6=9B=B4=E6=96=B0errorInfo?= =?UTF-8?q?=E5=BC=82=E5=B8=B8=E6=8D=95=E8=8E=B7?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/outputformat/RichOutputFormat.java | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java index 42bffa1336..d05e84013f 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java @@ -332,13 +332,19 @@ public void close() throws IOException { if(dirtyDataManager != null) { dirtyDataManager.close(); } + if(errorLimiter != null) { - // Wait a while before checking dirty data - Latch latch = newLatch("#5"); - latch.addOne(); - latch.waitUntil(numTasks); + try{ + // Wait a while before checking dirty data + Latch latch = newLatch("#5"); + latch.addOne(); + latch.waitUntil(numTasks); + + errorLimiter.updateErrorInfo(); + } catch (Exception e){ + LOG.warn("Update error info error when task closing:{}", e); + } - errorLimiter.updateErrorInfo(); errorLimiter.acquire(); errorLimiter.stop(); } From 44aa8f075e48c4e15cb7f176738b9efb74c070a8 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 22 Mar 2019 16:51:01 +0800 Subject: [PATCH 44/96] =?UTF-8?q?=E8=87=AA=E5=AE=9A=E4=B9=89sql=E6=97=B6de?= =?UTF-8?q?scColumnTypeList=E4=B8=8D=E6=A0=B9=E6=8D=AE=E8=A1=A8=E5=90=8D?= =?UTF-8?q?=E8=8E=B7=E5=8F=96?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 06a387673b..4a1894265c 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -171,8 +171,13 @@ public void openInternal(InputSplit inputSplit) throws IOException { columnCount = resultSet.getMetaData().getColumnCount(); hasNext = resultSet.next(); - if(descColumnTypeList == null) { + if (StringUtils.isEmpty(customSql)){ descColumnTypeList = DBUtil.analyzeTable(dbURL, username, password,databaseInterface,table,metaColumns); + } else { + descColumnTypeList = new ArrayList<>(); + for (MetaColumn metaColumn : metaColumns) { + descColumnTypeList.add(metaColumn.getName()); + } } } catch (SQLException se) { From 9c76a56dbbf2fec889f00bc680d3aab53d945a95 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 22 Mar 2019 20:47:17 +0800 Subject: [PATCH 45/96] =?UTF-8?q?=E4=BF=AE=E5=A4=8Dstring=E7=B1=BB?= =?UTF-8?q?=E5=9E=8B=E8=BD=AC=E4=B8=BAnull=E7=9A=84bug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java index fdce472e6d..ff50ff20e4 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java @@ -155,6 +155,8 @@ public static Object getWritableValue(Object writable) { ret = ((DateWritable) writable).get(); } else if(writable instanceof Writable) { ret = writable.toString(); + } else { + ret = writable.toString(); } return ret; From 5dfd73aa8c67ef3c91b8feb442a37ffa40720309 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 22 Mar 2019 20:47:17 +0800 Subject: [PATCH 46/96] =?UTF-8?q?=E4=BF=AE=E5=A4=8Dstring=E7=B1=BB?= =?UTF-8?q?=E5=9E=8B=E8=BD=AC=E4=B8=BAnull=E7=9A=84bug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java index fdce472e6d..ff50ff20e4 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java @@ -155,6 +155,8 @@ public static Object getWritableValue(Object writable) { ret = ((DateWritable) writable).get(); } else if(writable instanceof Writable) { ret = writable.toString(); + } else { + ret = writable.toString(); } return ret; From e2b72a0cb7b462d3cf9f011c493aae021175ff27 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 22 Mar 2019 20:47:17 +0800 Subject: [PATCH 47/96] =?UTF-8?q?=E4=BF=AE=E5=A4=8Dstring=E7=B1=BB?= =?UTF-8?q?=E5=9E=8B=E8=BD=AC=E4=B8=BAnull=E7=9A=84bug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java index fdce472e6d..ff50ff20e4 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java @@ -155,6 +155,8 @@ public static Object getWritableValue(Object writable) { ret = ((DateWritable) writable).get(); } else if(writable instanceof Writable) { ret = writable.toString(); + } else { + ret = writable.toString(); } return ret; From 1d78f67343977a40b97da9b761f8ac017dba6d66 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 25 Mar 2019 10:14:39 +0800 Subject: [PATCH 48/96] =?UTF-8?q?=E4=BF=AE=E5=A4=8DDate=E7=B1=BB=E5=9E=8B?= =?UTF-8?q?=E8=BD=ACTimestamp=E6=8A=A5=E9=94=99?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java index dbd9dbc79a..fe1b07a625 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java @@ -123,7 +123,7 @@ public static java.sql.Timestamp columnToTimestamp(Object column,SimpleDateForma Long rawData = (Long) column; return new java.sql.Timestamp(getMillSecond(rawData.toString())); } else if (column instanceof java.sql.Date) { - return (java.sql.Timestamp) column; + return new java.sql.Timestamp(((java.sql.Date) column).getTime()); } else if(column instanceof Timestamp) { return (Timestamp) column; } else if(column instanceof Date) { From aef73008ad247932b8581aab3eadd6657356c348 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 27 Mar 2019 20:46:15 +0800 Subject: [PATCH 49/96] =?UTF-8?q?es=20reader=20=E6=B7=BB=E5=8A=A0index=20?= =?UTF-8?q?=E5=92=8C=20type=20=E9=80=89=E9=A1=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flinkx/es/EsUtil.java | 23 +++++++++++++------ .../flinkx/es/reader/EsInputFormat.java | 8 +++++-- .../es/reader/EsInputFormatBuilder.java | 10 ++++++++ .../dtstack/flinkx/es/reader/EsReader.java | 7 ++++++ 4 files changed, 39 insertions(+), 9 deletions(-) diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java index 3c65db2821..34decfd126 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java @@ -31,6 +31,7 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.Strings; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.SearchHit; @@ -64,11 +65,19 @@ public static RestHighLevelClient getClient(String address) { return client; } - public static SearchResponse search(RestHighLevelClient client, String query, int from, int size) { - SearchRequest searchRequest = new SearchRequest(); + public static SearchResponse search(RestHighLevelClient client, String index, String type, String query, int from, int size) { + SearchRequest searchRequest = Strings.isNullOrEmpty(index) ? new SearchRequest() : new SearchRequest(index); + + if(!Strings.isNullOrEmpty(type)){ + searchRequest.types(type); + } + SearchSourceBuilder sourceBuilder = new SearchSourceBuilder(); sourceBuilder.from(from); - sourceBuilder.size(size); + + if(size > 0){ + sourceBuilder.size(size); + } if(StringUtils.isNotBlank(query)) { QueryBuilder qb = QueryBuilders.wrapperQuery(query); @@ -83,13 +92,13 @@ public static SearchResponse search(RestHighLevelClient client, String query, in } } - public static long searchCount(RestHighLevelClient client, String query) { - SearchResponse searchResponse = search(client, query, 0, 0); + public static long searchCount(RestHighLevelClient client, String index, String type, String query) { + SearchResponse searchResponse = search(client, index, type, query, 0, 0); return searchResponse.getHits().getTotalHits(); } - public static List> searchContent(RestHighLevelClient client, String query, int from, int size) { - SearchResponse searchResponse = search(client, query, from, size); + public static List> searchContent(RestHighLevelClient client, String index, String type, String query, int from, int size) { + SearchResponse searchResponse = search(client, index, type, query, from, size); SearchHits searchHits = searchResponse.getHits(); List> resultList = new ArrayList<>(); for(SearchHit searchHit : searchHits) { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java index e8548d4b59..2441f8281d 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java @@ -44,6 +44,10 @@ public class EsInputFormat extends RichInputFormat { protected String address; + protected String index; + + protected String type; + protected String query; protected List columnValues; @@ -79,7 +83,7 @@ public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOExce @Override public InputSplit[] createInputSplits(int splitNum) throws IOException { - long cnt = EsUtil.searchCount(client, query); + long cnt = EsUtil.searchCount(client, index, type, query); if (cnt < splitNum) { EsInputSplit[] splits = new EsInputSplit[1]; splits[0] = new EsInputSplit(0, (int)cnt); @@ -123,7 +127,7 @@ private void loadNextBatch() { if (from + range > to) { range = to - from; } - resultList = EsUtil.searchContent(client, query, from, range); + resultList = EsUtil.searchContent(client, index, type, query, from, range); from += range; pos = 0; } diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java index 3f9a7be530..c2d7566aad 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java @@ -65,6 +65,16 @@ public EsInputFormatBuilder setColumnTypes(List columnTypes) { return this; } + public EsInputFormatBuilder setIndex(String index){ + format.index = index; + return this; + } + + public EsInputFormatBuilder setType(String type){ + format.type = type; + return this; + } + @Override protected void checkFormat() { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java index e4fc711d81..6dc5ce5ded 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java @@ -41,6 +41,9 @@ public class EsReader extends DataReader { private String address; private String query; + private String index; + private String type; + protected List columnType; protected List columnValue; protected List columnName; @@ -49,6 +52,8 @@ public EsReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); address = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_ADDRESS); + index = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_INDEX); + type = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_TYPE); Object queryMap = readerConfig.getParameter().getVal(EsConfigKeys.KEY_QUERY); if(queryMap != null) { @@ -83,6 +88,8 @@ public DataStream readData() { builder.setColumnTypes(columnType); builder.setColumnValues(columnValue); builder.setAddress(address); + builder.setIndex(index); + builder.setType(type); builder.setQuery(query); builder.setBytes(bytes); builder.setMonitorUrls(monitorUrls); From af24289a611600542037d93a0da093121b96f0e1 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 27 Mar 2019 21:41:46 +0800 Subject: [PATCH 50/96] =?UTF-8?q?=E5=A4=84=E7=90=86es=20=E8=AF=BB=E5=8F=96?= =?UTF-8?q?=E6=95=B0=E6=8D=AE=E9=87=8D=E5=A4=8D=E9=97=AE=E9=A2=98=E5=92=8C?= =?UTF-8?q?=E8=AF=BB=E5=8F=96=E7=9A=84=E6=97=B6=E5=80=99=E6=95=B0=E7=BB=84?= =?UTF-8?q?=E8=B6=8A=E7=95=8C=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/flinkx/es/EsConfigKeys.java | 2 ++ .../src/main/java/com/dtstack/flinkx/es/EsUtil.java | 3 ++- .../java/com/dtstack/flinkx/es/reader/EsInputFormat.java | 8 ++++++-- .../dtstack/flinkx/es/reader/EsInputFormatBuilder.java | 7 +++++++ .../main/java/com/dtstack/flinkx/es/reader/EsReader.java | 3 +++ 5 files changed, 20 insertions(+), 3 deletions(-) diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java index 497da0b39a..fa8f8593fc 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java @@ -34,6 +34,8 @@ public class EsConfigKeys { public static final String KEY_TYPE = "type"; + public static final String KEY_BATCH_SIZE = "batchSize"; + public static final String KEY_BULK_ACTION = "bulkAction"; public static final String KEY_COLUMN_NAME = "name"; diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java index 34decfd126..6728f342ad 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java @@ -82,9 +82,10 @@ public static SearchResponse search(RestHighLevelClient client, String index, St if(StringUtils.isNotBlank(query)) { QueryBuilder qb = QueryBuilders.wrapperQuery(query); sourceBuilder.query(qb); - searchRequest.source(sourceBuilder); } + searchRequest.source(sourceBuilder); + try { return client.search(searchRequest); } catch (IOException e) { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java index 2441f8281d..018a2e5e1f 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java @@ -56,7 +56,7 @@ public class EsInputFormat extends RichInputFormat { protected List columnNames; - private int batch = 2; + protected int batchSize = 10; private int from; @@ -123,7 +123,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { } private void loadNextBatch() { - int range = batch; + int range = batchSize; if (from + range > to) { range = to - from; } @@ -139,6 +139,10 @@ public boolean reachedEnd() throws IOException { return true; } loadNextBatch(); + + //check again + return reachedEnd(); + } return false; } diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java index c2d7566aad..802aef3451 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java @@ -75,6 +75,13 @@ public EsInputFormatBuilder setType(String type){ return this; } + public EsInputFormatBuilder setBatchSize(Integer batchSize){ + if(batchSize != null && batchSize > 0){ + format.batchSize = batchSize; + } + return this; + } + @Override protected void checkFormat() { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java index 6dc5ce5ded..58aa792da2 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java @@ -43,6 +43,7 @@ public class EsReader extends DataReader { private String index; private String type; + private Integer batchSize; protected List columnType; protected List columnValue; @@ -54,6 +55,7 @@ public EsReader(DataTransferConfig config, StreamExecutionEnvironment env) { address = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_ADDRESS); index = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_INDEX); type = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_TYPE); + batchSize = readerConfig.getParameter().getIntVal(EsConfigKeys.KEY_BATCH_SIZE, 0); Object queryMap = readerConfig.getParameter().getVal(EsConfigKeys.KEY_QUERY); if(queryMap != null) { @@ -90,6 +92,7 @@ public DataStream readData() { builder.setAddress(address); builder.setIndex(index); builder.setType(type); + builder.setBatchSize(batchSize); builder.setQuery(query); builder.setBytes(bytes); builder.setMonitorUrls(monitorUrls); From b7dd82d295615913ce373b546994835c74b65410 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BE=90=E8=B6=85?= Date: Thu, 28 Mar 2019 12:05:44 +0800 Subject: [PATCH 51/96] Revert "Merge branch '1.5_v3.4.2_es_reader_no_index' into '1.5_v3.4.2' " This reverts merge request !18 --- .../com/dtstack/flinkx/es/EsConfigKeys.java | 2 -- .../java/com/dtstack/flinkx/es/EsUtil.java | 26 ++++++------------- .../flinkx/es/reader/EsInputFormat.java | 16 +++--------- .../es/reader/EsInputFormatBuilder.java | 17 ------------ .../dtstack/flinkx/es/reader/EsReader.java | 10 ------- 5 files changed, 12 insertions(+), 59 deletions(-) diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java index fa8f8593fc..497da0b39a 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java @@ -34,8 +34,6 @@ public class EsConfigKeys { public static final String KEY_TYPE = "type"; - public static final String KEY_BATCH_SIZE = "batchSize"; - public static final String KEY_BULK_ACTION = "bulkAction"; public static final String KEY_COLUMN_NAME = "name"; diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java index 6728f342ad..3c65db2821 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java @@ -31,7 +31,6 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.common.Strings; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.SearchHit; @@ -65,27 +64,18 @@ public static RestHighLevelClient getClient(String address) { return client; } - public static SearchResponse search(RestHighLevelClient client, String index, String type, String query, int from, int size) { - SearchRequest searchRequest = Strings.isNullOrEmpty(index) ? new SearchRequest() : new SearchRequest(index); - - if(!Strings.isNullOrEmpty(type)){ - searchRequest.types(type); - } - + public static SearchResponse search(RestHighLevelClient client, String query, int from, int size) { + SearchRequest searchRequest = new SearchRequest(); SearchSourceBuilder sourceBuilder = new SearchSourceBuilder(); sourceBuilder.from(from); - - if(size > 0){ - sourceBuilder.size(size); - } + sourceBuilder.size(size); if(StringUtils.isNotBlank(query)) { QueryBuilder qb = QueryBuilders.wrapperQuery(query); sourceBuilder.query(qb); + searchRequest.source(sourceBuilder); } - searchRequest.source(sourceBuilder); - try { return client.search(searchRequest); } catch (IOException e) { @@ -93,13 +83,13 @@ public static SearchResponse search(RestHighLevelClient client, String index, St } } - public static long searchCount(RestHighLevelClient client, String index, String type, String query) { - SearchResponse searchResponse = search(client, index, type, query, 0, 0); + public static long searchCount(RestHighLevelClient client, String query) { + SearchResponse searchResponse = search(client, query, 0, 0); return searchResponse.getHits().getTotalHits(); } - public static List> searchContent(RestHighLevelClient client, String index, String type, String query, int from, int size) { - SearchResponse searchResponse = search(client, index, type, query, from, size); + public static List> searchContent(RestHighLevelClient client, String query, int from, int size) { + SearchResponse searchResponse = search(client, query, from, size); SearchHits searchHits = searchResponse.getHits(); List> resultList = new ArrayList<>(); for(SearchHit searchHit : searchHits) { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java index 018a2e5e1f..e8548d4b59 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java @@ -44,10 +44,6 @@ public class EsInputFormat extends RichInputFormat { protected String address; - protected String index; - - protected String type; - protected String query; protected List columnValues; @@ -56,7 +52,7 @@ public class EsInputFormat extends RichInputFormat { protected List columnNames; - protected int batchSize = 10; + private int batch = 2; private int from; @@ -83,7 +79,7 @@ public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOExce @Override public InputSplit[] createInputSplits(int splitNum) throws IOException { - long cnt = EsUtil.searchCount(client, index, type, query); + long cnt = EsUtil.searchCount(client, query); if (cnt < splitNum) { EsInputSplit[] splits = new EsInputSplit[1]; splits[0] = new EsInputSplit(0, (int)cnt); @@ -123,11 +119,11 @@ public void openInternal(InputSplit inputSplit) throws IOException { } private void loadNextBatch() { - int range = batchSize; + int range = batch; if (from + range > to) { range = to - from; } - resultList = EsUtil.searchContent(client, index, type, query, from, range); + resultList = EsUtil.searchContent(client, query, from, range); from += range; pos = 0; } @@ -139,10 +135,6 @@ public boolean reachedEnd() throws IOException { return true; } loadNextBatch(); - - //check again - return reachedEnd(); - } return false; } diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java index 802aef3451..3f9a7be530 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java @@ -65,23 +65,6 @@ public EsInputFormatBuilder setColumnTypes(List columnTypes) { return this; } - public EsInputFormatBuilder setIndex(String index){ - format.index = index; - return this; - } - - public EsInputFormatBuilder setType(String type){ - format.type = type; - return this; - } - - public EsInputFormatBuilder setBatchSize(Integer batchSize){ - if(batchSize != null && batchSize > 0){ - format.batchSize = batchSize; - } - return this; - } - @Override protected void checkFormat() { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java index 58aa792da2..e4fc711d81 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java @@ -41,10 +41,6 @@ public class EsReader extends DataReader { private String address; private String query; - private String index; - private String type; - private Integer batchSize; - protected List columnType; protected List columnValue; protected List columnName; @@ -53,9 +49,6 @@ public EsReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); address = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_ADDRESS); - index = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_INDEX); - type = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_TYPE); - batchSize = readerConfig.getParameter().getIntVal(EsConfigKeys.KEY_BATCH_SIZE, 0); Object queryMap = readerConfig.getParameter().getVal(EsConfigKeys.KEY_QUERY); if(queryMap != null) { @@ -90,9 +83,6 @@ public DataStream readData() { builder.setColumnTypes(columnType); builder.setColumnValues(columnValue); builder.setAddress(address); - builder.setIndex(index); - builder.setType(type); - builder.setBatchSize(batchSize); builder.setQuery(query); builder.setBytes(bytes); builder.setMonitorUrls(monitorUrls); From 93b62b44232da813e4bf1c8f573602819c9ce503 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 28 Mar 2019 13:07:28 +0800 Subject: [PATCH 52/96] =?UTF-8?q?=E6=B7=BB=E5=8A=A0=E4=B8=80=E4=B8=AA?= =?UTF-8?q?=E7=A9=BA=E8=A1=8C?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flinkx/es/EsUtil.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java index 6728f342ad..f018f6d4ce 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java @@ -91,6 +91,7 @@ public static SearchResponse search(RestHighLevelClient client, String index, St } catch (IOException e) { throw new RuntimeException(e); } + } public static long searchCount(RestHighLevelClient client, String index, String type, String query) { From 1bed9947c199c7a9e8ca978738135cc5047ce7b0 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 27 Mar 2019 20:46:15 +0800 Subject: [PATCH 53/96] =?UTF-8?q?es=20reader=20=E6=B7=BB=E5=8A=A0index=20?= =?UTF-8?q?=E5=92=8C=20type=20=E9=80=89=E9=A1=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flinkx/es/EsUtil.java | 23 +++++++++++++------ .../flinkx/es/reader/EsInputFormat.java | 8 +++++-- .../es/reader/EsInputFormatBuilder.java | 10 ++++++++ .../dtstack/flinkx/es/reader/EsReader.java | 7 ++++++ 4 files changed, 39 insertions(+), 9 deletions(-) diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java index 0a4136cb74..3e7647b149 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java @@ -31,6 +31,7 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.Strings; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.SearchHit; @@ -64,11 +65,19 @@ public static RestHighLevelClient getClient(String address) { return client; } - public static SearchResponse search(RestHighLevelClient client, String query, int from, int size) { - SearchRequest searchRequest = new SearchRequest(); + public static SearchResponse search(RestHighLevelClient client, String index, String type, String query, int from, int size) { + SearchRequest searchRequest = Strings.isNullOrEmpty(index) ? new SearchRequest() : new SearchRequest(index); + + if(!Strings.isNullOrEmpty(type)){ + searchRequest.types(type); + } + SearchSourceBuilder sourceBuilder = new SearchSourceBuilder(); sourceBuilder.from(from); - sourceBuilder.size(size); + + if(size > 0){ + sourceBuilder.size(size); + } if(StringUtils.isNotBlank(query)) { QueryBuilder qb = QueryBuilders.wrapperQuery(query); @@ -84,13 +93,13 @@ public static SearchResponse search(RestHighLevelClient client, String query, in } - public static long searchCount(RestHighLevelClient client, String query) { - SearchResponse searchResponse = search(client, query, 0, 0); + public static long searchCount(RestHighLevelClient client, String index, String type, String query) { + SearchResponse searchResponse = search(client, index, type, query, 0, 0); return searchResponse.getHits().getTotalHits(); } - public static List> searchContent(RestHighLevelClient client, String query, int from, int size) { - SearchResponse searchResponse = search(client, query, from, size); + public static List> searchContent(RestHighLevelClient client, String index, String type, String query, int from, int size) { + SearchResponse searchResponse = search(client, index, type, query, from, size); SearchHits searchHits = searchResponse.getHits(); List> resultList = new ArrayList<>(); for(SearchHit searchHit : searchHits) { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java index e8548d4b59..2441f8281d 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java @@ -44,6 +44,10 @@ public class EsInputFormat extends RichInputFormat { protected String address; + protected String index; + + protected String type; + protected String query; protected List columnValues; @@ -79,7 +83,7 @@ public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOExce @Override public InputSplit[] createInputSplits(int splitNum) throws IOException { - long cnt = EsUtil.searchCount(client, query); + long cnt = EsUtil.searchCount(client, index, type, query); if (cnt < splitNum) { EsInputSplit[] splits = new EsInputSplit[1]; splits[0] = new EsInputSplit(0, (int)cnt); @@ -123,7 +127,7 @@ private void loadNextBatch() { if (from + range > to) { range = to - from; } - resultList = EsUtil.searchContent(client, query, from, range); + resultList = EsUtil.searchContent(client, index, type, query, from, range); from += range; pos = 0; } diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java index 3f9a7be530..c2d7566aad 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java @@ -65,6 +65,16 @@ public EsInputFormatBuilder setColumnTypes(List columnTypes) { return this; } + public EsInputFormatBuilder setIndex(String index){ + format.index = index; + return this; + } + + public EsInputFormatBuilder setType(String type){ + format.type = type; + return this; + } + @Override protected void checkFormat() { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java index e4fc711d81..6dc5ce5ded 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java @@ -41,6 +41,9 @@ public class EsReader extends DataReader { private String address; private String query; + private String index; + private String type; + protected List columnType; protected List columnValue; protected List columnName; @@ -49,6 +52,8 @@ public EsReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); address = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_ADDRESS); + index = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_INDEX); + type = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_TYPE); Object queryMap = readerConfig.getParameter().getVal(EsConfigKeys.KEY_QUERY); if(queryMap != null) { @@ -83,6 +88,8 @@ public DataStream readData() { builder.setColumnTypes(columnType); builder.setColumnValues(columnValue); builder.setAddress(address); + builder.setIndex(index); + builder.setType(type); builder.setQuery(query); builder.setBytes(bytes); builder.setMonitorUrls(monitorUrls); From e8b12e414ed28ee82a4561bf243903861a1848e4 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 27 Mar 2019 21:41:46 +0800 Subject: [PATCH 54/96] =?UTF-8?q?=E5=A4=84=E7=90=86es=20=E8=AF=BB=E5=8F=96?= =?UTF-8?q?=E6=95=B0=E6=8D=AE=E9=87=8D=E5=A4=8D=E9=97=AE=E9=A2=98=E5=92=8C?= =?UTF-8?q?=E8=AF=BB=E5=8F=96=E7=9A=84=E6=97=B6=E5=80=99=E6=95=B0=E7=BB=84?= =?UTF-8?q?=E8=B6=8A=E7=95=8C=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/flinkx/es/EsConfigKeys.java | 2 ++ .../src/main/java/com/dtstack/flinkx/es/EsUtil.java | 3 ++- .../java/com/dtstack/flinkx/es/reader/EsInputFormat.java | 8 ++++++-- .../dtstack/flinkx/es/reader/EsInputFormatBuilder.java | 7 +++++++ .../main/java/com/dtstack/flinkx/es/reader/EsReader.java | 3 +++ 5 files changed, 20 insertions(+), 3 deletions(-) diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java index 497da0b39a..fa8f8593fc 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java @@ -34,6 +34,8 @@ public class EsConfigKeys { public static final String KEY_TYPE = "type"; + public static final String KEY_BATCH_SIZE = "batchSize"; + public static final String KEY_BULK_ACTION = "bulkAction"; public static final String KEY_COLUMN_NAME = "name"; diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java index 3e7647b149..f018f6d4ce 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java @@ -82,9 +82,10 @@ public static SearchResponse search(RestHighLevelClient client, String index, St if(StringUtils.isNotBlank(query)) { QueryBuilder qb = QueryBuilders.wrapperQuery(query); sourceBuilder.query(qb); - searchRequest.source(sourceBuilder); } + searchRequest.source(sourceBuilder); + try { return client.search(searchRequest); } catch (IOException e) { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java index 2441f8281d..018a2e5e1f 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java @@ -56,7 +56,7 @@ public class EsInputFormat extends RichInputFormat { protected List columnNames; - private int batch = 2; + protected int batchSize = 10; private int from; @@ -123,7 +123,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { } private void loadNextBatch() { - int range = batch; + int range = batchSize; if (from + range > to) { range = to - from; } @@ -139,6 +139,10 @@ public boolean reachedEnd() throws IOException { return true; } loadNextBatch(); + + //check again + return reachedEnd(); + } return false; } diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java index c2d7566aad..802aef3451 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java @@ -75,6 +75,13 @@ public EsInputFormatBuilder setType(String type){ return this; } + public EsInputFormatBuilder setBatchSize(Integer batchSize){ + if(batchSize != null && batchSize > 0){ + format.batchSize = batchSize; + } + return this; + } + @Override protected void checkFormat() { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java index 6dc5ce5ded..58aa792da2 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java @@ -43,6 +43,7 @@ public class EsReader extends DataReader { private String index; private String type; + private Integer batchSize; protected List columnType; protected List columnValue; @@ -54,6 +55,7 @@ public EsReader(DataTransferConfig config, StreamExecutionEnvironment env) { address = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_ADDRESS); index = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_INDEX); type = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_TYPE); + batchSize = readerConfig.getParameter().getIntVal(EsConfigKeys.KEY_BATCH_SIZE, 0); Object queryMap = readerConfig.getParameter().getVal(EsConfigKeys.KEY_QUERY); if(queryMap != null) { @@ -90,6 +92,7 @@ public DataStream readData() { builder.setAddress(address); builder.setIndex(index); builder.setType(type); + builder.setBatchSize(batchSize); builder.setQuery(query); builder.setBytes(bytes); builder.setMonitorUrls(monitorUrls); From 4b3eb188f026f8e9e557ff32a405f521a3dacca0 Mon Sep 17 00:00:00 2001 From: toutian Date: Thu, 28 Mar 2019 21:39:36 +0800 Subject: [PATCH 55/96] period --- .../flinkx/inputformat/RichInputFormat.java | 5 ++ .../dtstack/flinkx/metrics/InputMetric.java | 61 +++++++++++++++++++ 2 files changed, 66 insertions(+) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java index e1fdf673c2..a8eb1588b7 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java @@ -21,6 +21,7 @@ import com.dtstack.flinkx.constants.Metrics; import com.dtstack.flinkx.metrics.InputMetric; import com.dtstack.flinkx.reader.ByteRateLimiter; +import com.dtstack.flinkx.util.SysUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; @@ -92,6 +93,10 @@ public Row nextRecord(Row row) throws IOException { @Override public void close() throws IOException { try{ + if (inputMetric.getDelayPeriodMill() != 0){ + SysUtil.sleep(inputMetric.getDelayPeriodMill()); + } + closeInternal(); }catch (Exception e){ throw new RuntimeException(e); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java index 0f11a718d1..0c2885bdf8 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java @@ -22,25 +22,86 @@ import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; +import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.concurrent.duration.FiniteDuration; + +import java.lang.reflect.Field; +import java.util.concurrent.RunnableScheduledFuture; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; /** * company: www.dtstack.com + * * @author: toutian * create: 2019/3/18 */ public class InputMetric { + protected final Logger LOG = LoggerFactory.getLogger(getClass()); private RuntimeContext runtimeContext; + private final static Long DEFAULT_PERIOD_MILLISECONDS = 10000L; + + private Long delayPeriodMill = 20000L; + public InputMetric(RuntimeContext runtimeContext, LongCounter numRead) { this.runtimeContext = runtimeContext; final MetricGroup flinkxInput = getRuntimeContext().getMetricGroup().addGroup(Metrics.METRIC_GROUP_KEY_FLINKX, Metrics.METRIC_GROUP_VALUE_INPUT); flinkxInput.gauge(Metrics.NUM_READS, new SimpleAccumulatorGauge(numRead)); + + initPeriod(); } private RuntimeContext getRuntimeContext() { return runtimeContext; } + + public Long getDelayPeriodMill() { + return delayPeriodMill; + } + + public void initPeriod() { + try { + MetricGroup mgObj = runtimeContext.getMetricGroup(); + Class amgCls = (Class) mgObj.getClass().getSuperclass().getSuperclass(); + Field registryField = amgCls.getDeclaredField("registry"); + registryField.setAccessible(true); + MetricRegistryImpl registryImplObj = (MetricRegistryImpl) registryField.get(mgObj); + if (registryImplObj.getReporters().isEmpty()) { + return; + } + Field executorField = registryImplObj.getClass().getDeclaredField("executor"); + executorField.setAccessible(true); + ScheduledExecutorService executor = (ScheduledExecutorService) executorField.get(registryImplObj); + Field scheduleField = (executor.getClass().getSuperclass().getDeclaredField("e")); + scheduleField.setAccessible(true); + ScheduledThreadPoolExecutor scheduleObj = (ScheduledThreadPoolExecutor) scheduleField.get(executor); + Runnable runableObj = scheduleObj.getQueue().iterator().next(); + RunnableScheduledFuture runableFuture = (RunnableScheduledFuture) runableObj; + Field outerTaskField = runableFuture.getClass().getDeclaredField("outerTask"); + outerTaskField.setAccessible(true); + Object scheduledFutureTask = outerTaskField.get(runableFuture); + Field periodField = scheduledFutureTask.getClass().getDeclaredField("period"); + periodField.setAccessible(true); + long schedulePeriod = (long) periodField.get(scheduledFutureTask); + long schedulePeriodMill = -1 * new FiniteDuration(schedulePeriod, TimeUnit.NANOSECONDS).toMillis(); + + LOG.info("InputMetric.scheduledFutureTask.schedulePeriodSecond(TimeUnit.MILLISECONDS):{} ...", schedulePeriodMill); + + if (schedulePeriodMill > DEFAULT_PERIOD_MILLISECONDS) { + this.delayPeriodMill = schedulePeriodMill * 2; + } + } catch (Exception e) { + LOG.error("{}", e); + } + + LOG.info("InputMetric.period:{} ...", delayPeriodMill); + } } From 53151d21fc2d052c04367df5d5c5d968e7dd2d06 Mon Sep 17 00:00:00 2001 From: toutian Date: Thu, 28 Mar 2019 21:45:09 +0800 Subject: [PATCH 56/96] mill --- .../main/java/com/dtstack/flinkx/metrics/InputMetric.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java index 0c2885bdf8..0fecc85b34 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - * + *

* http://www.apache.org/licenses/LICENSE-2.0 - * + *

* Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -93,7 +93,7 @@ public void initPeriod() { long schedulePeriod = (long) periodField.get(scheduledFutureTask); long schedulePeriodMill = -1 * new FiniteDuration(schedulePeriod, TimeUnit.NANOSECONDS).toMillis(); - LOG.info("InputMetric.scheduledFutureTask.schedulePeriodSecond(TimeUnit.MILLISECONDS):{} ...", schedulePeriodMill); + LOG.info("InputMetric.scheduledFutureTask.schedulePeriodMill:{} ...", schedulePeriodMill); if (schedulePeriodMill > DEFAULT_PERIOD_MILLISECONDS) { this.delayPeriodMill = schedulePeriodMill * 2; @@ -102,6 +102,6 @@ public void initPeriod() { LOG.error("{}", e); } - LOG.info("InputMetric.period:{} ...", delayPeriodMill); + LOG.info("InputMetric.delayPeriodMill:{} ...", delayPeriodMill); } } From efa8dd66f99a398f1e252e4949de2053ff4beadd Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 29 Mar 2019 14:00:35 +0800 Subject: [PATCH 57/96] =?UTF-8?q?fix=E5=A4=9A=E9=80=9A=E9=81=93=E4=B8=8D?= =?UTF-8?q?=E6=8C=87=E5=AE=9A=E5=88=87=E5=88=86=E9=94=AE=E6=95=B0=E6=8D=AE?= =?UTF-8?q?=E9=87=8D=E5=A4=8D=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 8e2fef9189..7d676a351a 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -378,6 +378,14 @@ private String getMaxvalueFromAccumulator(String[] monitors){ } private boolean canReadData(InputSplit split){ + /* + * If the user set multiple channels but does not specify a splitKey, + * the data is read only on the first channel to prevent data duplication + */ + if (numPartitions > 1 && StringUtils.isEmpty(splitKey) && getRuntimeContext().getIndexOfThisSubtask() > 0){ + return false; + } + if (StringUtils.isEmpty(increCol)){ return true; } From b9db2d5dcbb8ef0c43c88038cbcbb779dab804fd Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 25 Mar 2019 10:14:39 +0800 Subject: [PATCH 58/96] =?UTF-8?q?=E4=BF=AE=E5=A4=8DDate=E7=B1=BB=E5=9E=8B?= =?UTF-8?q?=E8=BD=ACTimestamp=E6=8A=A5=E9=94=99?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java index dbd9dbc79a..fe1b07a625 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java @@ -123,7 +123,7 @@ public static java.sql.Timestamp columnToTimestamp(Object column,SimpleDateForma Long rawData = (Long) column; return new java.sql.Timestamp(getMillSecond(rawData.toString())); } else if (column instanceof java.sql.Date) { - return (java.sql.Timestamp) column; + return new java.sql.Timestamp(((java.sql.Date) column).getTime()); } else if(column instanceof Timestamp) { return (Timestamp) column; } else if(column instanceof Date) { From 02877893cab62a72dbaa4727963db41d1465996f Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 22 Mar 2019 11:23:50 +0800 Subject: [PATCH 59/96] =?UTF-8?q?hdfs=20orc=E8=AF=BB=E5=8F=96=E7=B1=BB?= =?UTF-8?q?=E5=9E=8B=E8=BD=AC=E6=8D=A2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java index 4a0ac66695..0edde3ff8e 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java @@ -206,6 +206,8 @@ public Row nextRecordInternal(Row row) throws IOException { if(val instanceof String || val instanceof org.apache.hadoop.io.Text){ val = HdfsUtil.string2col(String.valueOf(val),metaColumn.getType(),metaColumn.getTimeFormat()); + } else if(val != null){ + val = HdfsUtil.getWritableValue(val); } row.setField(i,val); From effab08bd54d4a8d9a947f194f55b176803dd1a8 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 22 Mar 2019 20:47:17 +0800 Subject: [PATCH 60/96] =?UTF-8?q?=E4=BF=AE=E5=A4=8Dstring=E7=B1=BB?= =?UTF-8?q?=E5=9E=8B=E8=BD=AC=E4=B8=BAnull=E7=9A=84bug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java index fdce472e6d..ff50ff20e4 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java @@ -155,6 +155,8 @@ public static Object getWritableValue(Object writable) { ret = ((DateWritable) writable).get(); } else if(writable instanceof Writable) { ret = writable.toString(); + } else { + ret = writable.toString(); } return ret; From 23f9fb4be7807bac43efcf5767f5ba51b71a9987 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=B1=9F=E5=8D=9A=5F=E6=B1=9F=E5=8D=9A?= Date: Mon, 1 Apr 2019 10:29:41 +0800 Subject: [PATCH 61/96] Revert "Merge remote-tracking branch 'origin/1.5_v3.4.3' into 1.5_v3.5.1" This reverts commit a83da00d5e0c4e48eb4337d91f5c7173a16ec2e8 --- .../com/dtstack/flinkx/util/DateUtil.java | 2 +- .../com/dtstack/flinkx/es/EsConfigKeys.java | 2 -- .../java/com/dtstack/flinkx/es/EsUtil.java | 27 ++++++------------- .../flinkx/es/reader/EsInputFormat.java | 16 +++-------- .../es/reader/EsInputFormatBuilder.java | 17 ------------ .../dtstack/flinkx/es/reader/EsReader.java | 10 ------- 6 files changed, 13 insertions(+), 61 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java index fe1b07a625..dbd9dbc79a 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java @@ -123,7 +123,7 @@ public static java.sql.Timestamp columnToTimestamp(Object column,SimpleDateForma Long rawData = (Long) column; return new java.sql.Timestamp(getMillSecond(rawData.toString())); } else if (column instanceof java.sql.Date) { - return new java.sql.Timestamp(((java.sql.Date) column).getTime()); + return (java.sql.Timestamp) column; } else if(column instanceof Timestamp) { return (Timestamp) column; } else if(column instanceof Date) { diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java index fa8f8593fc..497da0b39a 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java @@ -34,8 +34,6 @@ public class EsConfigKeys { public static final String KEY_TYPE = "type"; - public static final String KEY_BATCH_SIZE = "batchSize"; - public static final String KEY_BULK_ACTION = "bulkAction"; public static final String KEY_COLUMN_NAME = "name"; diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java index f018f6d4ce..3c65db2821 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java @@ -31,7 +31,6 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.common.Strings; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.SearchHit; @@ -65,42 +64,32 @@ public static RestHighLevelClient getClient(String address) { return client; } - public static SearchResponse search(RestHighLevelClient client, String index, String type, String query, int from, int size) { - SearchRequest searchRequest = Strings.isNullOrEmpty(index) ? new SearchRequest() : new SearchRequest(index); - - if(!Strings.isNullOrEmpty(type)){ - searchRequest.types(type); - } - + public static SearchResponse search(RestHighLevelClient client, String query, int from, int size) { + SearchRequest searchRequest = new SearchRequest(); SearchSourceBuilder sourceBuilder = new SearchSourceBuilder(); sourceBuilder.from(from); - - if(size > 0){ - sourceBuilder.size(size); - } + sourceBuilder.size(size); if(StringUtils.isNotBlank(query)) { QueryBuilder qb = QueryBuilders.wrapperQuery(query); sourceBuilder.query(qb); + searchRequest.source(sourceBuilder); } - searchRequest.source(sourceBuilder); - try { return client.search(searchRequest); } catch (IOException e) { throw new RuntimeException(e); } - } - public static long searchCount(RestHighLevelClient client, String index, String type, String query) { - SearchResponse searchResponse = search(client, index, type, query, 0, 0); + public static long searchCount(RestHighLevelClient client, String query) { + SearchResponse searchResponse = search(client, query, 0, 0); return searchResponse.getHits().getTotalHits(); } - public static List> searchContent(RestHighLevelClient client, String index, String type, String query, int from, int size) { - SearchResponse searchResponse = search(client, index, type, query, from, size); + public static List> searchContent(RestHighLevelClient client, String query, int from, int size) { + SearchResponse searchResponse = search(client, query, from, size); SearchHits searchHits = searchResponse.getHits(); List> resultList = new ArrayList<>(); for(SearchHit searchHit : searchHits) { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java index 018a2e5e1f..e8548d4b59 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java @@ -44,10 +44,6 @@ public class EsInputFormat extends RichInputFormat { protected String address; - protected String index; - - protected String type; - protected String query; protected List columnValues; @@ -56,7 +52,7 @@ public class EsInputFormat extends RichInputFormat { protected List columnNames; - protected int batchSize = 10; + private int batch = 2; private int from; @@ -83,7 +79,7 @@ public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOExce @Override public InputSplit[] createInputSplits(int splitNum) throws IOException { - long cnt = EsUtil.searchCount(client, index, type, query); + long cnt = EsUtil.searchCount(client, query); if (cnt < splitNum) { EsInputSplit[] splits = new EsInputSplit[1]; splits[0] = new EsInputSplit(0, (int)cnt); @@ -123,11 +119,11 @@ public void openInternal(InputSplit inputSplit) throws IOException { } private void loadNextBatch() { - int range = batchSize; + int range = batch; if (from + range > to) { range = to - from; } - resultList = EsUtil.searchContent(client, index, type, query, from, range); + resultList = EsUtil.searchContent(client, query, from, range); from += range; pos = 0; } @@ -139,10 +135,6 @@ public boolean reachedEnd() throws IOException { return true; } loadNextBatch(); - - //check again - return reachedEnd(); - } return false; } diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java index 802aef3451..3f9a7be530 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java @@ -65,23 +65,6 @@ public EsInputFormatBuilder setColumnTypes(List columnTypes) { return this; } - public EsInputFormatBuilder setIndex(String index){ - format.index = index; - return this; - } - - public EsInputFormatBuilder setType(String type){ - format.type = type; - return this; - } - - public EsInputFormatBuilder setBatchSize(Integer batchSize){ - if(batchSize != null && batchSize > 0){ - format.batchSize = batchSize; - } - return this; - } - @Override protected void checkFormat() { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java index 58aa792da2..e4fc711d81 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java @@ -41,10 +41,6 @@ public class EsReader extends DataReader { private String address; private String query; - private String index; - private String type; - private Integer batchSize; - protected List columnType; protected List columnValue; protected List columnName; @@ -53,9 +49,6 @@ public EsReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); address = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_ADDRESS); - index = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_INDEX); - type = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_TYPE); - batchSize = readerConfig.getParameter().getIntVal(EsConfigKeys.KEY_BATCH_SIZE, 0); Object queryMap = readerConfig.getParameter().getVal(EsConfigKeys.KEY_QUERY); if(queryMap != null) { @@ -90,9 +83,6 @@ public DataStream readData() { builder.setColumnTypes(columnType); builder.setColumnValues(columnValue); builder.setAddress(address); - builder.setIndex(index); - builder.setType(type); - builder.setBatchSize(batchSize); builder.setQuery(query); builder.setBytes(bytes); builder.setMonitorUrls(monitorUrls); From 238b105be29829465cebe7b6ae0595a2172e236a Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 25 Mar 2019 10:14:39 +0800 Subject: [PATCH 62/96] =?UTF-8?q?=E4=BF=AE=E5=A4=8DDate=E7=B1=BB=E5=9E=8B?= =?UTF-8?q?=E8=BD=ACTimestamp=E6=8A=A5=E9=94=99?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java index dbd9dbc79a..fe1b07a625 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java @@ -123,7 +123,7 @@ public static java.sql.Timestamp columnToTimestamp(Object column,SimpleDateForma Long rawData = (Long) column; return new java.sql.Timestamp(getMillSecond(rawData.toString())); } else if (column instanceof java.sql.Date) { - return (java.sql.Timestamp) column; + return new java.sql.Timestamp(((java.sql.Date) column).getTime()); } else if(column instanceof Timestamp) { return (Timestamp) column; } else if(column instanceof Date) { From 5f8ac1d76d95f7f63802aa76897716afc81a6f4c Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 1 Apr 2019 10:43:35 +0800 Subject: [PATCH 63/96] Merge 1.5_v3.4.3 into 1.5_v3.5.2 --- .../com/dtstack/flinkx/es/EsConfigKeys.java | 2 ++ .../java/com/dtstack/flinkx/es/EsUtil.java | 27 +++++++++++++------ .../flinkx/es/reader/EsInputFormat.java | 16 ++++++++--- .../es/reader/EsInputFormatBuilder.java | 17 ++++++++++++ .../dtstack/flinkx/es/reader/EsReader.java | 10 +++++++ 5 files changed, 60 insertions(+), 12 deletions(-) diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java index 497da0b39a..fa8f8593fc 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java @@ -34,6 +34,8 @@ public class EsConfigKeys { public static final String KEY_TYPE = "type"; + public static final String KEY_BATCH_SIZE = "batchSize"; + public static final String KEY_BULK_ACTION = "bulkAction"; public static final String KEY_COLUMN_NAME = "name"; diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java index 3c65db2821..f018f6d4ce 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java @@ -31,6 +31,7 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.Strings; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.SearchHit; @@ -64,32 +65,42 @@ public static RestHighLevelClient getClient(String address) { return client; } - public static SearchResponse search(RestHighLevelClient client, String query, int from, int size) { - SearchRequest searchRequest = new SearchRequest(); + public static SearchResponse search(RestHighLevelClient client, String index, String type, String query, int from, int size) { + SearchRequest searchRequest = Strings.isNullOrEmpty(index) ? new SearchRequest() : new SearchRequest(index); + + if(!Strings.isNullOrEmpty(type)){ + searchRequest.types(type); + } + SearchSourceBuilder sourceBuilder = new SearchSourceBuilder(); sourceBuilder.from(from); - sourceBuilder.size(size); + + if(size > 0){ + sourceBuilder.size(size); + } if(StringUtils.isNotBlank(query)) { QueryBuilder qb = QueryBuilders.wrapperQuery(query); sourceBuilder.query(qb); - searchRequest.source(sourceBuilder); } + searchRequest.source(sourceBuilder); + try { return client.search(searchRequest); } catch (IOException e) { throw new RuntimeException(e); } + } - public static long searchCount(RestHighLevelClient client, String query) { - SearchResponse searchResponse = search(client, query, 0, 0); + public static long searchCount(RestHighLevelClient client, String index, String type, String query) { + SearchResponse searchResponse = search(client, index, type, query, 0, 0); return searchResponse.getHits().getTotalHits(); } - public static List> searchContent(RestHighLevelClient client, String query, int from, int size) { - SearchResponse searchResponse = search(client, query, from, size); + public static List> searchContent(RestHighLevelClient client, String index, String type, String query, int from, int size) { + SearchResponse searchResponse = search(client, index, type, query, from, size); SearchHits searchHits = searchResponse.getHits(); List> resultList = new ArrayList<>(); for(SearchHit searchHit : searchHits) { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java index e8548d4b59..018a2e5e1f 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java @@ -44,6 +44,10 @@ public class EsInputFormat extends RichInputFormat { protected String address; + protected String index; + + protected String type; + protected String query; protected List columnValues; @@ -52,7 +56,7 @@ public class EsInputFormat extends RichInputFormat { protected List columnNames; - private int batch = 2; + protected int batchSize = 10; private int from; @@ -79,7 +83,7 @@ public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOExce @Override public InputSplit[] createInputSplits(int splitNum) throws IOException { - long cnt = EsUtil.searchCount(client, query); + long cnt = EsUtil.searchCount(client, index, type, query); if (cnt < splitNum) { EsInputSplit[] splits = new EsInputSplit[1]; splits[0] = new EsInputSplit(0, (int)cnt); @@ -119,11 +123,11 @@ public void openInternal(InputSplit inputSplit) throws IOException { } private void loadNextBatch() { - int range = batch; + int range = batchSize; if (from + range > to) { range = to - from; } - resultList = EsUtil.searchContent(client, query, from, range); + resultList = EsUtil.searchContent(client, index, type, query, from, range); from += range; pos = 0; } @@ -135,6 +139,10 @@ public boolean reachedEnd() throws IOException { return true; } loadNextBatch(); + + //check again + return reachedEnd(); + } return false; } diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java index 3f9a7be530..802aef3451 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java @@ -65,6 +65,23 @@ public EsInputFormatBuilder setColumnTypes(List columnTypes) { return this; } + public EsInputFormatBuilder setIndex(String index){ + format.index = index; + return this; + } + + public EsInputFormatBuilder setType(String type){ + format.type = type; + return this; + } + + public EsInputFormatBuilder setBatchSize(Integer batchSize){ + if(batchSize != null && batchSize > 0){ + format.batchSize = batchSize; + } + return this; + } + @Override protected void checkFormat() { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java index e4fc711d81..58aa792da2 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java @@ -41,6 +41,10 @@ public class EsReader extends DataReader { private String address; private String query; + private String index; + private String type; + private Integer batchSize; + protected List columnType; protected List columnValue; protected List columnName; @@ -49,6 +53,9 @@ public EsReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); address = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_ADDRESS); + index = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_INDEX); + type = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_TYPE); + batchSize = readerConfig.getParameter().getIntVal(EsConfigKeys.KEY_BATCH_SIZE, 0); Object queryMap = readerConfig.getParameter().getVal(EsConfigKeys.KEY_QUERY); if(queryMap != null) { @@ -83,6 +90,9 @@ public DataStream readData() { builder.setColumnTypes(columnType); builder.setColumnValues(columnValue); builder.setAddress(address); + builder.setIndex(index); + builder.setType(type); + builder.setBatchSize(batchSize); builder.setQuery(query); builder.setBytes(bytes); builder.setMonitorUrls(monitorUrls); From 6e5c11abd430525bf26ca79523956fb1816a32d2 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 1 Apr 2019 13:58:16 +0800 Subject: [PATCH 64/96] =?UTF-8?q?=E4=BD=BF=E7=94=A8HttpClient=E4=BB=A3?= =?UTF-8?q?=E6=9B=BFUrl.open?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/inputformat/RichInputFormat.java | 10 +-- .../flinkx/outputformat/RichOutputFormat.java | 2 +- .../flinkx/reader/ByteRateLimiter.java | 81 ++++++++++--------- .../java/com/dtstack/flinkx/util/URLUtil.java | 28 +++++++ .../dtstack/flinkx/writer/ErrorLimiter.java | 48 ++++++----- .../rdb/inputformat/JdbcInputFormat.java | 64 ++++++++------- 6 files changed, 138 insertions(+), 95 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java index f7ca98edda..755f1d5142 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java @@ -23,7 +23,6 @@ import org.apache.commons.lang.StringUtils; import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; -import org.apache.flink.api.common.io.FinalizeOnMaster; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; @@ -41,7 +40,7 @@ * 用户只需覆盖openInternal,closeInternal等方法, 无需操心细节 * */ -public abstract class RichInputFormat extends org.apache.flink.api.common.io.RichInputFormat implements FinalizeOnMaster { +public abstract class RichInputFormat extends org.apache.flink.api.common.io.RichInputFormat { protected final Logger LOG = LoggerFactory.getLogger(getClass()); protected String jobName = "defaultJobName"; @@ -64,7 +63,7 @@ public void open(InputSplit inputSplit) throws IOException { openInternal(inputSplit); if (StringUtils.isNotBlank(this.monitorUrls) && this.bytes > 0) { - this.byteRateLimiter = new ByteRateLimiter(getRuntimeContext(), this.monitorUrls, this.bytes, 1); + this.byteRateLimiter = new ByteRateLimiter(getRuntimeContext(), this.monitorUrls, this.bytes, 2); this.byteRateLimiter.start(); } } @@ -101,11 +100,6 @@ public void close() throws IOException { protected abstract void closeInternal() throws IOException; - @Override - public void finalizeGlobal(int parallelism) throws IOException { - - } - @Override public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOException { return null; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java index d05e84013f..484ec61f0e 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/RichOutputFormat.java @@ -186,7 +186,7 @@ public void open(int taskNumber, int numTasks) throws IOException { //启动错误限制 if(StringUtils.isNotBlank(monitorUrl)) { if(errors != null || errorRatio != null) { - errorLimiter = new ErrorLimiter(context, monitorUrl, errors, errorRatio, 1); + errorLimiter = new ErrorLimiter(context, monitorUrl, errors, errorRatio, 2); errorLimiter.start(); } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java index 7ef854b918..9c76f70a8a 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java @@ -18,24 +18,19 @@ package com.dtstack.flinkx.reader; -import com.dtstack.flinkx.util.RetryUtil; import com.dtstack.flinkx.util.URLUtil; import com.google.common.util.concurrent.RateLimiter; import com.google.gson.Gson; import com.google.gson.internal.LinkedTreeMap; import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.hadoop.shaded.org.apache.http.impl.client.CloseableHttpClient; +import org.apache.flink.hadoop.shaded.org.apache.http.impl.client.HttpClientBuilder; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.Reader; -import java.net.URL; import java.util.List; import java.util.Map; -import java.util.concurrent.Callable; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -68,9 +63,12 @@ public class ByteRateLimiter { private int subtaskIndex; + private CloseableHttpClient httpClient; + private ScheduledExecutorService scheduledExecutorService; public ByteRateLimiter(RuntimeContext runtimeContext, String monitors, double expectedBytePerSecond, double samplePeriod) { + httpClient = HttpClientBuilder.create().build(); Preconditions.checkNotNull(runtimeContext); //DistributedRuntimeUDFContext context = (DistributedRuntimeUDFContext) runtimeContext; @@ -96,7 +94,8 @@ public ByteRateLimiter(RuntimeContext runtimeContext, String monitors, double ex for(; j < monitorUrls.length; ++j) { String url = monitorUrls[j]; LOG.info("monitor_url=" + url); - try (InputStream inputStream = URLUtil.open(url)){ + try { + URLUtil.open(httpClient, url); break; } catch (Exception e) { LOG.error("connected error: " + url); @@ -139,43 +138,41 @@ public void start() { () -> { for (int index = 0; index < 1; ++index) { String requestUrl = monitorUrls[index] + "/jobs/" + this.jobId + "/vertices/" + this.taskId; - try (InputStream inputStream = URLUtil.open(requestUrl)) { - try (Reader rd = new InputStreamReader(inputStream)) { - Map map = gson.fromJson(rd, Map.class); - double thisWriteBytes = 0; - double thisWriteRecords = 0; - double totalWriteBytes = 0; - double totalWriteRecords = 0; - - List list = (List) map.get("subtasks"); - for (int i = 0; i < list.size(); ++i) { - LinkedTreeMap subTask = list.get(i); - LinkedTreeMap subTaskMetrics = (LinkedTreeMap) subTask.get("metrics"); - double subWriteBytes = (double) subTaskMetrics.get("write-bytes"); - double subWriteRecords = (double) subTaskMetrics.get("write-records"); - if (i == subTaskIndex) { - thisWriteBytes = subWriteBytes; - thisWriteRecords = subWriteRecords; - } - totalWriteBytes += subWriteBytes; - totalWriteRecords += subWriteRecords; - } - - double thisWriteRatio = (totalWriteRecords == 0 ? 0 : thisWriteRecords / totalWriteRecords); - - if (totalWriteRecords > 1000 && totalWriteBytes != 0 && thisWriteRatio != 0) { - double bpr = totalWriteBytes / totalWriteRecords; - double permitsPerSecond = expectedBytePerSecond / bpr * thisWriteRatio; - rateLimiter.setRate(permitsPerSecond); + try { + String response = URLUtil.open(httpClient, requestUrl); + + Map map = gson.fromJson(response, Map.class); + double thisWriteBytes = 0; + double thisWriteRecords = 0; + double totalWriteBytes = 0; + double totalWriteRecords = 0; + + List list = (List) map.get("subtasks"); + for (int i = 0; i < list.size(); ++i) { + LinkedTreeMap subTask = list.get(i); + LinkedTreeMap subTaskMetrics = (LinkedTreeMap) subTask.get("metrics"); + double subWriteBytes = (double) subTaskMetrics.get("write-bytes"); + double subWriteRecords = (double) subTaskMetrics.get("write-records"); + if (i == subTaskIndex) { + thisWriteBytes = subWriteBytes; + thisWriteRecords = subWriteRecords; } + totalWriteBytes += subWriteBytes; + totalWriteRecords += subWriteRecords; + } - break; + double thisWriteRatio = (totalWriteRecords == 0 ? 0 : thisWriteRecords / totalWriteRecords); + if (totalWriteRecords > 1000 && totalWriteBytes != 0 && thisWriteRatio != 0) { + double bpr = totalWriteBytes / totalWriteRecords; + double permitsPerSecond = expectedBytePerSecond / bpr * thisWriteRatio; + rateLimiter.setRate(permitsPerSecond); } + + break; } catch (Exception e) { LOG.error("Get metrics error:",e); } - } }, 0, @@ -189,6 +186,14 @@ public void stop() { return; } + if (httpClient != null){ + try { + httpClient.close(); + } catch (Exception e){ + LOG.error("close httpClient error:{}", e); + } + } + if(scheduledExecutorService != null && !scheduledExecutorService.isShutdown()) { scheduledExecutorService.shutdown(); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/URLUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/URLUtil.java index b920b77fde..233ecd43bf 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/URLUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/URLUtil.java @@ -18,8 +18,16 @@ package com.dtstack.flinkx.util; +import org.apache.flink.hadoop.shaded.org.apache.http.HttpEntity; +import org.apache.flink.hadoop.shaded.org.apache.http.HttpStatus; +import org.apache.flink.hadoop.shaded.org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.flink.hadoop.shaded.org.apache.http.client.methods.HttpGet; +import org.apache.flink.hadoop.shaded.org.apache.http.impl.client.CloseableHttpClient; +import org.apache.flink.hadoop.shaded.org.apache.http.util.EntityUtils; + import java.io.InputStream; import java.net.URL; +import java.nio.charset.Charset; import java.util.concurrent.Callable; /** @@ -32,6 +40,8 @@ public class URLUtil { private static int SLEEP_TIME_MILLI_SECOND = 2000; + private static Charset charset = Charset.forName("UTF-8"); + public static InputStream open(String url) throws Exception{ return RetryUtil.executeWithRetry(new Callable() { @Override @@ -40,4 +50,22 @@ public InputStream call() throws Exception{ } },MAX_RETRY_TIMES,SLEEP_TIME_MILLI_SECOND,false); } + + public static String open(CloseableHttpClient httpClient, String url) throws Exception{ + return RetryUtil.executeWithRetry(new Callable() { + @Override + public String call() throws Exception{ + String respBody = null; + HttpGet httpGet = new HttpGet(url); + CloseableHttpResponse response = httpClient.execute(httpGet); + + if(response.getStatusLine().getStatusCode() == HttpStatus.SC_OK){ + HttpEntity entity = response.getEntity(); + respBody = EntityUtils.toString(entity,charset); + } + + return respBody; + } + },MAX_RETRY_TIMES,SLEEP_TIME_MILLI_SECOND,false); + } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java index e187f518d0..d9d67c980f 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java @@ -22,15 +22,13 @@ import com.google.gson.Gson; import com.google.gson.internal.LinkedTreeMap; import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.hadoop.shaded.org.apache.http.impl.client.CloseableHttpClient; +import org.apache.flink.hadoop.shaded.org.apache.http.impl.client.HttpClientBuilder; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.Reader; -import java.net.URL; import java.util.List; import java.util.Map; import java.util.concurrent.Executors; @@ -60,6 +58,8 @@ public class ErrorLimiter { private String errMsg = ""; private Row errorData; + private CloseableHttpClient httpClient; + public void setErrorData(Row errorData){ this.errorData = errorData; } @@ -73,10 +73,11 @@ public void setErrMsg(String errMsg) { } public ErrorLimiter(RuntimeContext runtimeContext, String monitors, int maxErrors, double samplePeriod) { - this(runtimeContext, monitors, maxErrors, Double.MAX_VALUE, 1); + this(runtimeContext, monitors, maxErrors, Double.MAX_VALUE, 2); } public ErrorLimiter(RuntimeContext runtimeContext, String monitors, Integer maxErrors, Double maxErrorRatio, double samplePeriod) { + httpClient = HttpClientBuilder.create().build(); Preconditions.checkArgument(runtimeContext != null || monitors != null, "Should specify rumtimeContext or monitorUrls"); Preconditions.checkArgument(samplePeriod > 0); @@ -102,7 +103,8 @@ public ErrorLimiter(RuntimeContext runtimeContext, String monitors, Integer maxE int j = 0; for(; j < monitorUrls.length; ++j) { String url = monitorUrls[j]; - try (InputStream inputStream = URLUtil.open(url)){ + try { + URLUtil.open(httpClient, url); break; } catch (Exception e) { LOG.error("connected error: " + url); @@ -123,7 +125,6 @@ public boolean isValid() { } public void start() { - if(scheduledExecutorService == null) { return; } @@ -140,22 +141,21 @@ public void updateErrorInfo(){ Gson gson = new Gson(); for(int index = 0; index < monitorUrls.length; ++index) { String requestUrl = monitorUrls[index] + "/jobs/" + jobId + "/accumulators"; - try(InputStream inputStream = URLUtil.open(requestUrl) ) { - try(Reader rd = new InputStreamReader(inputStream)) { - Map map = gson.fromJson(rd, Map.class); - List userTaskAccumulators = (List) map.get("user-task-accumulators"); - for(LinkedTreeMap accumulator : userTaskAccumulators) { - String name = (String) accumulator.get("name"); - if(name != null) { - if(name.equals("nErrors")) { - this.errors = Double.valueOf((String) accumulator.get("value")).intValue(); - } else if(name.equals("numRead")) { - this.numRead = Double.valueOf((String) accumulator.get("value")).intValue(); - } + try { + String response = URLUtil.open(httpClient, requestUrl); + Map map = gson.fromJson(response, Map.class); + List userTaskAccumulators = (List) map.get("user-task-accumulators"); + for(LinkedTreeMap accumulator : userTaskAccumulators) { + String name = (String) accumulator.get("name"); + if(name != null) { + if(name.equals("nErrors")) { + this.errors = Double.valueOf((String) accumulator.get("value")).intValue(); + } else if(name.equals("numRead")) { + this.numRead = Double.valueOf((String) accumulator.get("value")).intValue(); } } } - } catch (Exception e) { + } catch (Exception e){ LOG.error("Update data error:",e); } break; @@ -163,6 +163,14 @@ public void updateErrorInfo(){ } public void stop() { + if (httpClient != null){ + try { + httpClient.close(); + } catch (Exception e){ + LOG.error("close httpClient error:{}", e); + } + } + if(scheduledExecutorService != null && !scheduledExecutorService.isShutdown() && !scheduledExecutorService.isTerminated()) { scheduledExecutorService.shutdown(); } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 4a1894265c..6ef87ca993 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -37,6 +37,8 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; +import org.apache.flink.hadoop.shaded.org.apache.http.impl.client.CloseableHttpClient; +import org.apache.flink.hadoop.shaded.org.apache.http.impl.client.HttpClientBuilder; import org.apache.flink.types.Row; import java.io.IOException; import java.io.InputStream; @@ -322,50 +324,56 @@ private void getMaxValue(InputSplit inputSplit){ return; } - Map vars = getRuntimeContext().getMetricGroup().getAllVariables(); - String jobId = vars.get(""); + try (CloseableHttpClient httpClient = HttpClientBuilder.create().build()) { - String[] monitors; - if (monitorUrls.startsWith("http")){ - monitors = new String[] {String.format("%s/jobs/%s/accumulators", monitorUrls, jobId)}; - } else { - monitors = monitorUrls.split(","); - for (int i = 0; i < monitors.length; i++) { - monitors[i] = String.format("http://%s/jobs/%s/accumulators", monitors[i], jobId); + Map vars = getRuntimeContext().getMetricGroup().getAllVariables(); + String jobId = vars.get(""); + + String[] monitors; + if (monitorUrls.startsWith("http")) { + monitors = new String[]{String.format("%s/jobs/%s/accumulators", monitorUrls, jobId)}; + } else { + monitors = monitorUrls.split(","); + for (int i = 0; i < monitors.length; i++) { + monitors[i] = String.format("http://%s/jobs/%s/accumulators", monitors[i], jobId); + } } - } - /** - * The extra 10 times is to ensure that accumulator is updated - */ - int maxAcquireTimes = (queryTimeOut / requestAccumulatorInterval) + 10; + /** + * The extra 10 times is to ensure that accumulator is updated + */ + int maxAcquireTimes = (queryTimeOut / requestAccumulatorInterval) + 10; - int acquireTimes = 0; - while (StringUtils.isEmpty(maxValue) && acquireTimes < maxAcquireTimes){ - try { - Thread.sleep(requestAccumulatorInterval * 1000); - } catch (InterruptedException ignore) { - } + int acquireTimes = 0; + while (StringUtils.isEmpty(maxValue) && acquireTimes < maxAcquireTimes) { + try { + Thread.sleep(requestAccumulatorInterval * 1000); + } catch (InterruptedException ignore) { + } - maxValue = getMaxvalueFromAccumulator(monitors); - acquireTimes++; - } + maxValue = getMaxvalueFromAccumulator(httpClient, monitors); + acquireTimes++; + } - if (StringUtils.isEmpty(maxValue)){ - throw new RuntimeException("Can't get the max value from accumulator"); + if (StringUtils.isEmpty(maxValue)) { + throw new RuntimeException("Can't get the max value from accumulator"); + } + } catch (IOException e){ + throw new RuntimeException("Can't get the max value from accumulator:" + e); } } ((JdbcInputSplit) inputSplit).setEndLocation(maxValue); } - private String getMaxvalueFromAccumulator(String[] monitors){ + private String getMaxvalueFromAccumulator(CloseableHttpClient httpClient,String[] monitors){ String maxValue = null; Gson gson = new Gson(); for (String monitor : monitors) { LOG.info("Request url:" + monitor); - try (InputStream inputStream = URLUtil.open(monitor); Reader rd = new InputStreamReader(inputStream)) { - Map map = gson.fromJson(rd, Map.class); + try { + String response = URLUtil.open(httpClient, monitor); + Map map = gson.fromJson(response, Map.class); LOG.info("Accumulator data:" + gson.toJson(map)); From 36ec6e5e99667176760e5ce76fd437ed952bd4fb Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 1 Apr 2019 14:05:29 +0800 Subject: [PATCH 65/96] =?UTF-8?q?=E5=A4=9A=E9=80=9A=E9=81=93=E4=B8=8D?= =?UTF-8?q?=E6=8C=87=E5=AE=9A=E5=88=87=E5=88=86=E9=94=AE=E6=97=B6=E6=8A=9B?= =?UTF-8?q?=E5=87=BA=E5=BC=82=E5=B8=B8?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java | 8 -------- .../flinkx/rdb/inputformat/JdbcInputFormatBuilder.java | 5 +++++ 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 7d676a351a..8e2fef9189 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -378,14 +378,6 @@ private String getMaxvalueFromAccumulator(String[] monitors){ } private boolean canReadData(InputSplit split){ - /* - * If the user set multiple channels but does not specify a splitKey, - * the data is read only on the first channel to prevent data duplication - */ - if (numPartitions > 1 && StringUtils.isEmpty(splitKey) && getRuntimeContext().getIndexOfThisSubtask() > 0){ - return false; - } - if (StringUtils.isEmpty(increCol)){ return true; } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java index 240c3116d9..f1e4a55f39 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java @@ -22,6 +22,7 @@ import com.dtstack.flinkx.rdb.DatabaseInterface; import com.dtstack.flinkx.rdb.type.TypeConverterInterface; import com.dtstack.flinkx.reader.MetaColumn; +import org.apache.commons.lang.StringUtils; import java.util.List; @@ -129,6 +130,10 @@ protected void checkFormat() { if (format.drivername == null) { throw new IllegalArgumentException("No driver supplied"); } + + if (StringUtils.isEmpty(format.splitKey) && format.numPartitions > 1){ + throw new IllegalArgumentException("Must specify the split column when the channel is greater than 1"); + } } } From 674d8c0197f4d072ec91eb20fbdb6873da00b9db Mon Sep 17 00:00:00 2001 From: toutian Date: Mon, 1 Apr 2019 15:07:48 +0800 Subject: [PATCH 66/96] put data to dfs --- .../flinkx/inputformat/RichInputFormat.java | 4 +- .../com/dtstack/flinkx/reader/DataReader.java | 15 ++++++ .../flinkx/rdb/datareader/JdbcDataReader.java | 1 + .../rdb/inputformat/JdbcInputFormat.java | 49 ++++++++++++++++--- .../inputformat/JdbcInputFormatBuilder.java | 5 ++ 5 files changed, 66 insertions(+), 8 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java index a8eb1588b7..2bf982d372 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/RichInputFormat.java @@ -93,11 +93,11 @@ public Row nextRecord(Row row) throws IOException { @Override public void close() throws IOException { try{ + closeInternal(); + if (inputMetric.getDelayPeriodMill() != 0){ SysUtil.sleep(inputMetric.getDelayPeriodMill()); } - - closeInternal(); }catch (Exception e){ throw new RuntimeException(e); }finally { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReader.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReader.java index 8deb5e9b8c..5c31c8b861 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReader.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReader.java @@ -19,6 +19,7 @@ package com.dtstack.flinkx.reader; import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.DirtyConfig; import com.dtstack.flinkx.plugin.PluginLoader; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -28,8 +29,10 @@ import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; + import java.util.ArrayList; import java.util.List; +import java.util.Map; /** * Abstract specification of Reader Plugin @@ -51,6 +54,10 @@ public abstract class DataReader { protected List srcCols = new ArrayList<>(); + /** + * reuse hadoopConfig for metric + */ + protected Map hadoopConfig; public List getSrcCols() { return srcCols; @@ -76,6 +83,14 @@ protected DataReader(DataTransferConfig config, StreamExecutionEnvironment env) this.numPartitions = config.getJob().getSetting().getSpeed().getChannel(); this.bytes = config.getJob().getSetting().getSpeed().getBytes(); this.monitorUrls = config.getMonitorUrls(); + + DirtyConfig dirtyConfig = config.getJob().getSetting().getDirty(); + if (dirtyConfig != null) { + Map hadoopConfig = dirtyConfig.getHadoopConfig(); + if (hadoopConfig != null) { + this.hadoopConfig = hadoopConfig; + } + } } public abstract DataStream readData(); diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java index f3c2f6504e..1fa0705c6d 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java @@ -110,6 +110,7 @@ public DataStream readData() { builder.setQueryTimeOut(queryTimeOut == 0 ? databaseInterface.getQueryTimeout() : queryTimeOut); builder.setIncreCol(increColumn); builder.setStartLocation(startLocation); + builder.setHadoopConfig(hadoopConfig); boolean isSplitByKey = false; if(numPartitions > 1 && splitKey != null && splitKey.trim().length() != 0) { diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index ff7273944d..7c3b9df1fc 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -21,7 +21,6 @@ import com.dtstack.flinkx.common.ColumnType; import com.dtstack.flinkx.constants.Metrics; import com.dtstack.flinkx.enums.EDatabaseType; -import com.dtstack.flinkx.metrics.SimpleAccumulatorGauge; import com.dtstack.flinkx.rdb.DatabaseInterface; import com.dtstack.flinkx.rdb.type.TypeConverterInterface; import com.dtstack.flinkx.rdb.util.DBUtil; @@ -37,7 +36,6 @@ import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; -import org.apache.flink.metrics.MetricGroup; import org.apache.flink.types.Row; import java.io.IOException; import java.sql.*; @@ -45,6 +43,12 @@ import java.util.Date; import com.dtstack.flinkx.inputformat.RichInputFormat; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.io.IOUtils; +import org.codehaus.jackson.map.ObjectMapper; /** * InputFormat for reading data from a database and generate Rows. @@ -110,6 +114,11 @@ public class JdbcInputFormat extends RichInputFormat { protected StringAccumulator startLocationAccumulator; + /** + * The hadoop config for metric + */ + protected Map hadoopConfig; + public JdbcInputFormat() { resultSetType = ResultSet.TYPE_FORWARD_ONLY; resultSetConcurrency = ResultSet.CONCUR_READ_ONLY; @@ -122,19 +131,16 @@ public void configure(Configuration configuration) { private void setMetric(){ Map> accumulatorMap = getRuntimeContext().getAllAccumulators(); - final MetricGroup flinkxOutputMetricGroup = getRuntimeContext().getMetricGroup().addGroup(Metrics.METRIC_GROUP_KEY_FLINKX, Metrics.METRIC_GROUP_VALUE_OUTPUT); if(!accumulatorMap.containsKey(Metrics.TABLE_COL)){ tableColAccumulator = new StringAccumulator(); tableColAccumulator.add(table + "-" + increCol); getRuntimeContext().addAccumulator(Metrics.TABLE_COL,tableColAccumulator); - flinkxOutputMetricGroup.gauge(Metrics.TABLE_COL, new SimpleAccumulatorGauge(tableColAccumulator)); } if(!accumulatorMap.containsKey(Metrics.END_LOCATION)){ endLocationAccumulator = new MaximumAccumulator(); getRuntimeContext().addAccumulator(Metrics.END_LOCATION,endLocationAccumulator); - flinkxOutputMetricGroup.gauge(Metrics.END_LOCATION, new SimpleAccumulatorGauge(endLocationAccumulator)); } if (startLocation != null){ @@ -143,7 +149,6 @@ private void setMetric(){ startLocationAccumulator = new StringAccumulator(); startLocationAccumulator.add(startLocation); getRuntimeContext().addAccumulator(Metrics.START_LOCATION,startLocationAccumulator); - flinkxOutputMetricGroup.gauge(Metrics.START_LOCATION, new SimpleAccumulatorGauge(startLocationAccumulator)); } } @@ -299,8 +304,40 @@ private long getLocation(Object increVal){ } } + private void uploadMetricData() throws IOException { + FSDataOutputStream out = null; + try { + org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration(); + + if(hadoopConfig != null) { + for (Map.Entry entry : hadoopConfig.entrySet()) { + conf.set(entry.getKey(), entry.getValue()); + } + } + + Map vars = getRuntimeContext().getMetricGroup().getAllVariables(); + String jobId = vars.get(""); + String taskId = vars.get(""); + String subtaskIndex = vars.get(""); + LOG.info("jobId:{} taskId:{} subtaskIndex:{}", jobId, taskId, subtaskIndex); + Path remotePath = new Path(conf.get("fs.defaultFS"), "/tmp/logs/admin/logs/" + jobId+"_"+taskId+"_"+subtaskIndex); + out = FileSystem.create(remotePath.getFileSystem(conf), remotePath, new FsPermission(FsPermission.createImmutable((short) 0777))); + + Map metrics = new HashMap<>(3); + metrics.put(Metrics.TABLE_COL, table + "-" + increCol); + metrics.put(Metrics.START_LOCATION, startLocationAccumulator.getLocalValue()); + metrics.put(Metrics.END_LOCATION, endLocationAccumulator.getLocalValue()); + out.writeUTF(new ObjectMapper().writeValueAsString(metrics)); + } finally { + IOUtils.closeStream(out); + } + } + @Override public void closeInternal() throws IOException { + if(increCol != null && hadoopConfig != null) { + uploadMetricData(); + } DBUtil.closeDBResources(resultSet,statement,dbConn); parameterValues = null; } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java index d2abbb4819..e021a65c43 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java @@ -24,6 +24,7 @@ import com.dtstack.flinkx.reader.MetaColumn; import java.util.List; +import java.util.Map; /** * The builder of JdbcInputFormat @@ -100,6 +101,10 @@ public void setIncreColType(String increColType){ format.increColType = increColType; } + public void setHadoopConfig(Map dirtyHadoopConfig) { + format.hadoopConfig = dirtyHadoopConfig; + } + @Override protected void checkFormat() { if (format.username == null) { From ca7d067891216241009ba27bf8add0198bf5a28d Mon Sep 17 00:00:00 2001 From: toutian Date: Mon, 1 Apr 2019 16:16:47 +0800 Subject: [PATCH 67/96] fix close error --- .../src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java index 5c89bb0173..5f70f4eb3c 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java @@ -170,7 +170,7 @@ public static void closeDBResources(ResultSet rs, Statement stmt, public static void commit(Connection conn){ try { - if (!conn.getAutoCommit() && !conn.isClosed()){ + if (!conn.isClosed() && !conn.getAutoCommit()){ LOG.info("Start commit connection"); conn.commit(); LOG.info("Commit connection successful"); From 58aa55ba281e6732de8e8a96f9f94b876da809ef Mon Sep 17 00:00:00 2001 From: toutian Date: Mon, 1 Apr 2019 18:17:16 +0800 Subject: [PATCH 68/96] test hdfs metric ok --- .../flinkx/rdb/inputformat/JdbcInputFormat.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 83b4758373..83759ecd13 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -526,8 +526,12 @@ private void uploadMetricData() throws IOException { Map metrics = new HashMap<>(3); metrics.put(Metrics.TABLE_COL, table + "-" + increCol); - metrics.put(Metrics.START_LOCATION, startLocationAccumulator.getLocalValue()); - metrics.put(Metrics.END_LOCATION, endLocationAccumulator.getLocalValue()); + if (startLocationAccumulator != null){ + metrics.put(Metrics.START_LOCATION, startLocationAccumulator.getLocalValue()); + } + if (endLocationAccumulator != null){ + metrics.put(Metrics.END_LOCATION, endLocationAccumulator.getLocalValue()); + } out.writeUTF(new ObjectMapper().writeValueAsString(metrics)); } finally { IOUtils.closeStream(out); @@ -536,7 +540,7 @@ private void uploadMetricData() throws IOException { @Override public void closeInternal() throws IOException { - if(increCol != null && hadoopConfig != null) { + if(StringUtils.isNotEmpty(increCol) && hadoopConfig != null) { uploadMetricData(); } DBUtil.closeDBResources(resultSet,statement,dbConn); From bef0ff7fad3eff2dc9abdbc5463f55e73edfd365 Mon Sep 17 00:00:00 2001 From: toutian Date: Tue, 2 Apr 2019 09:59:17 +0800 Subject: [PATCH 69/96] path --- .../com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 83759ecd13..1a92816cb5 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -521,7 +521,8 @@ private void uploadMetricData() throws IOException { String taskId = vars.get(""); String subtaskIndex = vars.get(""); LOG.info("jobId:{} taskId:{} subtaskIndex:{}", jobId, taskId, subtaskIndex); - Path remotePath = new Path(conf.get("fs.defaultFS"), "/tmp/logs/admin/logs/" + jobId+"_"+taskId+"_"+subtaskIndex); + + Path remotePath = new Path(conf.get("fs.defaultFS"), "/tmp/logs/admin/logs/" + jobId + "/" + taskId + "_" + subtaskIndex); out = FileSystem.create(remotePath.getFileSystem(conf), remotePath, new FsPermission(FsPermission.createImmutable((short) 0777))); Map metrics = new HashMap<>(3); From 8a858d86b6173b049ab98fcfb4c61fd934104572 Mon Sep 17 00:00:00 2001 From: toutian Date: Tue, 2 Apr 2019 11:13:43 +0800 Subject: [PATCH 70/96] test interval --- .../src/main/java/com/dtstack/flinkx/metrics/InputMetric.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java index 0fecc85b34..f052f86a3a 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/InputMetric.java @@ -47,7 +47,7 @@ public class InputMetric { private final static Long DEFAULT_PERIOD_MILLISECONDS = 10000L; - private Long delayPeriodMill = 20000L; + private Long delayPeriodMill = 12000L; public InputMetric(RuntimeContext runtimeContext, LongCounter numRead) { this.runtimeContext = runtimeContext; @@ -96,7 +96,7 @@ public void initPeriod() { LOG.info("InputMetric.scheduledFutureTask.schedulePeriodMill:{} ...", schedulePeriodMill); if (schedulePeriodMill > DEFAULT_PERIOD_MILLISECONDS) { - this.delayPeriodMill = schedulePeriodMill * 2; + this.delayPeriodMill = (long) (schedulePeriodMill * 1.2); } } catch (Exception e) { LOG.error("{}", e); From 74943352bc9392337a60a79fa3009b9a0a21bc3e Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 8 Apr 2019 20:36:32 +0800 Subject: [PATCH 71/96] =?UTF-8?q?=E6=B7=BB=E5=8A=A0git.branch?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-core/pom.xml | 2 +- flinkx-db2/flinkx-db2-reader/pom.xml | 2 +- flinkx-db2/flinkx-db2-writer/pom.xml | 2 +- flinkx-es/flinkx-es-reader/pom.xml | 2 +- flinkx-es/flinkx-es-writer/pom.xml | 2 +- flinkx-ftp/flinkx-ftp-reader/pom.xml | 2 +- flinkx-ftp/flinkx-ftp-writer/pom.xml | 2 +- flinkx-hbase/flinkx-hbase-reader/pom.xml | 2 +- flinkx-hbase/flinkx-hbase-writer/pom.xml | 2 +- flinkx-hdfs/flinkx-hdfs-reader/pom.xml | 2 +- flinkx-hdfs/flinkx-hdfs-writer/pom.xml | 2 +- flinkx-mongodb/flinkx-mongodb-reader/pom.xml | 2 +- flinkx-mongodb/flinkx-mongodb-writer/pom.xml | 2 +- flinkx-mysql/flinkx-mysql-dreader/pom.xml | 2 +- flinkx-mysql/flinkx-mysql-reader/pom.xml | 2 +- flinkx-mysql/flinkx-mysql-writer/pom.xml | 2 +- flinkx-odps/flinkx-odps-reader/pom.xml | 2 +- flinkx-odps/flinkx-odps-writer/pom.xml | 2 +- flinkx-oracle/flinkx-oracle-reader/pom.xml | 2 +- flinkx-oracle/flinkx-oracle-writer/pom.xml | 2 +- .../flinkx-postgresql-reader/pom.xml | 2 +- .../flinkx-postgresql-writer/pom.xml | 2 +- flinkx-rdb/pom.xml | 2 +- flinkx-redis/flinkx-redis-writer/pom.xml | 2 +- .../flinkx-sqlserver-reader/pom.xml | 2 +- .../flinkx-sqlserver-writer/pom.xml | 2 +- flinkx-stream/flinkx-stream-reader/pom.xml | 2 +- flinkx-stream/flinkx-stream-writer/pom.xml | 2 +- pom.xml | 22 +++++++++++++++++++ 29 files changed, 50 insertions(+), 28 deletions(-) diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index 161f3e2eed..a9e77e590f 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -159,7 +159,7 @@ + tofile="${basedir}/../plugins/flinkx-${git.branch}.jar" /> diff --git a/flinkx-db2/flinkx-db2-reader/pom.xml b/flinkx-db2/flinkx-db2-reader/pom.xml index a2c7d2f925..4fc55afc97 100644 --- a/flinkx-db2/flinkx-db2-reader/pom.xml +++ b/flinkx-db2/flinkx-db2-reader/pom.xml @@ -68,7 +68,7 @@ + tofile="${basedir}/../../plugins/db2reader/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-db2/flinkx-db2-writer/pom.xml b/flinkx-db2/flinkx-db2-writer/pom.xml index 8b2f7ffabd..49c90730a4 100644 --- a/flinkx-db2/flinkx-db2-writer/pom.xml +++ b/flinkx-db2/flinkx-db2-writer/pom.xml @@ -68,7 +68,7 @@ + tofile="${basedir}/../../plugins/db2writer/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-es/flinkx-es-reader/pom.xml b/flinkx-es/flinkx-es-reader/pom.xml index 40c0cc78c5..9f617b5854 100644 --- a/flinkx-es/flinkx-es-reader/pom.xml +++ b/flinkx-es/flinkx-es-reader/pom.xml @@ -74,7 +74,7 @@ + tofile="${basedir}/../../plugins/esreader/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-es/flinkx-es-writer/pom.xml b/flinkx-es/flinkx-es-writer/pom.xml index 759c6a16fe..885ec9737c 100644 --- a/flinkx-es/flinkx-es-writer/pom.xml +++ b/flinkx-es/flinkx-es-writer/pom.xml @@ -75,7 +75,7 @@ + tofile="${basedir}/../../plugins/eswriter/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-ftp/flinkx-ftp-reader/pom.xml b/flinkx-ftp/flinkx-ftp-reader/pom.xml index fee2388531..7ca313e801 100644 --- a/flinkx-ftp/flinkx-ftp-reader/pom.xml +++ b/flinkx-ftp/flinkx-ftp-reader/pom.xml @@ -95,7 +95,7 @@ under the License. + tofile="${basedir}/../../plugins/ftpreader/${project.name}-${git.branch}.jar"/> diff --git a/flinkx-ftp/flinkx-ftp-writer/pom.xml b/flinkx-ftp/flinkx-ftp-writer/pom.xml index 15297d5c45..2022d3ead1 100644 --- a/flinkx-ftp/flinkx-ftp-writer/pom.xml +++ b/flinkx-ftp/flinkx-ftp-writer/pom.xml @@ -94,7 +94,7 @@ under the License. + tofile="${basedir}/../../plugins/ftpwriter/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-hbase/flinkx-hbase-reader/pom.xml b/flinkx-hbase/flinkx-hbase-reader/pom.xml index ada607a2f2..e7f06daac8 100644 --- a/flinkx-hbase/flinkx-hbase-reader/pom.xml +++ b/flinkx-hbase/flinkx-hbase-reader/pom.xml @@ -92,7 +92,7 @@ + tofile="${basedir}/../../plugins/hbasereader/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-hbase/flinkx-hbase-writer/pom.xml b/flinkx-hbase/flinkx-hbase-writer/pom.xml index e7ce7428a3..361bdcb4b8 100644 --- a/flinkx-hbase/flinkx-hbase-writer/pom.xml +++ b/flinkx-hbase/flinkx-hbase-writer/pom.xml @@ -79,7 +79,7 @@ + tofile="${basedir}/../../plugins/hbasewriter/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-hdfs/flinkx-hdfs-reader/pom.xml b/flinkx-hdfs/flinkx-hdfs-reader/pom.xml index a60bb14ca9..afb0f02ac3 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-reader/pom.xml @@ -118,7 +118,7 @@ under the License. + tofile="${basedir}/../../plugins/hdfsreader/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml index de7c514b09..94c88df44b 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml @@ -119,7 +119,7 @@ under the License. + tofile="${basedir}/../../plugins/hdfswriter/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml index ab40ec8100..7370b820ef 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml @@ -66,7 +66,7 @@ + tofile="${basedir}/../../plugins/mongodbreader/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml index 95035030f1..13ea9450a5 100644 --- a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml @@ -66,7 +66,7 @@ + tofile="${basedir}/../../plugins/mongodbwriter/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-mysql/flinkx-mysql-dreader/pom.xml b/flinkx-mysql/flinkx-mysql-dreader/pom.xml index c15ff9cf26..5298a2fd9a 100644 --- a/flinkx-mysql/flinkx-mysql-dreader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-dreader/pom.xml @@ -68,7 +68,7 @@ + tofile="${basedir}/../../plugins/mysqldreader/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-mysql/flinkx-mysql-reader/pom.xml b/flinkx-mysql/flinkx-mysql-reader/pom.xml index 4a73e6b7e9..ee232417c5 100644 --- a/flinkx-mysql/flinkx-mysql-reader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-reader/pom.xml @@ -68,7 +68,7 @@ + tofile="${basedir}/../../plugins/mysqlreader/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-mysql/flinkx-mysql-writer/pom.xml b/flinkx-mysql/flinkx-mysql-writer/pom.xml index f4cb55dac2..e4275ce0e6 100644 --- a/flinkx-mysql/flinkx-mysql-writer/pom.xml +++ b/flinkx-mysql/flinkx-mysql-writer/pom.xml @@ -68,7 +68,7 @@ + tofile="${basedir}/../../plugins/mysqlwriter/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-odps/flinkx-odps-reader/pom.xml b/flinkx-odps/flinkx-odps-reader/pom.xml index 2b8d9d968a..7fcb63629c 100644 --- a/flinkx-odps/flinkx-odps-reader/pom.xml +++ b/flinkx-odps/flinkx-odps-reader/pom.xml @@ -66,7 +66,7 @@ + tofile="${basedir}/../../plugins/odpsreader/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-odps/flinkx-odps-writer/pom.xml b/flinkx-odps/flinkx-odps-writer/pom.xml index 1e65017236..7b7fc17abc 100644 --- a/flinkx-odps/flinkx-odps-writer/pom.xml +++ b/flinkx-odps/flinkx-odps-writer/pom.xml @@ -79,7 +79,7 @@ + tofile="${basedir}/../../plugins/odpswriter/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-oracle/flinkx-oracle-reader/pom.xml b/flinkx-oracle/flinkx-oracle-reader/pom.xml index 7b096de29c..675275e8d6 100644 --- a/flinkx-oracle/flinkx-oracle-reader/pom.xml +++ b/flinkx-oracle/flinkx-oracle-reader/pom.xml @@ -68,7 +68,7 @@ + tofile="${basedir}/../../plugins/oraclereader/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-oracle/flinkx-oracle-writer/pom.xml b/flinkx-oracle/flinkx-oracle-writer/pom.xml index d717a2094d..eca5ebdd39 100644 --- a/flinkx-oracle/flinkx-oracle-writer/pom.xml +++ b/flinkx-oracle/flinkx-oracle-writer/pom.xml @@ -68,7 +68,7 @@ + tofile="${basedir}/../../plugins/oraclewriter/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml index 412449f844..46f159e0cf 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml @@ -69,7 +69,7 @@ + tofile="${basedir}/../../plugins/postgresqlreader/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml index 5dddde0f65..423f100eb9 100644 --- a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml @@ -68,7 +68,7 @@ + tofile="${basedir}/../../plugins/postgresqlwriter/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-rdb/pom.xml b/flinkx-rdb/pom.xml index 993958e2c6..b9e273bde3 100644 --- a/flinkx-rdb/pom.xml +++ b/flinkx-rdb/pom.xml @@ -41,7 +41,7 @@ + tofile="${basedir}/../plugins/common/${project.name}-${git.branch}.jar"/> diff --git a/flinkx-redis/flinkx-redis-writer/pom.xml b/flinkx-redis/flinkx-redis-writer/pom.xml index 233c3b032a..0417a7cd91 100644 --- a/flinkx-redis/flinkx-redis-writer/pom.xml +++ b/flinkx-redis/flinkx-redis-writer/pom.xml @@ -68,7 +68,7 @@ + tofile="${basedir}/../../plugins/rediswriter/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml index c2420e7b74..5c840aad11 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml @@ -68,7 +68,7 @@ + tofile="${basedir}/../../plugins/sqlserverreader/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml index 95afbc5477..dfd68e834a 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml @@ -68,7 +68,7 @@ + tofile="${basedir}/../../plugins/sqlserverwriter/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-stream/flinkx-stream-reader/pom.xml b/flinkx-stream/flinkx-stream-reader/pom.xml index 888220e57b..cf0954e678 100644 --- a/flinkx-stream/flinkx-stream-reader/pom.xml +++ b/flinkx-stream/flinkx-stream-reader/pom.xml @@ -68,7 +68,7 @@ + tofile="${basedir}/../../plugins/streamreader/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-stream/flinkx-stream-writer/pom.xml b/flinkx-stream/flinkx-stream-writer/pom.xml index e524de8b97..57065c8e22 100644 --- a/flinkx-stream/flinkx-stream-writer/pom.xml +++ b/flinkx-stream/flinkx-stream-writer/pom.xml @@ -64,7 +64,7 @@ + tofile="${basedir}/../../plugins/streamwriter/${project.name}-${git.branch}.jar" /> diff --git a/pom.xml b/pom.xml index 32f7a14d6b..e7d1b578e0 100644 --- a/pom.xml +++ b/pom.xml @@ -70,6 +70,28 @@ flinkx-java-docs + + pl.project13.maven + git-commit-id-plugin + 2.2.6 + + + + revision + + + + + yyyy.MM.dd HH:mm:ss + true + true + + false + -dirty + false + + + From eaa009195a96b2ff76e849ca5aa6a2aeda4af127 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 9 Apr 2019 17:42:57 +0800 Subject: [PATCH 72/96] rename method name --- .../main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java | 4 ++-- .../src/main/java/com/dtstack/flinkx/util/URLUtil.java | 2 +- .../src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java | 4 ++-- .../com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java index 9c76f70a8a..9b2c2bad28 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java @@ -95,7 +95,7 @@ public ByteRateLimiter(RuntimeContext runtimeContext, String monitors, double ex String url = monitorUrls[j]; LOG.info("monitor_url=" + url); try { - URLUtil.open(httpClient, url); + URLUtil.get(httpClient, url); break; } catch (Exception e) { LOG.error("connected error: " + url); @@ -139,7 +139,7 @@ public void start() { for (int index = 0; index < 1; ++index) { String requestUrl = monitorUrls[index] + "/jobs/" + this.jobId + "/vertices/" + this.taskId; try { - String response = URLUtil.open(httpClient, requestUrl); + String response = URLUtil.get(httpClient, requestUrl); Map map = gson.fromJson(response, Map.class); double thisWriteBytes = 0; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/URLUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/URLUtil.java index 233ecd43bf..04ef38c524 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/URLUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/URLUtil.java @@ -51,7 +51,7 @@ public InputStream call() throws Exception{ },MAX_RETRY_TIMES,SLEEP_TIME_MILLI_SECOND,false); } - public static String open(CloseableHttpClient httpClient, String url) throws Exception{ + public static String get(CloseableHttpClient httpClient, String url) throws Exception{ return RetryUtil.executeWithRetry(new Callable() { @Override public String call() throws Exception{ diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java index d9d67c980f..3a4c6d654e 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java @@ -104,7 +104,7 @@ public ErrorLimiter(RuntimeContext runtimeContext, String monitors, Integer maxE for(; j < monitorUrls.length; ++j) { String url = monitorUrls[j]; try { - URLUtil.open(httpClient, url); + URLUtil.get(httpClient, url); break; } catch (Exception e) { LOG.error("connected error: " + url); @@ -142,7 +142,7 @@ public void updateErrorInfo(){ for(int index = 0; index < monitorUrls.length; ++index) { String requestUrl = monitorUrls[index] + "/jobs/" + jobId + "/accumulators"; try { - String response = URLUtil.open(httpClient, requestUrl); + String response = URLUtil.get(httpClient, requestUrl); Map map = gson.fromJson(response, Map.class); List userTaskAccumulators = (List) map.get("user-task-accumulators"); for(LinkedTreeMap accumulator : userTaskAccumulators) { diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 6ef87ca993..910288f320 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -372,7 +372,7 @@ private String getMaxvalueFromAccumulator(CloseableHttpClient httpClient,String[ for (String monitor : monitors) { LOG.info("Request url:" + monitor); try { - String response = URLUtil.open(httpClient, monitor); + String response = URLUtil.get(httpClient, monitor); Map map = gson.fromJson(response, Map.class); LOG.info("Accumulator data:" + gson.toJson(map)); From 88c14f0a9bfbd3065f0bdbac04ae98cb1d63ff87 Mon Sep 17 00:00:00 2001 From: toutian Date: Tue, 9 Apr 2019 19:33:31 +0800 Subject: [PATCH 73/96] =?UTF-8?q?=E8=A7=A3=E5=86=B3carbon=E5=88=86?= =?UTF-8?q?=E5=8C=BA=E8=A1=A8=E7=BC=BA=E5=B0=91=E5=88=86=E5=8C=BA=E5=AD=97?= =?UTF-8?q?=E6=AE=B5=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/carbondata/writer/CarbonOutputFormat.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbonOutputFormat.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbonOutputFormat.java index 8b9d4807e9..2fbac5852f 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbonOutputFormat.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbonOutputFormat.java @@ -106,12 +106,8 @@ public void configure(Configuration parameters) { } private void parsePartition(){ - if(carbonTable.getPartitionInfo() == null) { - return; - } - if(partition == null || partition.trim().length() == 0) { - return; + throw new IllegalArgumentException("The table have partition field,'partition' should not be empty"); } partition = partition.trim(); From 6bb513c797be7ce4fc027f4cb1f0a13d474cecf5 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 9 Apr 2019 16:47:50 +0800 Subject: [PATCH 74/96] =?UTF-8?q?es=E8=BF=9E=E6=8E=A5=E5=8F=82=E6=95=B0?= =?UTF-8?q?=E5=8F=AF=E9=85=8D=E7=BD=AE?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flinkx/es/EsConfigKeys.java | 4 ++++ .../java/com/dtstack/flinkx/es/EsUtil.java | 20 ++++++++++++++++--- .../flinkx/es/reader/EsInputFormat.java | 4 +++- .../es/reader/EsInputFormatBuilder.java | 5 +++++ .../dtstack/flinkx/es/reader/EsReader.java | 7 +++++++ .../flinkx/es/writer/EsOutputFormat.java | 5 ++++- .../es/writer/EsOutputFormatBuilder.java | 5 +++++ .../dtstack/flinkx/es/writer/EsWriter.java | 8 ++++++++ 8 files changed, 53 insertions(+), 5 deletions(-) diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java index fa8f8593fc..30d33608bc 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java @@ -50,4 +50,8 @@ public class EsConfigKeys { public static final String KEY_ID_COLUMN_VALUE = "value"; + public static final String KEY_TIMEOUT = "timeout"; + + public static final String KEY_PATH_PREFIX = "pathPrefix"; + } diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java index f018f6d4ce..5c0758e38d 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java @@ -22,6 +22,7 @@ import com.dtstack.flinkx.util.DateUtil; import com.dtstack.flinkx.util.StringUtil; import com.dtstack.flinkx.util.TelnetUtil; +import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.math.NumberUtils; import org.apache.flink.types.Row; @@ -30,6 +31,7 @@ import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.common.Strings; import org.elasticsearch.index.query.QueryBuilder; @@ -52,7 +54,7 @@ */ public class EsUtil { - public static RestHighLevelClient getClient(String address) { + public static RestHighLevelClient getClient(String address,Map config) { List httpHostList = new ArrayList<>(); String[] addr = address.split(","); for(String add : addr) { @@ -60,8 +62,20 @@ public static RestHighLevelClient getClient(String address) { TelnetUtil.telnet(pair[0], Integer.valueOf(pair[1])); httpHostList.add(new HttpHost(pair[0], Integer.valueOf(pair[1]), "http")); } - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder(httpHostList.toArray(new HttpHost[httpHostList.size()]))); + + RestClientBuilder builder = RestClient.builder(httpHostList.toArray(new HttpHost[httpHostList.size()])); + + Integer timeout = MapUtils.getInteger(config, EsConfigKeys.KEY_TIMEOUT); + if (timeout != null){ + builder.setMaxRetryTimeoutMillis(timeout); + } + + String pathPrefix = MapUtils.getString(config, EsConfigKeys.KEY_PATH_PREFIX); + if (StringUtils.isNotEmpty(pathPrefix)){ + builder.setPathPrefix(pathPrefix); + } + + RestHighLevelClient client = new RestHighLevelClient(builder); return client; } diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java index 018a2e5e1f..5507088af9 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java @@ -58,6 +58,8 @@ public class EsInputFormat extends RichInputFormat { protected int batchSize = 10; + protected Map clientConfig; + private int from; private int to; @@ -73,7 +75,7 @@ public class EsInputFormat extends RichInputFormat { @Override public void configure(Configuration configuration) { - client = EsUtil.getClient(address); + client = EsUtil.getClient(address, clientConfig); } @Override diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java index 802aef3451..0fcc7ec79e 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java @@ -20,6 +20,7 @@ import com.dtstack.flinkx.inputformat.RichInputFormatBuilder; import java.util.List; +import java.util.Map; /** * The builder class of EsInputFormat @@ -82,6 +83,10 @@ public EsInputFormatBuilder setBatchSize(Integer batchSize){ return this; } + public EsInputFormatBuilder setClientConfig(Map clientConfig){ + format.clientConfig = clientConfig; + return this; + } @Override protected void checkFormat() { diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java index 58aa792da2..b987f04037 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java @@ -27,6 +27,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -44,6 +45,7 @@ public class EsReader extends DataReader { private String index; private String type; private Integer batchSize; + private Map clientConfig; protected List columnType; protected List columnValue; @@ -57,6 +59,10 @@ public EsReader(DataTransferConfig config, StreamExecutionEnvironment env) { type = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_TYPE); batchSize = readerConfig.getParameter().getIntVal(EsConfigKeys.KEY_BATCH_SIZE, 0); + clientConfig = new HashMap<>(); + clientConfig.put(EsConfigKeys.KEY_TIMEOUT, readerConfig.getParameter().getVal(EsConfigKeys.KEY_TIMEOUT)); + clientConfig.put(EsConfigKeys.KEY_PATH_PREFIX, readerConfig.getParameter().getVal(EsConfigKeys.KEY_PATH_PREFIX)); + Object queryMap = readerConfig.getParameter().getVal(EsConfigKeys.KEY_QUERY); if(queryMap != null) { query = new Gson().toJson(queryMap); @@ -93,6 +99,7 @@ public DataStream readData() { builder.setIndex(index); builder.setType(type); builder.setBatchSize(batchSize); + builder.setClientConfig(clientConfig); builder.setQuery(query); builder.setBytes(bytes); builder.setMonitorUrls(monitorUrls); diff --git a/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java b/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java index 6a757c8798..29dbcec517 100644 --- a/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java +++ b/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java @@ -30,6 +30,7 @@ import org.elasticsearch.client.RestHighLevelClient; import java.io.IOException; import java.util.List; +import java.util.Map; /** * The OutputFormat class of ElasticSearch @@ -55,6 +56,8 @@ public class EsOutputFormat extends RichOutputFormat { protected List columnNames; + protected Map clientConfig; + private transient RestHighLevelClient client; private transient BulkRequest bulkRequest; @@ -62,7 +65,7 @@ public class EsOutputFormat extends RichOutputFormat { @Override public void configure(Configuration configuration) { - client = EsUtil.getClient(address); + client = EsUtil.getClient(address, clientConfig); bulkRequest = new BulkRequest(); } diff --git a/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormatBuilder.java b/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormatBuilder.java index 4323ee4162..30edca2dac 100644 --- a/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormatBuilder.java +++ b/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormatBuilder.java @@ -20,6 +20,7 @@ import com.dtstack.flinkx.outputformat.RichOutputFormatBuilder; import java.util.List; +import java.util.Map; /** * The Builder class of EsOutputFormat @@ -67,6 +68,10 @@ public void setColumnTypes(List columnTypes) { format.columnTypes = columnTypes; } + public EsOutputFormatBuilder setClientConfig(Map clientConfig){ + format.clientConfig = clientConfig; + return this; + } @Override protected void checkFormat() { diff --git a/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsWriter.java b/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsWriter.java index 3ac8ba26f4..a218349ff7 100644 --- a/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsWriter.java +++ b/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsWriter.java @@ -27,6 +27,7 @@ import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; import org.apache.flink.types.Row; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -48,6 +49,8 @@ public class EsWriter extends DataWriter { private int bulkAction; + private Map clientConfig; + private List columnTypes; private List columnNames; @@ -66,6 +69,10 @@ public EsWriter(DataTransferConfig config) { index = writerConfig.getParameter().getStringVal(EsConfigKeys.KEY_INDEX); bulkAction = writerConfig.getParameter().getIntVal(EsConfigKeys.KEY_BULK_ACTION, DEFAULT_BULK_ACTION); + clientConfig = new HashMap<>(); + clientConfig.put(EsConfigKeys.KEY_TIMEOUT, writerConfig.getParameter().getVal(EsConfigKeys.KEY_TIMEOUT)); + clientConfig.put(EsConfigKeys.KEY_PATH_PREFIX, writerConfig.getParameter().getVal(EsConfigKeys.KEY_PATH_PREFIX)); + List columns = writerConfig.getParameter().getColumn(); if(columns != null || columns.size() != 0) { columnTypes = new ArrayList<>(); @@ -111,6 +118,7 @@ public DataStreamSink writeData(DataStream dataSet) { builder.setIndex(index); builder.setType(type); builder.setBatchInterval(bulkAction); + builder.setClientConfig(clientConfig); builder.setColumnNames(columnNames); builder.setColumnTypes(columnTypes); builder.setIdColumnIndices(idColumnIndices); From 845b2d238f45e5b2e06f6e9fc744d6d5ed703053 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 9 Apr 2019 16:58:21 +0800 Subject: [PATCH 75/96] =?UTF-8?q?timeout=E5=8D=95=E4=BD=8D=E6=94=B9?= =?UTF-8?q?=E4=B8=BA=E7=A7=92?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flinkx/es/EsUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java index 5c0758e38d..704b638113 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsUtil.java @@ -67,7 +67,7 @@ public static RestHighLevelClient getClient(String address,Map co Integer timeout = MapUtils.getInteger(config, EsConfigKeys.KEY_TIMEOUT); if (timeout != null){ - builder.setMaxRetryTimeoutMillis(timeout); + builder.setMaxRetryTimeoutMillis(timeout * 1000); } String pathPrefix = MapUtils.getString(config, EsConfigKeys.KEY_PATH_PREFIX); From a5f8883784b36c7b834f295bbe1da3fb0cc73a46 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 10 Apr 2019 20:06:09 +0800 Subject: [PATCH 76/96] =?UTF-8?q?=E5=AF=B9=E6=89=B9=E9=87=8F=E5=86=99?= =?UTF-8?q?=E5=85=A5es=E7=BB=93=E6=9E=9C=E8=BF=9B=E8=A1=8C=E5=A4=84?= =?UTF-8?q?=E7=90=86?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/es/writer/EsOutputFormat.java | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java b/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java index 29dbcec517..dbdb70bba4 100644 --- a/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java +++ b/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java @@ -25,7 +25,9 @@ import org.apache.commons.lang.StringUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; +import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.client.RestHighLevelClient; import java.io.IOException; @@ -95,7 +97,23 @@ protected void writeMultipleRecordsInternal() throws Exception { request = request.source(EsUtil.rowToJsonMap(row, columnNames, columnTypes)); bulkRequest.add(request); } - client.bulk(bulkRequest); + + BulkResponse response = client.bulk(bulkRequest); + if (response.hasFailures()){ + if (dirtyDataManager != null){ + BulkItemResponse[] itemResponses = response.getItems(); + WriteRecordException exception; + for (int i = 0; i < itemResponses.length; i++) { + if(itemResponses[i].isFailed()){ + exception = new WriteRecordException(itemResponses[i].getFailureMessage() + ,itemResponses[i].getFailure().getCause()); + dirtyDataManager.writeData(rows.get(i), exception); + } + } + } else { + LOG.warn(response.buildFailureMessage()); + } + } } @Override From eeb16a5d15d89db6658cf30e1a50d44d3c674797 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 10 Apr 2019 20:11:05 +0800 Subject: [PATCH 77/96] =?UTF-8?q?=E5=A4=9A=E9=80=9A=E9=81=93=E6=97=B6?= =?UTF-8?q?=E6=A3=80=E6=9F=A5=E5=88=87=E5=88=86=E9=94=AE?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../rdb/inputformat/DistributedJdbcInputFormatBuilder.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/DistributedJdbcInputFormatBuilder.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/DistributedJdbcInputFormatBuilder.java index 00ed8f03f2..bfa6225e64 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/DistributedJdbcInputFormatBuilder.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/DistributedJdbcInputFormatBuilder.java @@ -23,6 +23,7 @@ import com.dtstack.flinkx.rdb.DatabaseInterface; import com.dtstack.flinkx.rdb.type.TypeConverterInterface; import com.dtstack.flinkx.reader.MetaColumn; +import org.apache.commons.lang.StringUtils; import java.util.List; @@ -122,6 +123,10 @@ protected void checkFormat() { if(!dataSource.getJdbcUrl().startsWith(jdbcPrefix)){ throw new IllegalArgumentException("Multiple data sources must be of the same type"); } + + if (StringUtils.isEmpty(format.splitKey) && format.numPartitions > 1){ + throw new IllegalArgumentException("Must specify the split column when the channel is greater than 1"); + } } } } From fa5746a3123e3c082eccb96d38d6735309aadc7c Mon Sep 17 00:00:00 2001 From: jiangbo Date: Thu, 11 Apr 2019 17:13:25 +0800 Subject: [PATCH 78/96] =?UTF-8?q?writerConfig=E9=87=8CjdbcUrl=E6=A0=BC?= =?UTF-8?q?=E5=BC=8F=E5=85=BC=E5=AE=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flinkx/config/WriterConfig.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/WriterConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/WriterConfig.java index d3d0af9081..2df69d966b 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/WriterConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/WriterConfig.java @@ -18,6 +18,8 @@ package com.dtstack.flinkx.config; +import org.apache.commons.collections.CollectionUtils; + import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -100,7 +102,12 @@ public class ConnectionConfig extends AbstractConfig { public ConnectionConfig(Map map) { super(map); - jdbcUrl = getStringVal(KEY_JDBC_URL); + Object jdbcUrlObj = internalMap.get(KEY_JDBC_URL); + if(jdbcUrlObj instanceof String){ + jdbcUrl = jdbcUrlObj.toString(); + } else if(jdbcUrlObj instanceof List && CollectionUtils.isNotEmpty((List) jdbcUrlObj)){ + jdbcUrl = ((List) jdbcUrlObj).get(0).toString(); + } table = (List) getVal(KEY_TABLE_LIST); } From a44ed25bd778e47ccfe592c04b4b84d588856ac5 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Thu, 11 Apr 2019 11:07:56 +0800 Subject: [PATCH 79/96] =?UTF-8?q?=E8=BF=87=E6=BB=A4=E6=9D=A1=E4=BB=B6?= =?UTF-8?q?=E7=A9=BA=E4=B8=B2=E5=88=A4=E6=96=AD?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java index 3b5cdbad99..f4bef90eda 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java @@ -580,8 +580,11 @@ public static String getQuerySql(DatabaseInterface databaseInterface,String tabl filter.append(" AND ").append(databaseInterface.getSplitFilter(splitKey)); } - if(StringUtils.isNotEmpty(customFilter)) { - filter.append(" AND ").append(customFilter); + if (customFilter != null){ + customFilter = customFilter.trim(); + if (customFilter.length() > 0){ + filter.append(" AND ").append(customFilter); + } } if (realTimeIncreSync){ From dc4e56ea3d099ff46f9890e2d20392adc17408ae Mon Sep 17 00:00:00 2001 From: jiangbo Date: Thu, 18 Apr 2019 14:34:30 +0800 Subject: [PATCH 80/96] =?UTF-8?q?fix=20hdfs=E8=AF=BB=E5=8F=96text=E6=A0=BC?= =?UTF-8?q?=E5=BC=8F=E6=96=87=E4=BB=B6=E5=A4=9A"\u0000"=E7=9A=84bug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java index e66b07877a..a38c9b89c6 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java @@ -30,13 +30,13 @@ import org.apache.hadoop.mapred.TextInputFormat; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.FileSplit; + import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; import java.nio.charset.Charset; import java.nio.charset.UnsupportedCharsetException; -import java.util.List; import java.util.Map; /** @@ -81,12 +81,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { value = new Text(); } - - @Override public Row nextRecordInternal(Row row) throws IOException { - byte[] data = ((Text)value).getBytes(); - String line = new String(data, charsetName); + String line = new String(((Text)value).getBytes(), 0, ((Text)value).getLength(), charsetName); String[] fields = line.split(delimiter); if (metaColumns.size() == 1 && "*".equals(metaColumns.get(0).getName())){ From 56578a8e5376ea0d894aec8b2f60c7af014e88a3 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Thu, 18 Apr 2019 16:03:41 +0800 Subject: [PATCH 81/96] =?UTF-8?q?=E7=BC=96=E8=AF=91=E6=89=93=E5=8C=85?= =?UTF-8?q?=E5=89=8D=E5=88=A0=E9=99=A4=E5=8E=9F=E6=9C=89jar=E5=8C=85?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-core/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index a9e77e590f..8660b01683 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -153,6 +153,7 @@ + From aadeae177dae13bffb5a6eae19c9e823a605054c Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 19 Apr 2019 11:13:22 +0800 Subject: [PATCH 82/96] =?UTF-8?q?fix=E8=AF=BB=E5=8F=96parquet=E8=A1=A8bug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java index d5114ad336..f1f1c3d210 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java @@ -172,6 +172,10 @@ public boolean reachedEnd() throws IOException { private Object getData(Group currentLine,String type,int index){ Object data = null; try{ + if (index == -1){ + return null; + } + Type colSchemaType = currentLine.getType().getType(index); switch (type){ case "tinyint" : From feed7c67f32707878e09fcba8309f9dd5d82b1f2 Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Thu, 25 Apr 2019 15:29:55 +0800 Subject: [PATCH 83/96] =?UTF-8?q?fix=E7=A6=81=E7=94=A8=E5=8F=96max?= =?UTF-8?q?=E5=80=BC=E5=8A=9F=E8=83=BD?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/rdb/datareader/JdbcConfigKeys.java | 2 +- .../flinkx/rdb/datareader/JdbcDataReader.java | 12 +++++------- .../flinkx/rdb/inputformat/JdbcInputFormat.java | 14 +++++++------- .../rdb/inputformat/JdbcInputFormatBuilder.java | 4 ++-- .../java/com/dtstack/flinkx/rdb/util/DBUtil.java | 4 ++-- 5 files changed, 17 insertions(+), 19 deletions(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java index 3779b26c41..4ecb1f9842 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcConfigKeys.java @@ -46,5 +46,5 @@ public class JdbcConfigKeys { public static final String KEY_CUSTOM_SQL = "customSql"; - public static final String KEY_REALTIME_INCRE_SYNC = "realTimeIncreSync"; + public static final String KEY_USE_MAX_FUNC = "useMaxFunc"; } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java index 26bcbb5ed8..2b9a580044 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java @@ -70,7 +70,7 @@ public class JdbcDataReader extends DataReader { protected int requestAccumulatorInterval; - protected boolean realTimeIncreSync; + protected boolean useMaxFunc; protected String customSql; @@ -100,9 +100,7 @@ public JdbcDataReader(DataTransferConfig config, StreamExecutionEnvironment env) increColumn = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_INCRE_COLUMN); startLocation = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_START_LOCATION,null); customSql = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_CUSTOM_SQL,null); - - realTimeIncreSync = readerConfig.getParameter().getBooleanVal(JdbcConfigKeys.KEY_REALTIME_INCRE_SYNC,true); - realTimeIncreSync = StringUtils.isNotEmpty(increColumn); + useMaxFunc = readerConfig.getParameter().getBooleanVal(JdbcConfigKeys.KEY_USE_MAX_FUNC,true); } @Override @@ -127,7 +125,7 @@ public DataStream readData() { builder.setStartLocation(startLocation); builder.setSplitKey(splitKey); builder.setNumPartitions(numPartitions); - builder.setRealTimeIncreSync(realTimeIncreSync); + builder.setUseMaxFunc(useMaxFunc); builder.setCustomSql(customSql); builder.setHadoopConfig(hadoopConfig); @@ -136,9 +134,9 @@ public DataStream readData() { String query; if (StringUtils.isNotEmpty(customSql)){ - query = DBUtil.buildQuerySqlWithCustomSql(databaseInterface, customSql, isSplitByKey, splitKey, realTimeIncreSync); + query = DBUtil.buildQuerySqlWithCustomSql(databaseInterface, customSql, isSplitByKey, splitKey, useMaxFunc); } else { - query = DBUtil.getQuerySql(databaseInterface, table, metaColumns, splitKey, where, isSplitByKey, realTimeIncreSync); + query = DBUtil.getQuerySql(databaseInterface, table, metaColumns, splitKey, where, isSplitByKey, useMaxFunc); } builder.setQuery(query); diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index cfd612dfd7..871d7c47c3 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -116,7 +116,7 @@ public class JdbcInputFormat extends RichInputFormat { protected int requestAccumulatorInterval; - protected boolean realTimeIncreSync; + protected boolean useMaxFunc; protected int numPartitions; @@ -152,7 +152,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { ClassUtil.forName(drivername, getClass().getClassLoader()); - if (realTimeIncreSync){ + if (useMaxFunc){ getMaxValue(inputSplit); } @@ -249,7 +249,7 @@ public Row nextRecordInternal(Row row) throws IOException { } } - if(increCol != null && !realTimeIncreSync){ + if(increCol != null && !useMaxFunc){ if (ColumnType.isTimeType(increColType)){ Timestamp increVal = resultSet.getTimestamp(increColIndex + 1); if(increVal != null){ @@ -293,7 +293,7 @@ private void initMetric(InputSplit split){ if(!accumulatorMap.containsKey(Metrics.END_LOCATION) && endLocation != null){ endLocationAccumulator = new MaximumAccumulator(); - if(realTimeIncreSync){ + if(useMaxFunc){ endLocationAccumulator.add(endLocation); } @@ -301,7 +301,7 @@ private void initMetric(InputSplit split){ } if (!accumulatorMap.containsKey(Metrics.START_LOCATION) && startLocation != null){ - if(!realTimeIncreSync){ + if(!useMaxFunc){ endLocationAccumulator.add(startLocation); } @@ -407,7 +407,7 @@ private boolean canReadData(InputSplit split){ return true; } - if (!realTimeIncreSync){ + if (!useMaxFunc){ return true; } @@ -426,7 +426,7 @@ private String buildQuerySql(InputSplit inputSplit){ .replace("${M}", String.valueOf(jdbcInputSplit.getMod())); } - if (realTimeIncreSync){ + if (useMaxFunc){ String incrementFilter = DBUtil.buildIncrementFilter(databaseInterface, increColType, increCol, jdbcInputSplit.getStartLocation(), jdbcInputSplit.getEndLocation(), customSql); diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java index 0cb38f54fd..8230f47b6d 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormatBuilder.java @@ -105,8 +105,8 @@ public void setIncreColType(String increColType){ format.increColType = increColType; } - public void setRealTimeIncreSync(boolean realTimeIncreSync){ - format.realTimeIncreSync = realTimeIncreSync; + public void setUseMaxFunc(boolean useMaxFunc){ + format.useMaxFunc = useMaxFunc; } public void setNumPartitions(int numPartitions){ diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java index f4bef90eda..aab1e804c4 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java @@ -532,7 +532,7 @@ public static long getMillis(long startLocation){ } public static String buildQuerySqlWithCustomSql(DatabaseInterface databaseInterface,String customSql, - boolean isSplitByKey,String splitKey,boolean realTimeIncreSync){ + boolean isSplitByKey,String splitKey,boolean useMaxFunc){ StringBuilder querySql = new StringBuilder(); querySql.append(String.format(CUSTOM_SQL_TEMPLATE, customSql, TEMPORARY_TABLE_NAME)); querySql.append(" WHERE 1=1 "); @@ -541,7 +541,7 @@ public static String buildQuerySqlWithCustomSql(DatabaseInterface databaseInterf querySql.append(" And ").append(databaseInterface.getSplitFilterWithTmpTable(TEMPORARY_TABLE_NAME, splitKey)); } - if (realTimeIncreSync){ + if (useMaxFunc){ querySql.append(" ").append(INCREMENT_FILTER_PLACEHOLDER); } From f5c5eadeb464ad8c3008783635544c69b1745834 Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Thu, 25 Apr 2019 16:09:35 +0800 Subject: [PATCH 84/96] fix npe --- .../com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 871d7c47c3..e1e7336875 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -289,10 +289,9 @@ private void initMetric(InputSplit split){ getRuntimeContext().addAccumulator(Metrics.TABLE_COL,tableColAccumulator); } + endLocationAccumulator = new MaximumAccumulator(); String endLocation = ((JdbcInputSplit)split).getEndLocation(); if(!accumulatorMap.containsKey(Metrics.END_LOCATION) && endLocation != null){ - endLocationAccumulator = new MaximumAccumulator(); - if(useMaxFunc){ endLocationAccumulator.add(endLocation); } @@ -300,12 +299,12 @@ private void initMetric(InputSplit split){ getRuntimeContext().addAccumulator(Metrics.END_LOCATION,endLocationAccumulator); } + startLocationAccumulator = new StringAccumulator(); if (!accumulatorMap.containsKey(Metrics.START_LOCATION) && startLocation != null){ if(!useMaxFunc){ endLocationAccumulator.add(startLocation); } - startLocationAccumulator = new StringAccumulator(); startLocationAccumulator.add(startLocation); getRuntimeContext().addAccumulator(Metrics.START_LOCATION,startLocationAccumulator); } From 90f45f71e109dd74f48bddf4b9e632a5bb750b39 Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Thu, 25 Apr 2019 16:41:03 +0800 Subject: [PATCH 85/96] fix npe --- .../rdb/inputformat/JdbcInputFormat.java | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index e1e7336875..c5ef8dc0a9 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -291,23 +291,17 @@ private void initMetric(InputSplit split){ endLocationAccumulator = new MaximumAccumulator(); String endLocation = ((JdbcInputSplit)split).getEndLocation(); - if(!accumulatorMap.containsKey(Metrics.END_LOCATION) && endLocation != null){ - if(useMaxFunc){ - endLocationAccumulator.add(endLocation); - } - - getRuntimeContext().addAccumulator(Metrics.END_LOCATION,endLocationAccumulator); + if(endLocation != null && useMaxFunc){ + endLocationAccumulator.add(endLocation); } + getRuntimeContext().addAccumulator(Metrics.END_LOCATION,endLocationAccumulator); startLocationAccumulator = new StringAccumulator(); - if (!accumulatorMap.containsKey(Metrics.START_LOCATION) && startLocation != null){ - if(!useMaxFunc){ - endLocationAccumulator.add(startLocation); - } - + if (startLocation != null && !useMaxFunc){ + endLocationAccumulator.add(startLocation); startLocationAccumulator.add(startLocation); - getRuntimeContext().addAccumulator(Metrics.START_LOCATION,startLocationAccumulator); } + getRuntimeContext().addAccumulator(Metrics.START_LOCATION,startLocationAccumulator); for (int i = 0; i < metaColumns.size(); i++) { if (metaColumns.get(i).getName().equals(increCol)){ From fc1e0968e0bd3a8cf978c6863dd526fde355722b Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Fri, 26 Apr 2019 09:57:13 +0800 Subject: [PATCH 86/96] fix --- .../flinkx/rdb/datareader/JdbcDataReader.java | 4 ++++ .../flinkx/rdb/inputformat/JdbcInputFormat.java | 15 ++++++++------- 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java index 2b9a580044..e72ecf6891 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java @@ -101,6 +101,10 @@ public JdbcDataReader(DataTransferConfig config, StreamExecutionEnvironment env) startLocation = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_START_LOCATION,null); customSql = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_CUSTOM_SQL,null); useMaxFunc = readerConfig.getParameter().getBooleanVal(JdbcConfigKeys.KEY_USE_MAX_FUNC,true); + + if(StringUtils.isEmpty(increColumn)){ + useMaxFunc = false; + } } @Override diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index c5ef8dc0a9..7dfe6e2dea 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -289,19 +289,20 @@ private void initMetric(InputSplit split){ getRuntimeContext().addAccumulator(Metrics.TABLE_COL,tableColAccumulator); } + startLocationAccumulator = new StringAccumulator(); + if (startLocation != null){ + startLocationAccumulator.add(startLocation); + } + getRuntimeContext().addAccumulator(Metrics.START_LOCATION,startLocationAccumulator); + endLocationAccumulator = new MaximumAccumulator(); String endLocation = ((JdbcInputSplit)split).getEndLocation(); if(endLocation != null && useMaxFunc){ endLocationAccumulator.add(endLocation); - } - getRuntimeContext().addAccumulator(Metrics.END_LOCATION,endLocationAccumulator); - - startLocationAccumulator = new StringAccumulator(); - if (startLocation != null && !useMaxFunc){ + } else { endLocationAccumulator.add(startLocation); - startLocationAccumulator.add(startLocation); } - getRuntimeContext().addAccumulator(Metrics.START_LOCATION,startLocationAccumulator); + getRuntimeContext().addAccumulator(Metrics.END_LOCATION,endLocationAccumulator); for (int i = 0; i < metaColumns.size(); i++) { if (metaColumns.get(i).getName().equals(increCol)){ From 4b8636da25db6d018e01502b157996c34e60daad Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Fri, 26 Apr 2019 15:53:49 +0800 Subject: [PATCH 87/96] =?UTF-8?q?fix=20useMaxFunc=E4=B8=BAfalse=E6=97=B6sq?= =?UTF-8?q?l=E6=8B=BC=E6=8E=A5=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java | 4 ++-- .../dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java | 2 +- .../src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java | 8 ++++---- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java index e72ecf6891..46989f94ea 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java @@ -138,9 +138,9 @@ public DataStream readData() { String query; if (StringUtils.isNotEmpty(customSql)){ - query = DBUtil.buildQuerySqlWithCustomSql(databaseInterface, customSql, isSplitByKey, splitKey, useMaxFunc); + query = DBUtil.buildQuerySqlWithCustomSql(databaseInterface, customSql, isSplitByKey, splitKey, StringUtils.isNotEmpty(increColumn)); } else { - query = DBUtil.getQuerySql(databaseInterface, table, metaColumns, splitKey, where, isSplitByKey, useMaxFunc); + query = DBUtil.getQuerySql(databaseInterface, table, metaColumns, splitKey, where, isSplitByKey, StringUtils.isNotEmpty(increColumn)); } builder.setQuery(query); diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 7dfe6e2dea..304b8ec2fd 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -420,7 +420,7 @@ private String buildQuerySql(InputSplit inputSplit){ .replace("${M}", String.valueOf(jdbcInputSplit.getMod())); } - if (useMaxFunc){ + if (StringUtils.isNotEmpty(increCol)){ String incrementFilter = DBUtil.buildIncrementFilter(databaseInterface, increColType, increCol, jdbcInputSplit.getStartLocation(), jdbcInputSplit.getEndLocation(), customSql); diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java index aab1e804c4..489c5a1b16 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java @@ -532,7 +532,7 @@ public static long getMillis(long startLocation){ } public static String buildQuerySqlWithCustomSql(DatabaseInterface databaseInterface,String customSql, - boolean isSplitByKey,String splitKey,boolean useMaxFunc){ + boolean isSplitByKey,String splitKey,boolean increment){ StringBuilder querySql = new StringBuilder(); querySql.append(String.format(CUSTOM_SQL_TEMPLATE, customSql, TEMPORARY_TABLE_NAME)); querySql.append(" WHERE 1=1 "); @@ -541,7 +541,7 @@ public static String buildQuerySqlWithCustomSql(DatabaseInterface databaseInterf querySql.append(" And ").append(databaseInterface.getSplitFilterWithTmpTable(TEMPORARY_TABLE_NAME, splitKey)); } - if (useMaxFunc){ + if (increment){ querySql.append(" ").append(INCREMENT_FILTER_PLACEHOLDER); } @@ -554,7 +554,7 @@ public static String getQuerySql(DatabaseInterface databaseInterface,String tabl } public static String getQuerySql(DatabaseInterface databaseInterface,String table,List metaColumns, - String splitKey,String customFilter,boolean isSplitByKey,boolean realTimeIncreSync) { + String splitKey,String customFilter,boolean isSplitByKey,boolean increment) { StringBuilder sb = new StringBuilder(); List selectColumns = new ArrayList<>(); @@ -587,7 +587,7 @@ public static String getQuerySql(DatabaseInterface databaseInterface,String tabl } } - if (realTimeIncreSync){ + if (increment){ filter.append(" ").append(INCREMENT_FILTER_PLACEHOLDER); } From 74f186b4f2485fc7559bdb5998fd4fea12f953d0 Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Fri, 26 Apr 2019 18:27:53 +0800 Subject: [PATCH 88/96] =?UTF-8?q?useMaxFunc=E4=B8=BAfalse=E6=97=B6?= =?UTF-8?q?=E4=B8=8D=E8=AF=BB=E5=8F=96=E8=BE=B9=E7=95=8C=E5=80=BC?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/rdb/inputformat/JdbcInputFormat.java | 4 ++-- .../com/dtstack/flinkx/rdb/util/DBUtil.java | 17 +++++++++++------ 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java index 304b8ec2fd..76685e9cbb 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/inputformat/JdbcInputFormat.java @@ -422,7 +422,7 @@ private String buildQuerySql(InputSplit inputSplit){ if (StringUtils.isNotEmpty(increCol)){ String incrementFilter = DBUtil.buildIncrementFilter(databaseInterface, increColType, increCol, - jdbcInputSplit.getStartLocation(), jdbcInputSplit.getEndLocation(), customSql); + jdbcInputSplit.getStartLocation(), jdbcInputSplit.getEndLocation(), customSql, useMaxFunc); if(StringUtils.isNotEmpty(incrementFilter)){ incrementFilter = " and " + incrementFilter; @@ -455,7 +455,7 @@ private String getMaxValueFromDb() { } String startSql = DBUtil.buildStartLocationSql(databaseInterface, increColType, - databaseInterface.quoteColumn(increCol), startLocation); + databaseInterface.quoteColumn(increCol), startLocation, useMaxFunc); if(StringUtils.isNotEmpty(startSql)){ queryMaxValueSql += " where " + startSql; } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java index 489c5a1b16..ef72f08efe 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java @@ -366,14 +366,14 @@ public static Object clobToString(Object obj) throws Exception{ } public static String buildIncrementFilter(DatabaseInterface databaseInterface,String increColType,String increCol, - String startLocation,String endLocation, String customSql){ + String startLocation,String endLocation, String customSql, boolean useMaxFunc){ StringBuilder filter = new StringBuilder(); if (StringUtils.isNotEmpty(customSql)){ increCol = String.format("%s.%s", TEMPORARY_TABLE_NAME, databaseInterface.quoteColumn(increCol)); } - String startFilter = buildStartLocationSql(databaseInterface, increColType, increCol, startLocation); + String startFilter = buildStartLocationSql(databaseInterface, increColType, increCol, startLocation, useMaxFunc); if (StringUtils.isNotEmpty(startFilter)){ filter.append(startFilter); } @@ -419,7 +419,7 @@ public static String buildEndLocationSql(DatabaseInterface databaseInterface,Str return endLocationSql; } - public static String buildStartLocationSql(DatabaseInterface databaseInterface,String increColType,String increCol,String startLocation){ + public static String buildStartLocationSql(DatabaseInterface databaseInterface,String increColType,String increCol,String startLocation,boolean useMaxFunc){ if(StringUtils.isEmpty(startLocation)){ return null; @@ -428,6 +428,11 @@ public static String buildStartLocationSql(DatabaseInterface databaseInterface,S String startLocationSql; String startTimeStr; + String operator = " >= "; + if(!useMaxFunc){ + operator = " > "; + } + if(ColumnType.isTimeType(increColType) || (databaseInterface.getDatabaseType() == EDatabaseType.SQLServer && ColumnType.NVARCHAR.name().equals(increColType))){ startTimeStr = getStartTimeStr(databaseInterface.getDatabaseType(),Long.parseLong(startLocation)); @@ -437,12 +442,12 @@ public static String buildStartLocationSql(DatabaseInterface databaseInterface,S startTimeStr = String.format("'%s'",startTimeStr); } - startLocationSql = increCol + " >= " + startTimeStr; + startLocationSql = increCol + operator + startTimeStr; } else if(ColumnType.isNumberType(increColType)){ - startLocationSql = increCol + " >= " + startLocation; + startLocationSql = increCol + operator + startLocation; } else { startTimeStr = String.format("'%s'",startLocation); - startLocationSql = increCol + " >= " + startTimeStr; + startLocationSql = increCol + operator + startTimeStr; } return startLocationSql; From 4a00a1dc2284ba22c4af28b872c92886950a27d1 Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Sun, 28 Apr 2019 19:56:42 +0800 Subject: [PATCH 89/96] fix column to string --- .../src/main/java/com/dtstack/flinkx/util/StringUtil.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java index dcb59a78a9..fe24e91436 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java @@ -167,10 +167,10 @@ public static String col2string(Object column, String type) { result = Boolean.valueOf(rowData.trim()); break; case DATE: - result = DateUtil.dateToString((java.util.Date)column); + result = DateUtil.dateToString(DateUtil.columnToDate(column, null)); break; case TIMESTAMP: - result = DateUtil.timestampToString((java.util.Date)column); + result = DateUtil.timestampToString(DateUtil.columnToTimestamp(column, null)); break; default: result = rowData; From 064475771ef02ff14aa0e6896ac96613f4097201 Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Sun, 28 Apr 2019 20:13:34 +0800 Subject: [PATCH 90/96] fix column to string --- .../src/main/java/com/dtstack/flinkx/util/StringUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java index fe24e91436..2e46965fc3 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java @@ -167,7 +167,7 @@ public static String col2string(Object column, String type) { result = Boolean.valueOf(rowData.trim()); break; case DATE: - result = DateUtil.dateToString(DateUtil.columnToDate(column, null)); + result = DateUtil.timestampToString(DateUtil.columnToDate(column, null)); break; case TIMESTAMP: result = DateUtil.timestampToString(DateUtil.columnToTimestamp(column, null)); From ac2af6a6d6ee949908d87a612ad1274fa4603b86 Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Sun, 28 Apr 2019 20:33:04 +0800 Subject: [PATCH 91/96] =?UTF-8?q?=E5=8C=BA=E5=88=86date=E5=92=8Cdatetime?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flinkx/util/StringUtil.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java index 2e46965fc3..1fa9aac713 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java @@ -167,8 +167,9 @@ public static String col2string(Object column, String type) { result = Boolean.valueOf(rowData.trim()); break; case DATE: - result = DateUtil.timestampToString(DateUtil.columnToDate(column, null)); + result = DateUtil.dateToString(DateUtil.columnToDate(column, null)); break; + case DATETIME: case TIMESTAMP: result = DateUtil.timestampToString(DateUtil.columnToTimestamp(column, null)); break; From 0c5871bb92ba0923a893982d28a73e5ffc67813a Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Mon, 29 Apr 2019 15:58:56 +0800 Subject: [PATCH 92/96] =?UTF-8?q?oracle=E5=A2=9E=E9=87=8F=E5=AD=97?= =?UTF-8?q?=E6=AE=B5=E5=8C=BA=E5=88=86date=E5=92=8Ctimestamp=E7=B1=BB?= =?UTF-8?q?=E5=9E=8B?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flinkx/rdb/util/DBUtil.java | 101 ++++++++---------- 1 file changed, 45 insertions(+), 56 deletions(-) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java index ef72f08efe..024a91e936 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java @@ -390,67 +390,46 @@ public static String buildIncrementFilter(DatabaseInterface databaseInterface,St return filter.toString(); } - public static String buildEndLocationSql(DatabaseInterface databaseInterface,String increColType,String increCol,String endLocation){ - - if(StringUtils.isEmpty(endLocation)){ - return null; - } - - String endLocationSql; - String endTimeStr; - - if(ColumnType.isTimeType(increColType) || (databaseInterface.getDatabaseType() == EDatabaseType.SQLServer && ColumnType.NVARCHAR.name().equals(increColType))){ - endTimeStr = getStartTimeStr(databaseInterface.getDatabaseType(),Long.parseLong(endLocation)); - - if (databaseInterface.getDatabaseType() == EDatabaseType.Oracle){ - endTimeStr = String.format("TO_TIMESTAMP('%s','YYYY-MM-DD HH24:MI:SS:FF6')",endTimeStr); - } else { - endTimeStr = String.format("'%s'",endTimeStr); - } - - endLocationSql = increCol + " < " + endTimeStr; - } else if(ColumnType.isNumberType(increColType)){ - endLocationSql = increCol + " < " + endLocation; - } else { - endTimeStr = String.format("'%s'",endLocation); - endLocationSql = increCol + " < " + endTimeStr; - } - - return endLocationSql; - } - - public static String buildStartLocationSql(DatabaseInterface databaseInterface,String increColType,String increCol,String startLocation,boolean useMaxFunc){ - + public static String buildStartLocationSql(DatabaseInterface databaseInterface,String incrementColType, + String incrementCol,String startLocation,boolean useMaxFunc){ if(StringUtils.isEmpty(startLocation)){ return null; } - String startLocationSql; - String startTimeStr; - String operator = " >= "; if(!useMaxFunc){ operator = " > "; } - if(ColumnType.isTimeType(increColType) || (databaseInterface.getDatabaseType() == EDatabaseType.SQLServer && ColumnType.NVARCHAR.name().equals(increColType))){ - startTimeStr = getStartTimeStr(databaseInterface.getDatabaseType(),Long.parseLong(startLocation)); + return getLocationSql(databaseInterface, incrementColType, incrementCol, startLocation, operator); + } - if (databaseInterface.getDatabaseType() == EDatabaseType.Oracle){ - startTimeStr = String.format("TO_TIMESTAMP('%s','YYYY-MM-DD HH24:MI:SS:FF6')",startTimeStr); - } else { - startTimeStr = String.format("'%s'",startTimeStr); - } + public static String buildEndLocationSql(DatabaseInterface databaseInterface,String incrementColType,String incrementCol, + String endLocation){ + if(StringUtils.isEmpty(endLocation)){ + return null; + } - startLocationSql = increCol + operator + startTimeStr; - } else if(ColumnType.isNumberType(increColType)){ - startLocationSql = increCol + operator + startLocation; + return getLocationSql(databaseInterface, incrementColType, incrementCol, endLocation, " < "); + } + + private static String getLocationSql(DatabaseInterface databaseInterface, String incrementColType, String incrementCol, + String endLocation, String operator) { + String endTimeStr; + String endLocationSql; + boolean isTimeType = ColumnType.isTimeType(incrementColType) + || (databaseInterface.getDatabaseType() == EDatabaseType.SQLServer && ColumnType.NVARCHAR.name().equals(incrementColType)); + if(isTimeType){ + endTimeStr = getTimeStr(databaseInterface.getDatabaseType(), Long.parseLong(endLocation), incrementColType); + endLocationSql = incrementCol + operator + endTimeStr; + } else if(ColumnType.isNumberType(incrementColType)){ + endLocationSql = incrementCol + operator + endLocation; } else { - startTimeStr = String.format("'%s'",startLocation); - startLocationSql = increCol + operator + startTimeStr; + endTimeStr = String.format("'%s'",endLocation); + endLocationSql = incrementCol + operator + endTimeStr; } - return startLocationSql; + return endLocationSql; } public static String buildWhereSql(String where,String startSql,String endSql){ @@ -477,22 +456,32 @@ public static String buildWhereSql(String where,String startSql,String endSql){ return whereBuilder.toString(); } - private static String getStartTimeStr(EDatabaseType databaseType,Long startLocation){ - String startTimeStr; + private static String getTimeStr(EDatabaseType databaseType,Long startLocation,String incrementColType){ + String timeStr; Timestamp ts = new Timestamp(getMillis(startLocation)); ts.setNanos(getNanos(startLocation)); - startTimeStr = getNanosTimeStr(ts.toString()); + timeStr = getNanosTimeStr(ts.toString()); if(databaseType == EDatabaseType.SQLServer){ - startTimeStr = startTimeStr.substring(0,23); + timeStr = timeStr.substring(0,23); } else { - startTimeStr = startTimeStr.substring(0,26); + timeStr = timeStr.substring(0,26); } - return startTimeStr; + if (databaseType == EDatabaseType.Oracle){ + if(ColumnType.TIMESTAMP.name().equals(incrementColType)){ + timeStr = String.format("TO_TIMESTAMP('%s','YYYY-MM-DD HH24:MI:SS:FF6')",timeStr); + } else { + timeStr = String.format("TO_DATE('%s','YYYY-MM-DD HH24:MI:SS')", timeStr); + } + } else { + timeStr = String.format("'%s'",timeStr); + } + + return timeStr; } - public static String getNanosTimeStr(String timeStr){ + private static String getNanosTimeStr(String timeStr){ if(timeStr.length() < 29){ timeStr += StringUtils.repeat("0",29 - timeStr.length()); } @@ -500,7 +489,7 @@ public static String getNanosTimeStr(String timeStr){ return timeStr; } - public static int getNanos(long startLocation){ + private static int getNanos(long startLocation){ String timeStr = String.valueOf(startLocation); int nanos; if (timeStr.length() == SECOND_LENGTH){ @@ -518,7 +507,7 @@ public static int getNanos(long startLocation){ return nanos; } - public static long getMillis(long startLocation){ + private static long getMillis(long startLocation){ String timeStr = String.valueOf(startLocation); long millisSecond; if (timeStr.length() == SECOND_LENGTH){ From 88b6dd1360507b4f86cee8ccc31d3a25e6c121af Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Sun, 5 May 2019 15:43:53 +0800 Subject: [PATCH 93/96] =?UTF-8?q?=E4=BF=AE=E6=94=B9oracle=E5=A2=9E?= =?UTF-8?q?=E9=87=8F=E5=AD=97=E6=AE=B5date=E7=B1=BB=E5=9E=8B=E7=9A=84?= =?UTF-8?q?=E6=97=A5=E6=9C=9F=E6=A0=BC=E5=BC=8F?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java | 1 + flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java | 1 + 2 files changed, 2 insertions(+) diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java index 46989f94ea..16391e8fbc 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/datareader/JdbcDataReader.java @@ -102,6 +102,7 @@ public JdbcDataReader(DataTransferConfig config, StreamExecutionEnvironment env) customSql = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_CUSTOM_SQL,null); useMaxFunc = readerConfig.getParameter().getBooleanVal(JdbcConfigKeys.KEY_USE_MAX_FUNC,true); + increColumn = StringUtils.isEmpty(increColumn) ? null : increColumn; if(StringUtils.isEmpty(increColumn)){ useMaxFunc = false; } diff --git a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java index 024a91e936..daaaa923ac 100644 --- a/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java +++ b/flinkx-rdb/src/main/java/com/dtstack/flinkx/rdb/util/DBUtil.java @@ -472,6 +472,7 @@ private static String getTimeStr(EDatabaseType databaseType,Long startLocation,S if(ColumnType.TIMESTAMP.name().equals(incrementColType)){ timeStr = String.format("TO_TIMESTAMP('%s','YYYY-MM-DD HH24:MI:SS:FF6')",timeStr); } else { + timeStr = timeStr.substring(0, 19); timeStr = String.format("TO_DATE('%s','YYYY-MM-DD HH24:MI:SS')", timeStr); } } else { From 2ab453da587b6fc7645858d625bd2cb8d46c1132 Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Mon, 6 May 2019 21:37:36 +0800 Subject: [PATCH 94/96] =?UTF-8?q?=E8=8E=B7=E5=8F=96coreJar=E5=90=8D?= =?UTF-8?q?=E7=A7=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flinkx/launcher/Launcher.java | 35 ++++++++++++++++--- 1 file changed, 31 insertions(+), 4 deletions(-) diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index 8cc037977b..72333381b3 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -21,10 +21,14 @@ import com.dtstack.flinkx.config.ContentConfig; import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.util.SysUtil; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.util.Preconditions; + import java.io.File; +import java.io.FileNotFoundException; +import java.io.FilenameFilter; import java.net.MalformedURLException; import java.net.URL; import java.util.ArrayList; @@ -38,6 +42,8 @@ */ public class Launcher { + public static final String CORE_JAR_NAME_PREFIX = "flinkx"; + private static List initFlinkxArgList(LauncherOptions launcherOptions) { List argList = new ArrayList<>(); argList.add("-job"); @@ -75,9 +81,6 @@ private static List analyzeUserClasspath(String content, String pluginRoot) return urlList; } - - - public static void main(String[] args) throws Exception { LauncherOptions launcherOptions = new LauncherOptionParser(args).getLauncherOptions(); String mode = launcherOptions.getMode(); @@ -93,7 +96,8 @@ public static void main(String[] args) throws Exception { String pluginRoot = launcherOptions.getPlugin(); String content = launcherOptions.getJob(); - File jarFile = new File(pluginRoot + File.separator + "flinkx.jar"); + String coreJarName = getCoreJarFileName(pluginRoot); + File jarFile = new File(pluginRoot + File.separator + coreJarName); List urlList = analyzeUserClasspath(content, pluginRoot); String[] remoteArgs = argList.toArray(new String[argList.size()]); PackagedProgram program = new PackagedProgram(jarFile, urlList, remoteArgs); @@ -101,4 +105,27 @@ public static void main(String[] args) throws Exception { clusterClient.shutdown(); } } + + private static String getCoreJarFileName (String pluginRoot) throws FileNotFoundException{ + String coreJarFileName = null; + File pluginDir = new File(pluginRoot); + if (pluginDir.exists() && pluginDir.isDirectory()){ + File[] jarFiles = pluginDir.listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + return name.toLowerCase().startsWith(CORE_JAR_NAME_PREFIX) && name.toLowerCase().endsWith(".jar"); + } + }); + + if (jarFiles != null && jarFiles.length > 0){ + coreJarFileName = jarFiles[0].getName(); + } + } + + if (StringUtils.isEmpty(coreJarFileName)){ + throw new FileNotFoundException("Can not find core jar file in path:" + pluginRoot); + } + + return coreJarFileName; + } } From 9ab08d48a75d4f1892850cd4877c8721d479e03d Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Wed, 8 May 2019 23:03:46 +0800 Subject: [PATCH 95/96] =?UTF-8?q?=E6=9B=B4=E6=96=B0=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 115 ++++--- docs/db2reader.md | 152 --------- docs/db2writer.md | 156 --------- docs/esreader.md | 168 +++++----- docs/eswriter.md | 225 ++++++------- docs/ftpreader.md | 257 +++++++-------- docs/ftpwriter.md | 198 +++++------- docs/hbasereader.md | 207 +++++------- docs/hbasewriter.md | 296 ++++++++---------- docs/hdfsreader.md | 256 +++++++-------- docs/hdfswriter.md | 243 ++++++-------- docs/mongodbreader.md | 197 +++++++----- docs/mongodbwriter.md | 192 ++++++------ docs/mysqldreader.md | 145 --------- docs/mysqlreader.md | 151 --------- docs/mysqlwriter.md | 171 ---------- docs/odpsreader.md | 204 ++++++------ docs/odpswriter.md | 162 ++++------ docs/oraclereader.md | 143 --------- docs/oraclewriter.md | 156 --------- docs/postgresqlreader.md | 101 ------ docs/postgresqlwriter.md | 120 ------- docs/rdbdreader.md | 158 ++++++++++ docs/rdbreader.md | 241 ++++++++++++++ docs/rdbwriter.md | 134 ++++++++ docs/rediswriter.md | 226 +++++++------ docs/sqlserverreader.md | 161 ---------- docs/sqlserverwriter.md | 159 ---------- .../dtstack/flinkx/es/reader/EsReader.java | 2 +- .../dtstack/flinkx/hdfs/HdfsConfigKeys.java | 6 - .../flinkx/hdfs/writer/HdfsWriter.java | 11 - .../flinkx/odps/writer/OdpsWriter.java | 5 +- 32 files changed, 1878 insertions(+), 3240 deletions(-) delete mode 100644 docs/db2reader.md delete mode 100644 docs/db2writer.md delete mode 100644 docs/mysqldreader.md delete mode 100644 docs/mysqlreader.md delete mode 100644 docs/mysqlwriter.md delete mode 100644 docs/oraclereader.md delete mode 100644 docs/oraclewriter.md delete mode 100644 docs/postgresqlreader.md delete mode 100644 docs/postgresqlwriter.md create mode 100644 docs/rdbdreader.md create mode 100644 docs/rdbreader.md create mode 100644 docs/rdbwriter.md delete mode 100644 docs/sqlserverreader.md delete mode 100644 docs/sqlserverwriter.md diff --git a/README.md b/README.md index 599f8f149e..4ee91d03bf 100644 --- a/README.md +++ b/README.md @@ -4,11 +4,10 @@ * **FlinkX是在是袋鼠云内部广泛使用的基于flink的分布式离线数据同步框架,实现了多种异构数据源之间高效的数据迁移。** - 不同的数据源头被抽象成不同的Reader插件,不同的数据目标被抽象成不同的Writer插件。理论上,FlinkX框架可以支持任意数据源类型的数据同步工作。作为一套生态系统,每接入一套新数据源该新加入的数据源即可实现和现有的数据源互通。

## 2 工作原理 @@ -16,14 +15,13 @@ 在底层实现上,FlinkX依赖Flink,数据同步任务会被翻译成StreamGraph在Flink上执行,工作原理如下图:
- +
## 3 快速起步 ### 3.1 运行模式 - * 单机模式:对应Flink集群的单机模式 * standalone模式:对应Flink集群的分布式模式 * yarn模式:对应Flink集群的yarn模式 @@ -34,7 +32,6 @@ * Flink集群: 1.4及以上(单机模式不需要安装Flink集群) * 操作系统:理论上不限,但是目前只编写了shell启动脚本,用户可以可以参考shell脚本编写适合特定操作系统的启动脚本。 - ### 3.3 打包 进入项目根目录,使用maven打包: @@ -50,39 +47,46 @@ mvn clean package -Dmaven.test.skip #### 3.4.1 命令行参数选项 * **model** - * 描述:执行模式,也就是flink集群的工作模式 - * local: 本地模式 - * standalone: 独立部署模式的flink集群 - * yarn: yarn模式的flink集群 - * 必选:否 - * 默认值:local + + * 描述:执行模式,也就是flink集群的工作模式 + * local: 本地模式 + * standalone: 独立部署模式的flink集群 + * yarn: yarn模式的flink集群,需要提前在yarn上启动一个flink session,使用默认名称"Flink session cluster" + * 必选:否 + * 默认值:local * **job** - * 描述:数据同步任务描述文件的存放路径;该描述文件中使用json字符串存放任务信息。 - * 必选:是 - * 默认值:无 - + + * 描述:数据同步任务描述文件的存放路径;该描述文件中使用json字符串存放任务信息。 + * 必选:是 + * 默认值:无 + * **plugin** - * 描述:插件根目录地址,也就是打包后产生的plugins目录。 - * 必选:是 - * 默认值:无 - + + * 描述:插件根目录地址,也就是打包后产生的plugins目录。 + * 必选:是 + * 默认值:无 + * **flinkconf** - * 描述:flink配置文件所在的目录(单机模式下不需要),如/hadoop/flink-1.4.0/conf - * 必选:否 - * 默认值:无 - + + * 描述:flink配置文件所在的目录(单机模式下不需要),如/hadoop/flink-1.4.0/conf + * 必选:否 + * 默认值:无 + * **yarnconf** - * 描述:Hadoop配置文件(包括hdfs和yarn)所在的目录(单机模式下不需要),如/hadoop/etc/hadoop - * 必选:否 - * 默认值:无 + + * 描述:Hadoop配置文件(包括hdfs和yarn)所在的目录(单机模式下不需要),如/hadoop/etc/hadoop + * 必选:否 + * 默认值:无 #### 3.4.2 启动数据同步任务 + * **以本地模式启动数据同步任务** ``` bin/flinkx -mode local -job /Users/softfly/company/flink-data-transfer/jobs/task_to_run.json -plugin /Users/softfly/company/flink-data-transfer/plugins ``` + * **以standalone模式启动数据同步任务** ``` @@ -101,12 +105,13 @@ bin/flinkx -mode yarn -job /Users/softfly/company/flinkx/jobs/mysql_to_mysql.jso ``` { - "job": { - "setting": {...}, - "content": [...] - } + "job": { + "setting": {...}, + "content": [...] + } } ``` + 数据同步任务包括一个job元素,而这个元素包括setting和content两部分。 * setting: 用于配置限速、错误控制和脏数据管理 @@ -115,12 +120,13 @@ bin/flinkx -mode yarn -job /Users/softfly/company/flinkx/jobs/mysql_to_mysql.jso ### 4.1 setting ``` - "setting": { - "speed": {...}, - "errorLimit": {...}, - "dirty": {...} - } + "setting": { + "speed": {...}, + "errorLimit": {...}, + "dirty": {...} + } ``` + setting包括speed、errorLimit和dirty三部分,分别描述限速、错误控制和脏数据管理的配置信息 #### 4.1.1 speed @@ -133,7 +139,7 @@ setting包括speed、errorLimit和dirty三部分,分别描述限速、错误 ``` * channel: 任务并发数 -* bytes: 每秒字节数,默认为0(不限速) +* bytes: 每秒字节数,默认为 Long.MAX_VALUE #### 4.1.2 errorLimit @@ -150,7 +156,7 @@ setting包括speed、errorLimit和dirty三部分,分别描述限速、错误 #### 4.1.3 dirty ``` - "dirty": { + "dirty": { "path": "/tmp", "hadoopConfig": { "fs.default.name": "hdfs://ns1", @@ -176,18 +182,17 @@ setting包括speed、errorLimit和dirty三部分,分别描述限速、错误 "reader": { "name": "...", "parameter": { - ... + ... } }, "writer": { "name": "...", "parameter": { - ... + ... } } } ] - ``` * reader: 用于读取数据的插件的信息 @@ -203,46 +208,34 @@ reader和writer包括name和parameter,分别表示插件名称和插件参数 ### 5.1 读取插件 -* [MySQL读取插件](docs/mysqlreader.md) -* [MySQL分库分表读取插件](docs/mysqldreader.md) -* [Oracle读取插件](docs/oraclereader.md) -* [SQLServer读取插件](docs/sqlserverreader.md) +* [关系数据库读取插件](docs/rdbreader.md) +* [分库分表读取插件](docs/rdbdreader.md) * [HDFS读取插件](docs/hdfsreader.md) * [HBase读取插件](docs/hbasereader.md) * [Elasticsearch读取插件](docs/esreader.md) * [Ftp读取插件](docs/ftpreader.md) * [Odps读取插件](docs/odpsreader.md) -* [PostgreSQL读取插件](docs/postgresqlreader.md) * [MongoDB读取插件](docs/mongodbreader.md) -* [DB2读取插件](docs/db2reader.md) +* [Stream读取插件](docs/streamreader.md) +* [Carbondata读取插件](docs/carbondatareader.md) ### 5.2 写入插件 -* [MySQL写入插件](docs/mysqlwriter.md) -* [Oracle写入插件](docs/oraclewriter.md) -* [SQLServer写入插件](docs/sqlserverwriter.md) +* [关系数据库写入插件](docs/rdbwriter.md) * [HDFS写入插件](docs/hdfswriter.md) * [HBase写入插件](docs/hbasewriter.md) * [Elasticsearch写入插件](docs/eswriter.md) * [Ftp写入插件](docs/ftpwriter.md) * [Odps写入插件](docs/odpswriter.md) -* [PostgreSQL写入插件](docs/postgresqlwriter.md) * [MongoDB写入插件](docs/mongodbwriter.md) * [Redis写入插件](docs/rediswriter.md) -* [DB2写入插件](docs/db2writer.md) +* [Stream写入插件](docs/streamwriter.md) +* [Carbondata写入插件](docs/carbondatawriter.md) ## 6.版本说明 - 1.flinkx的分支版本跟flink的版本对应,比如:flinkx v1.4.0 对应 flink1.4.0,现在支持flink1.4和1.5 + 1.flinkx的分支版本跟flink的版本对应,比如:flinkx v1.4.0 对应 flink1.4.0,现在支持flink1.4和1.5 ## 7.招聘信息 - 1.大数据平台开发工程师,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至sishu@dtstack.com。 - - - - - - - - + 1.大数据平台开发工程师,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至sishu@dtstack.com。 diff --git a/docs/db2reader.md b/docs/db2reader.md deleted file mode 100644 index 3a1da429b1..0000000000 --- a/docs/db2reader.md +++ /dev/null @@ -1,152 +0,0 @@ -# MySQL分库分表读取插件(db2reader) - -## 1. 配置样例 - -``` -{ - "job": { - "content": [ - { - "reader": { - "fetchSize": 1024, - "parameter": { - "password": "abc123", - "column": [ - "smallint_col", - "integer_col", - "bigint_col", - "decimal_col", - "real_col", - "double_col" - ], - "where": "", - "connection": [ - { - "password": "abc123", - "jdbcUrl": ["jdbc:db2://172.16.1.191:50000/flinkx"], - "table": [ - "db2_stand_all" - ], - "username": "dtstack" - } - ], - "splitPk": "", - "username": "dtstack" - }, - "name": "db2reader" - }, - "writer": { - "parameter": { - "print":true - }, - "name": "streamwriter" - } - } - ], - "setting": { - "errorLimit": { - }, - "speed": { - "bytes": 0, - "channel": 1 - } - } - } -} - -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填db2reader,否则Flinkx将无法正常加载该插件包。 - - * 必选:是
- - * 默认值:无
- -* **connection** - - * 描述:需要读取的数据源数组。 - - * 必选:是 - - * 默认值:无 - - * 元素: - - * username:具体数据源的用户名,如果不填则使用全局的用户名。 - - * password:具体数据源的密码,如果不填则使用全局的密码。 - - * jdbcUrl:数据源连接url,只支持写单个连接。 - - * table:要查询的表名称,可写多张表,多张表的表结构必须一致。 - -* **jdbcUrl** - - * 描述:针对db2数据库的jdbc连接字符串 - - jdbcUrl按照DB2官方规范,并可以填写连接附件控制信息。具体请参看[DB2官方文档](https://www.ibm.com/analytics/us/en/db2/)。 - - * 必选:是
- - * 默认值:无
- -* **username** - - * 描述:全局数据源的用户名
- - * 必选:否
- - * 默认值:无
- -* **password** - - * 描述:全局数据源的密码
- - * 必选:否
- - * 默认值:无
- -* **where** - - * 描述:筛选条件,MysqldReader根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > $bizdate 。注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。
- - where条件可以有效地进行业务增量同步。如果不填写where语句,包括不提供where的key或者value,FlinkX均视作同步全量数据。 - - * 必选:否
- - * 默认值:无
- -* **splitPk** - - * 描述:MysqldReader进行数据抽取时,如果指定splitPk,表示用户希望使用splitPk代表的字段进行数据分片,FlinkX因此会启动并发任务进行数据同步,这样可以大大提供数据同步的效能。 - - 推荐splitPk用户使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 - -  目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,MysqldReader将报错! - -  如果splitPk不填写,包括不提供splitPk或者splitPk值为空,FlinkX视作使用单通道同步该表数据。 - - * 必选:否
- - * 默认值:空
- - - -* **column** - - * 描述:所配置的表中需要同步的列名集合。 - - 支持列裁剪,即列可以挑选部分列进行导出。 - - 支持列换序,即列可以不按照表schema信息进行导出。 - - 暂不支持常量列。 - - * 必选:是
- - * 默认值:无
- diff --git a/docs/db2writer.md b/docs/db2writer.md deleted file mode 100644 index de205310ec..0000000000 --- a/docs/db2writer.md +++ /dev/null @@ -1,156 +0,0 @@ -# MySQL写入插件(db2writer) - -## 1. 配置样例 - -``` -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "column": [ - { - "name": "id", - "type": "int", - "value": 26 - }, - { - "name": "name", - "type": "string", - "value": "xxxxxx" - } - ], - "sliceRecordCount": 2 - }, - "name": "streamreader" - }, - "writer": { - "parameter": { - "postSql": [], - "password": "abc123", - "session": [], - "column": [ - "id", - "name" - ], - "connection": [ - { - "jdbcUrl": "jdbc:db2://172.16.1.191:50000/flinkx", - "table": [ - "flinkx_test" - ] - } - ], - "writeMode": "replace", - "preSql": [], - "username": "dtstack", - "batchSize":1 - }, - "name": "db2writer" - } - } - ], - "setting": { - "errorLimit": { - "record": 0, - "percentage": 0 - }, - "speed": { - "bytes": 0, - "channel": 1 - } - } - } -} - -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填db2writer,否则Flinkx将无法正常加载该插件包。 - * 必选:是
- - * 默认值:无
- -* **jdbcUrl** - - * 描述:针对db2数据库的jdbc连接字符串 - - jdbcUrl按照DB2官方规范,并可以填写连接附件控制信息。具体请参看[DB2官方文档](https://www.ibm.com/analytics/us/en/db2/)。 - - * 必选:是
- - * 默认值:无
- -* **username** - - * 描述:数据源的用户名
- - * 必选:是
- - * 默认值:无
- -* **password** - - * 描述:数据源指定用户名的密码
- - * 必选:是
- - * 默认值:无
- -* **column** - - * 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 - - * 必选:是
- - * 默认值:否
- - * 默认值:无
- -* **preSql** - - * 描述:写入数据到目的表前,会先执行这里的一组标准语句。 - - * 必选:否
- - * 默认值:无
- -* **postSql** - - * 描述:写入数据到目的表后,会执行这里的一组标准语句。 - - * 必选:否
- - * 默认值:无
- -* **table** - - * 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表。 - - 注意:table 和 jdbcUrl 必须包含在 connection 配置单元中 - - * 必选:是
- - * 默认值:无
- -* **writeMode** - - * 描述:控制写入数据到目标表采用 `insert into` 或者 `merge into` 语句
- - * 必选:是
- - * 所有选项:insert/replace/update
- - * 默认值:insert
- -* **batchSize** - - * 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与Mysql的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况。
- - * 必选:否
- - * 默认值:1024
\ No newline at end of file diff --git a/docs/esreader.md b/docs/esreader.md index 0313ba62d2..8ac84506e8 100644 --- a/docs/esreader.md +++ b/docs/esreader.md @@ -4,105 +4,95 @@ ``` { - "job": { - "setting": { - "speed": { - "channel": 2, - "bytes": 10000 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { - "name": "esreader", - "parameter": { - "address": "rdos1:9200,rdos2:9200", - "query": { - "match": { - "col2": "hallo" - } - }, - "column": [ - { - "name": "xx.yy.zz", - "type": "string" - }, - { - "name": "col2", - "type": "string" - } - ] - } - }, - "writer": { - "name": "mysqlwriter", - "parameter": { - "writeMode": "insert", - "username": "dtstack", - "password": "abc123", - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true", - "table": [ - "tb333" - ] - } - ] - } - } - } - ] - } + "job": { + "setting": {}, + "content": [{ + "reader": { + "name": "esreader", + "parameter": { + "address": "host1:9200,host2:9200", + "query": { + "match": { + "match_all": {} + } + }, + "index": "indexTest", + "type": "type1", + "batchSize": 0, + "timeout": 10, + "column": [{ + "name": "xx.yy.zz", + "type": "string", + "value": "value" + }] + } + }, + "writer": {} + }] + } } - ``` ## 2. 参数说明 * **address** + + * 描述:Elasticsearch地址,单个节点地址采用host:port形式,多个节点的地址用逗号连接 + + * 必选:是 + + * 默认值:无 - * 描述:Elasticsearch地址,单个节点地址采用host:port形式,多个节点的地址用逗号连接
- - * 必选:是
- - * 默认值:无
- * **query** + + * 描述:Elasticsearch查询表达式,[查询表达式](https://www.elastic.co/guide/cn/elasticsearch/guide/current/query-dsl-intro.html) + + * 必选:否 + + * 默认值:无,默认为全查询 - * 描述:Elasticsearch查询表达式,[查询表达式](https://www.elastic.co/guide/cn/elasticsearch/guide/current/query-dsl-intro.html)
+* **batchSize** + + * 描述:每次读取数据条数 + + * 必选:否 + + * 默认值:10 - * 必选:否
+* **timeout** + + * 描述:连接超时时间 + + * 必选:否 + + * 默认值:无 - * 默认值:无,默认为全查询
- -* **column** - - * 描述:读取elasticsearch的查询结果的若干个列,每列形式如下
- * 普通列 - - ``` - { - "name": "xx.yy.zz", //支持列的多级嵌套,用.连接 - "type": "string" - } - ``` - * 常数列 - - ``` - { - "value": "xxx", // 常量值 - "type": "string" //常量类型 - } - ``` +* **index** + + * 描述:要查询的索引名称 + + * 必选:否 + + * 默认值:无 - * 必选:是
+* **type** + + * 描述:要查询的类型 + + * 必选:否 + + * 默认值:无 - * 默认值:无
+* **column** + + * 描述:读取elasticsearch的查询结果的若干个列,每列形式如下 + + * name:字段名称,可使用多级格式查找 + + * type:字段类型,当name没有指定时,则返回常量列,值为value指定 + + * value:常量列的值 + + * 必选:是 + + * 默认值:无 diff --git a/docs/eswriter.md b/docs/eswriter.md index 1b4ffacf29..da8f3a49a4 100644 --- a/docs/eswriter.md +++ b/docs/eswriter.md @@ -4,145 +4,118 @@ ``` { - "job": { - "setting": { - "speed": { - "channel": 3, - "bytes": 10000000 - }, - "errorLimit": { - "record": 0, - "percentage": 20 - } - }, - "content": [ - { - "reader": { - "name": "mysqlreader", - "parameter": { - "username": "dtstack", - "password": "abc123", - "column": [ - "col1", - "col2" - ], - "splitPk": "col1", - "connection": [ - { - "table": [ - "tb2" - ], - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true" - ] - } - ] - } - }, - "writer": { - "name": "eswriter", - "parameter": { - "address": "rdos1:9200,rdos2:9200", - "index": "yoshi", - "type": "nani", - "bulkAction": 3, - "idColumn": [ - { - "index": 0, - "type": "int" - } - ], - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - } - ] - } - } - } - ] - } + "job": { + "setting": {}, + "content": [{ + "reader": {}, + "writer": { + "name": "eswriter", + "parameter": { + "address": "host1:9200,host2:9200", + "index": "indexTest", + "type": "type1", + "bulkAction": 100, + "timeout": 100, + "idColumn": [{ + "index": 0, + "type": "int" + }], + "column": [{ + "name": "col1", + "type": "string" + }] + } + } + }] + } } ``` ## 2. 参数说明 * **address** + + * 描述:Elasticsearch地址,单个节点地址采用host:port形式,多个节点的地址用逗号连接 + + * 必选:是 + + * 默认值:无 - * 描述:Elasticsearch地址,单个节点地址采用host:port形式,多个节点的地址用逗号连接
- - * 必选:是
- - * 默认值:无
- * **index** + + * 描述:Elasticsearch 索引值 + + * 必选:是 + + * 默认值:无 - * 描述:Elasticsearch 索引值
- - * 必选:是
- - * 默认值:无
- * **type** + + * 描述:Elasticsearch 索引类型 + + * 必选:是 + + * 默认值:无 - * 描述:Elasticsearch 索引类型
- - * 必选:是
- - * 默认值:无
- * **column** + + * 描述:写入elasticsearch的若干个列,每列形式如下 + + ``` + { + "name": "列名", + "type": "列类型" + } + ``` + + * 必选:是 + + * 默认值:无 - * 描述:写入elasticsearch的若干个列,每列形式如下
- - ``` - { - "name": "列名", - "type": "列类型" - } - ``` - - * 必选:是
- - * 默认值:无
- * **idColumns** + + * 描述:用于构造文档id的若干个列,每列形式如下 + + * 普通列 + + ``` + { + "index": 0, // 前面column属性中列的序号,从0开始 + "type": "string" 列的类型,默认为string + } + ``` + + * 常数列 + + ``` + { + "value": "ffff", // 常数值 + "type": "string" // 常数列的类型,默认为string + } + ``` + + * 必选:否 + + * 注意: + + * 如果不指定idColumns属性,则会随机产生文档id + + * 如果指定的字段值存在重复或者指定了常数,按照es的逻辑,同样值的doc只会保留一份 + + * 默认值:无 - * 描述:用于构造文档id的若干个列,每列形式如下
- - * 普通列 - - ``` - { - "index": 0, // 前面column属性中列的序号,从0开始 - "type": "string" 列的类型,默认为string - } - ``` - - * 常数列 - - ``` - { - "value": "ffff", // 常数值 - "type": "string" // 常数列的类型,默认为string - } - ``` - - * 必选:否
- 如果不指定idColumns属性,则会随机产生文档id - - * 默认值:无
- - * **bulkAction** - - * 描述:批量写入的记录条数
- - * 必选:是
- - * 默认值:100
\ No newline at end of file + + * 描述:批量写入的记录条数 + + * 必选:是 + + * 默认值:100 + +* **timeout** + + * 描述:连接超时时间,如果bulkAction指定的数值过大,写入数据可能会超时,这时可以配置超时时间 + + * 必选:否 + + * 默认值:无 diff --git a/docs/ftpreader.md b/docs/ftpreader.md index a889116135..ba7b5ee95a 100644 --- a/docs/ftpreader.md +++ b/docs/ftpreader.md @@ -5,175 +5,144 @@ ``` { "job": { - "setting": { - "speed": { - "channel": 1, - "bytes": 10000 - }, - "errorLimit": { - "record": 0, - "percentage": 50 - } - }, - "content": [ - { - "reader": { - "name": "ftpreader", - "parameter": { - "protocol": "sftp", - "host": "node01" , - "port": 22, - "username": "mysftp", - "password": "oh1986mygod", - "column": [ - { - "index": 0 - }, - { - "index": 1 - }, - { - "value": "youcan", - "type": "string" - } - ], - "path": "/upload", - "encoding": "UTF-8", - "fieldDelimiter": "\\t", - "isFirstLineHeader":true - } - }, - "writer": { - "parameter": { - "password": "abc123", - "column": [ - "col1", - "col2", - "col3" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?charset=utf8", - "table": [ - "sb5" - ] - } - ], - "writeMode": "insert", - "username": "dtstack" - }, - "name": "mysqlwriter" + "setting": {}, + "content": [{ + "reader": { + "name": "ftpreader", + "parameter": { + "protocol": "sftp", + "host": "127.0.0.1", + "port": 22, + "username": "username", + "password": "password", + "column": [{ + "index": 0, + "type": "", + "value": "value" + }], + "path": "/upload", + "encoding": "UTF-8", + "fieldDelimiter": ",", + "isFirstLineHeader": true } - } - ] + }, + "writer": {} + }] } } - ``` ## 2. 参数说明 * **protocol** - - * 描述:ftp服务器协议,目前支持传输协议有ftp和sftp。
- - * 必选:是
- - * 默认值:无
+ + * 描述:ftp服务器协议,目前支持传输协议有ftp和sftp。 + + * 必选:是 + + * 默认值:无 * **host** - - * 描述:ftp服务器地址。
- - * 必选:是
- - * 默认值:无
+ + * 描述:ftp服务器地址。 + + * 必选:是 + + * 默认值:无 * **port** - - * 描述:ftp服务器端口。
- - * 必选:否
- - * 默认值:若传输协议是sftp协议,默认值是22;若传输协议是标准ftp协议,默认值是21
+ + * 描述:ftp服务器端口。 + + * 必选:否 + + * 默认值:若传输协议是sftp协议,默认值是22;若传输协议是标准ftp协议,默认值是21 * **connectPattern** - - * 描述:连接模式(主动模式或者被动模式)。该参数只在传输协议是标准ftp协议时使用,值只能为:PORT (主动),PASV(被动)。两种模式主要的不同是数据连接建立的不同。对于Port模式,是客户端在本地打开一个端口等服务器去连接建立数据连接,而Pasv模式就是服务器打开一个端口等待客户端去建立一个数据连接。
- - * 必选:否
- - * 默认值:PASV
+ + * 描述:连接模式(主动模式或者被动模式)。该参数只在传输协议是标准ftp协议时使用,值只能为:PORT (主动),PASV(被动)。两种模式主要的不同是数据连接建立的不同。对于Port模式,是客户端在本地打开一个端口等服务器去连接建立数据连接,而Pasv模式就是服务器打开一个端口等待客户端去建立一个数据连接。 + + * 必选:否 + + * 默认值:PASV * **username** - - * 描述:ftp服务器访问用户名。
- - * 必选:是
- - * 默认值:无
+ + * 描述:ftp服务器访问用户名。 + + * 必选:是 + + * 默认值:无 * **password** - - * 描述:ftp服务器访问密码。
- - * 必选:是
- - * 默认值:无
+ + * 描述:ftp服务器访问密码。 + + * 必选:是 + + * 默认值:无 * **path** - - * 描述:远程FTP文件系统的路径信息,注意这里可以支持填写多个路径。
- - * 必选:是
- - * 默认值:/
+ + * 描述:远程FTP文件系统的路径信息,注意这里可以支持填写多个路径。 + + * 必选:是 + + * 默认值:/ * **column** - - * 描述:读取字段列表,type指定源数据的类型,index指定当前列来自于文本第几列(以0开始),value指定当前类型为常量。 - - - 用户可以指定column字段信息,配置如下: - - ```json - { - "index": 0 //从远程FTP文件文本第一列获取int字段 - }, - { - "type": "string", - "value": "alibaba" //从FtpReader内部生成alibaba的字符串字段作为当前字段 - } - ``` - - 对于用户指定Column信息,type必须填写,index/value必须选择其一。 - - * 必选:是
- - * 默认值:全部按照string类型读取
+ + * 描述:需要读取的字段。 + + * 格式:支持2中格式 + + 1.读取全部字段,如果字段数量很多,可以使用下面的写法: + + ``` + "column":["*"] + ``` + + 2.指定具体信息: + + ``` + "column": [{ + "index": 0, + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" + }] + ``` + + * 属性说明: + + * index:字段索引 + + * type:字段类型,ftp读取的为文本文件,本质上都是字符串类型,这里可以指定要转成的类型 + + * format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + + * value:如果没有指定index,则会把value的值作为常量列返回,如果指定了index,当读取的字段的值为null时,会以此value值作为默认值返回 + + * 必选:是 + + * 默认值:无 * **fieldDelimiter** - - * 描述:读取的字段分隔符
- - * 必选:是
- - * 默认值:,
+ + * 描述:读取的字段分隔符 + + * 必选:是 + + * 默认值:, * **encoding** + + * 描述:读取文件的编码配置。 + * 必选:否 + * 默认值:utf-8 - * 描述:读取文件的编码配置。
- - * 必选:否
- - * 默认值:utf-8
- * **isFirstLineHeader** - - * 描述:首行是否为标题行,如果是则不读取第一行。
- - * 必选:否
- - * 默认值:false
- - + + * 描述:首行是否为标题行,如果是则不读取第一行。 + * 必选:否 + * 默认值:false diff --git a/docs/ftpwriter.md b/docs/ftpwriter.md index 873a2259ea..e4da51b536 100644 --- a/docs/ftpwriter.md +++ b/docs/ftpwriter.md @@ -5,149 +5,107 @@ ``` { "job": { - "setting": { - "speed": { - "channel": 2 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { + "setting": {}, + "content": [{ + "reader": {}, + "writer": { + "name": "ftpwriter", "parameter": { - "password": "abc123", - "columnTypes": [ - "java.lang.String", - "java.lang.String" - ], - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?charset=utf8" - ], - "table": [ - "tb1" - ] - } - ], - "splitPk": "col1", - "username": "dtstack" - }, - "name": "mysqlreader" - }, - "writer": { - "name": "ftpwriter", - "parameter": { - "protocol": "sftp", - "host": "node03", - "port": 22, - "username": "mysftp", - "password": "oh1986mygod", - "writeMode": "overwrite", - "path": "/upload/xxx", - "fieldDelimiter": ",", - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - } - ] - } + "protocol": "sftp", + "host": "127.0.0.1", + "port": 22, + "username": "username", + "password": "password", + "writeMode": "overwrite", + "path": "/sftp", + "fieldDelimiter": ",", + "connectPattern": "PASV", + "column": [{ + "type": "string" + }] } } - ] + }] } } - ``` ## 2. 参数说明 * **protocol** + + * 描述:ftp服务器协议,目前支持传输协议有ftp和sftp。 + + * 必选:是 + + * 默认值:无 - * 描述:ftp服务器协议,目前支持传输协议有ftp和sftp。
- - * 必选:是
- - * 默认值:无
- * **host** + + * 描述:ftp服务器地址。 + + * 必选:是 + + * 默认值:无 - * 描述:ftp服务器地址。
- - * 必选:是
- - * 默认值:无
- * **port** + + * 描述:ftp服务器端口。 + + * 必选:否 + + * 默认值:若传输协议是sftp协议,默认值是22;若传输协议是标准ftp协议,默认值是21 - * 描述:ftp服务器端口。
- - * 必选:否
- - * 默认值:若传输协议是sftp协议,默认值是22;若传输协议是标准ftp协议,默认值是21
- - * **username** - - * 描述:ftp服务器访问用户名。
- - * 必选:是
- - * 默认值:无
+ + * 描述:ftp服务器访问用户名。 + + * 必选:是 + + * 默认值:无 * **password** - - * 描述:ftp服务器访问密码。
- - * 必选:是
- - * 默认值:无
+ + * 描述:ftp服务器访问密码。 + + * 必选:是 + + * 默认值:无 + +* **connectPattern** + + * 描述:连接模式(主动模式或者被动模式)。该参数只在传输协议是标准ftp协议时使用,值只能为:PORT (主动),PASV(被动)。两种模式主要的不同是数据连接建立的不同。对于Port模式,是客户端在本地打开一个端口等服务器去连接建立数据连接,而Pasv模式就是服务器打开一个端口等待客户端去建立一个数据连接。 + + * 必选:否 + + * 默认值:PASV * **path** - - * 描述:FTP文件系统的路径信息,FtpWriter会写入Path目录下属多个文件。
- - * 必选:是
- - * 默认值:无
- + + * 描述:FTP文件系统的路径信息,FtpWriter会写入Path目录下属多个文件。 + + * 必选:是 + + * 默认值:无 * **writeMode** - - * 描述:FtpWriter写入前数据清理处理模式:
- - * overwrite,覆盖 - * append,追加 - - * 必选:是
- - * 默认值:无
+ + * 描述:FtpWriter写入前数据清理处理模式: + * overwrite,覆盖 + * append,追加 + * 必选:是 + * 默认值:无 * **fieldDelimiter** + + * 描述:写入的字段分隔符 + + * 必选:否 + + * 默认值:, - * 描述:读取的字段分隔符
- - * 必选:否
- - * 默认值:,
- * **encoding** - - * 描述:读取文件的编码配置。
- - * 必选:否
- - * 默认值:utf-8
- + + * 描述:读取文件的编码配置。 + * 必选:否 + * 默认值:utf-8 diff --git a/docs/hbasereader.md b/docs/hbasereader.md index 422b0187c4..a076698337 100644 --- a/docs/hbasereader.md +++ b/docs/hbasereader.md @@ -4,142 +4,99 @@ ``` { - "job": { - "setting": { - "speed": { - "channel": 2, - "bytes": 10000 - }, - "errorLimit": { - "record": 0, - "percentage": 2 - } - }, - "content": [ - { - "reader": { - "name": "hbasereader", - "parameter": { - "hbaseConfig": { - "hbase.zookeeper.property.clientPort": "2181", - "hbase.rootdir": "hdfs://ns1/hbase", - "hbase.cluster.distributed": "true", - "hbase.zookeeper.quorum": "node01,node02,node03", - "zookeeper.znode.parent": "/hbase" - }, - "table": "sb5", - "encodig": "utf-8", - "column": [ - { - "name": "rowkey", - "type": "string" - }, - { - "name": "cf1:id", - "type": "string" - } - ], - "range": { - "startRowkey": "", - "endRowkey": "", - "isBinaryRowkey": true - } - } - }, - "writer": { - "parameter": { - "password": "abc123", - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?charset=utf8", - "table": [ - "sb5" - ] - } - ], - "writeMode": "insert", - "username": "dtstack" - }, - "name": "mysqlwriter" - } - } - ] - } + "job": { + "setting": {}, + "content": [{ + "reader": { + "name": "hbasereader", + "parameter": { + "hbaseConfig": { + "hbase.zookeeper.property.clientPort": "2181", + "hbase.rootdir": "hdfs://ns1/hbase", + "hbase.cluster.distributed": "true", + "hbase.zookeeper.quorum": "host1,host2,host3", + "zookeeper.znode.parent": "/hbase" + }, + "table": "tableTest", + "encodig": "utf-8", + "column": [{ + "name": "rowkey", + "type": "string" + }, + { + "name": "cf1:id", + "type": "string" + } + ], + "range": { + "startRowkey": "", + "endRowkey": "", + "isBinaryRowkey": true + } + } + }, + "writer": {} + }] + } } - ``` ## 2. 参数说明 * **hbaseConfig** + + * 描述:hbase的连接配置,以json的形式组织 (见hbase-site.xml) + + * 必选:是 + + * 默认值:无 - * 描述:hbase的连接配置,以json的形式组织 (见hbase-site.xml)
- - * 必选:是
- - * 默认值:无
- * **encoding** + + * 描述:字符编码 + + * 必选:无 + + * 默认值:utf-8 - * 描述:字符编码
- - * 必选:无
- - * 默认值:utf-8
- * **table** + + * 描述:hbase表名 + + * 必选:是 + + * 默认值:无 - * 描述:hbase表名
- - * 必选:是
- - * 默认值:无
- * **range** - - * 描述:指定hbasereader读取的rowkey范围。
- startRowkey:指定开始rowkey;
- endRowkey指定结束rowkey;
- isBinaryRowkey:指定配置的startRowkey和endRowkey转换为byte[]时的方式,默认值为false,若为true,则调用Bytes.toBytesBinary(rowkey)方法进行转换;若为false:则调用Bytes.toBytes(rowkey)
- 配置格式如下: - - ``` - "range": { - "startRowkey": "aaa", - "endRowkey": "ccc", - "isBinaryRowkey":false -} - ``` -
- - * 必选:否
- - * 默认值:无
+ + * 描述:指定hbasereader读取的rowkey范围。 + + * startRowkey:指定开始rowkey; + + * endRowkey指定结束rowkey; + + * isBinaryRowkey:指定配置的startRowkey和endRowkey转换为byte[]时的方式,默认值为false,若为true,则调用Bytes.toBytesBinary(rowkey)方法进行转换;若为false:则调用Bytes.toBytes(rowkey),配置格式如下: + + ``` + "range": { + "startRowkey": "aaa", + "endRowkey": "ccc", + "isBinaryRowkey":false + } + ``` + + * 必选:否 + + * 默认值:无 * **column** - - * 描述:要读取的hbase字段,normal 模式与multiVersionFixedColumn 模式下必填项。 - name指定读取的hbase列,除了rowkey外,必须为 列族:列名 的格式,type指定源数据的类型,format指定日期类型的格式,value指定当前类型为常量,不从hbase读取数据,而是根据value值自动生成对应的列。配置格式如下: - - ``` - "column": -[ - { - "name": "rowkey", - "type": "string" - }, - { - "value": "test", - "type": "string" - } -] - - ``` - - * 必选:是
- - * 默认值:无
\ No newline at end of file + + * 描述:要读取的hbase字段,normal 模式与multiVersionFixedColumn 模式下必填项。 + + * name:指定读取的hbase列,除了rowkey外,必须为 列族:列名 的格式; + + * type:指定源数据的类型,format指定日期类型的格式,value指定当前类型为常量,不从hbase读取数据,而是根据value值自动生成对应的列。 + + * 必选:是 + + * 默认值:无 diff --git a/docs/hbasewriter.md b/docs/hbasewriter.md index ba627d6824..5c29c46f0c 100644 --- a/docs/hbasewriter.md +++ b/docs/hbasewriter.md @@ -4,187 +4,159 @@ ``` { - "job": { - "setting": { - "speed": { - "channel": 1 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { - "name": "mysqlreader", - "parameter": { - "username": "dtstack", - "password": "abc123", - "column": [ - "col1", - "col2" - ], - "splitPk": "col1", - "connection": [ - { - "table": [ - "tb2" - ], - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true" - ] - } - ] - } - }, - "writer": { - "name": "hbasewriter", - "parameter": { - "hbaseConfig": { - "hbase.zookeeper.property.clientPort": "2181", - "hbase.rootdir": "hdfs://ns1/hbase", - "hbase.cluster.distributed": "true", - "hbase.zookeeper.quorum": "node01,node02,node03", - "zookeeper.znode.parent": "/hbase" - }, - "table": "tb1", - "rowkeyColumn": [ - { - "index": 0, - "type": "string" - }, - { - "value": "_postfix", - "type": "string" - } - ], - "column": [ - { - "name": "cf1:id", - "type": "string" - }, - { - "name": "cf1:vv", - "type": "string" - } - ] - } - } - } - ] - } + "job": { + "setting": { + "speed": {}, + "content": [{ + "reader": {}, + "writer": { + "name": "hbasewriter", + "parameter": { + "hbaseConfig": { + "hbase.zookeeper.property.clientPort": "2181", + "hbase.rootdir": "hdfs://ns1/hbase", + "hbase.cluster.distributed": "true", + "hbase.zookeeper.quorum": "host1,host2,host3", + "zookeeper.znode.parent": "/hbase" + }, + "table": "tableTest", + "rowkeyColumn": [{ + "index": 0, + "type": "string" + }, + { + "value": "_postfix", + "type": "string" + } + ], + "column": [{ + "name": "cf1:id", + "type": "string" + }, + { + "name": "cf1:vv", + "type": "string" + } + ] + } + } + }] + } + } } - ``` ## 2. 参数说明 * **hbaseConfig** + + * 描述:hbase的连接配置,以json的形式组织 (见hbase-site.xml) + + * 必选:是 + + * 默认值:无 - * 描述:hbase的连接配置,以json的形式组织 (见hbase-site.xml)
- - * 必选:是
- - * 默认值:无
- * **table** + + * 描述:hbase表名 + + * 必选:是 + + * 默认值:无 - * 描述:hbase表名
- - * 必选:是
- - * 默认值:无
- - * **column** + + * 描述:写入hbase表的若干个列,hbase表的每一列由列簇和列名组成,用":"连接 + + ``` + { + "name": "cf1:id", // 列簇:列名 + "type": "string" // 列类型 + } + ``` + + * 必选:是 + + * 默认值:无 - * 描述:写入hbase表的若干个列,hbase表的每一列由列簇和列名组成,用连接
- - ``` - { - "name": "cf1:id", // 列簇:列名 - "type": "string" // 列类型 - } - ``` - - * 必选:是
- - * 默认值:无
- * **rowkeyColumn** + + * 描述:用于构造rowkey的若干个列,每列形式如下 + + * 普通列 + + ``` + { + "index": 0, // 该列在column属性中的序号,从0开始 + "type": "string" 列的类型,默认为string + } + ``` + + * 常数列 + + ``` + { + "value": "ffff", // 常数值 + "type": "string" // 常数列的类型,默认为string + } + ``` + + * 必选:否 + + 如果不指定idColumns属性,则会随机产生文档id + + * 默认值:无 - * 描述:用于构造rowkey的若干个列,每列形式如下
- - * 普通列 - - ``` - { - "index": 0, // 该列在column属性中的序号,从0开始 - "type": "string" 列的类型,默认为string - } - ``` - - * 常数列 - - ``` - { - "value": "ffff", // 常数值 - "type": "string" // 常数列的类型,默认为string - } - ``` - - * 必选:否
- 如果不指定idColumns属性,则会随机产生文档id - - * 默认值:无
- * **versionColumn** + + * 描述:指定写入hbase的时间戳。支持:当前时间、指定时间列,指定时间,三者选一。若不配置表示用当前时间。index:指定对应reader端column的索引,从0开始,需保证能转换为long,若是Date类型,会尝试用yyyy-MM-dd HH:mm:ss和yyyy-MM-dd HH:mm:ss SSS去解析;若不指定index;value:指定时间的值,类型为字符串。配置格式如下: + + ``` + "versionColumn":{ + "index":1 + } + ``` + + 或者 + + ``` + "versionColumn":{ + "value":"123456789" + } + ``` - * 描述:指定写入hbase的时间戳。支持:当前时间、指定时间列,指定时间,三者选一。若不配置表示用当前时间。index:指定对应reader端column的索引,从0开始,需保证能转换为long,若是Date类型,会尝试用yyyy-MM-dd HH:mm:ss和yyyy-MM-dd HH:mm:ss SSS去解析;若不指定index;value:指定时间的值,类型为字符串。配置格式如下: - - ``` -"versionColumn":{ - "index":1 -} - - ``` - - 或者 - - ``` -"versionColumn":{ - "value":"123456789" -} - * **encoding** + + * 描述:字符编码 + + * 必选:无 + + * 默认值:utf-8 - * 描述:字符编码
- - * 必选:无
- - * 默认值:utf-8
- * **nullMode** + + * 描述:读取的null值时,如何处理。支持两种方式: + + * (1)skip:表示不向hbase写这列; + + * (2)empty:写入HConstants.EMPTY_BYTE_ARRAY,即new byte [0] + + * 必选:否 + + * 默认值:skip - * 描述:读取的null值时,如何处理。支持两种方式:(1)skip:表示不向hbase写这列;(2)empty:写入HConstants.EMPTY_BYTE_ARRAY,即new byte [0]
- - * 必选:否
- - * 默认值:skip
- * **writeBufferSize** + + * 描述:设置HBae client的写buffer大小,单位字节。配合autoflush使用。autoflush,开启(true)表示Hbase client在写的时候有一条put就执行一次更新;关闭(false),表示Hbase client在写的时候只有当put填满客户端写缓存时,才实际向HBase服务端发起写请求 + + * 必选:否 + + * 默认值:8M - * 描述:设置HBae client的写buffer大小,单位字节。配合autoflush使用。autoflush,开启(true)表示Hbase client在写的时候有一条put就执行一次更新;关闭(false),表示Hbase client在写的时候只有当put填满客户端写缓存时,才实际向HBase服务端发起写请求
- - * 必选:否
- - * 默认值:8M
- * **walFlag** - - * 描述:在HBae client向集群中的RegionServer提交数据时(Put/Delete操作),首先会先写WAL(Write Ahead Log)日志(即HLog,一个RegionServer上的所有Region共享一个HLog),只有当WAL日志写成功后,再接着写MemStore,然后客户端被通知提交数据成功;如果写WAL日志失败,客户端则被通知提交失败。关闭(false)放弃写WAL日志,从而提高数据写入的性能。
- - * 必选:否
- - * 默认值:false
\ No newline at end of file + + * 描述:在HBae client向集群中的RegionServer提交数据时(Put/Delete操作),首先会先写WAL(Write Ahead Log)日志(即HLog,一个RegionServer上的所有Region共享一个HLog),只有当WAL日志写成功后,再接着写MemStore,然后客户端被通知提交数据成功;如果写WAL日志失败,客户端则被通知提交失败。关闭(false)放弃写WAL日志,从而提高数据写入的性能。 + + * 必选:否 + + * 默认值:false diff --git a/docs/hdfsreader.md b/docs/hdfsreader.md index dc64cef75a..457d4ee8d4 100644 --- a/docs/hdfsreader.md +++ b/docs/hdfsreader.md @@ -4,161 +4,141 @@ ``` { - "job": { - "content": [ - { - "reader": { - "parameter": { - "path": "hdfs://ns1/user/hive/warehouse/wujing_test.db/kepa_250", - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.nameservices": "ns1" + "job": { + "content": [{ + "reader": { + "parameter": { + "path": "hdfs://ns1/user/hive/warehouse/wujing_test.db/test", + "hadoopConfig": { + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", + "dfs.nameservices": "ns1" + }, + "defaultFS": "hdfs://ns1", + "column": [{ + "name": "col1", + "index": 0, + "type": "string", + "value": "", + "format": "" + }], + "fieldDelimiter": "", + "encoding": "utf-8", + "fileType": "orc" + }, + "name": "hdfsreader" }, - "column": [ - { - "name": "col1", - "index": 0, - "type": "string" - }, - { - "name": "col2", - "index": 1, - "type": "string" - } - ], - "defaultFS": "hdfs://ns1", - "fieldDelimiter": "", - "encoding": "utf-8", - "fileType": "orc" - }, - "name": "hdfsreader" - }, - "writer": { - "parameter": { - "password": "abc123", - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?charset=utf8", - "table": [ - "sb5" - ] - } - ], - "writeMode": "insert", - "username": "dtstack" - }, - "name": "mysqlwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 100 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } + "writer": {} + }], + "setting": {} } - } } ``` - ## 2. 参数说明 * **path** - - * 描述:要读取的文件路径,多个路径可以用逗号隔开 - - * 必选:是
- - * 默认值:无
+ + * 描述:要读取的文件路径,多个路径可以用逗号隔开 + + * 必选:是
+ + * 默认值:无
* **defaultFS** - - * 描述:Hadoop hdfs文件系统namenode节点地址。
- - * 必选:是
- - * 默认值:无
+ + * 描述:Hadoop hdfs文件系统namenode节点地址。
+ + * 必选:是
+ + * 默认值:无
* **fileType** - - * 描述:文件的类型,目前只支持用户配置为"text"、"orc"
- - text表示textfile文件格式 - - orc表示orcfile文件格式 - - * 必选:是
- - * 默认值:无
- + + * 描述:文件的类型,目前只支持用户配置为"text"、"orc"、“parquet” + + * text:textfile文件格式 + + * orc:orcfile文件格式 + + * parquet:parquet文件格式 + + * 必选:是
+ + * 默认值:无
* **column** - - * 描述:读取字段列表,type指定源数据的类型, - - ```json -{ - "type": "long", - "index": 0 //从本地文件文本第一列获取int字段 -}, -{ - "type": "string", - "value": "yesyoucan" //HdfsReader内部生成yesyoucan的字符串字段作为当前字段 -} - ``` - - 对于用户指定Column信息,type必须填写,index/value必须选择其一。 - - * 必选:是
- - * 默认值:全部按照string类型读取
+ + * 描述:需要读取的字段。 + + * 格式:支持3中格式 + + 1.读取全部字段,如果字段数量很多,可以使用下面的写法: + + ``` + "column":[*] + ``` + + 2.只指定字段名称: + + ``` + "column":["id","name"] + ``` + + 3.指定具体信息: + + ``` + "column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" + }] + ``` + + * 属性说明: + + * name:字段名称 + + * index:字段索引,当读取text格式的文件时指定此属性 + + * type:字段类型,可以和数据文件里的字段类型不一样,程序会做一次类型转换 + + * format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + + * value:如果数据文件里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 * **fieldDelimiter** - - * 描述:读取的字段分隔符
- - **另外需要注意的是,HdfsReader在读取textfile数据时,需要指定字段分割符,HdfsReader在读取orcfile时,用户无需指定字段分割符** - - * 必选:否
- - * 默认值:\\001
- + + * 描述:读取的字段分隔符
+ + * 注意:在读取text格式文件时需要指定此参数 + + * 必选:否
+ + * 默认值:“\001”
* **encoding** + + * 描述:读取文件的编码配置。 + * 必选:否 + * 默认值:utf-8 - * 描述:读取文件的编码配置。
- - * 必选:否
- - * 默认值:utf-8
- * **hadoopConfig** - - * 描述:hadoopConfig里可以配置与Hadoop相关的一些高级参数,比如HA的配置。
- - ```json - "hadoopConfig":{ - "dfs.nameservices": "testDfs", - "dfs.ha.namenodes.testDfs": "namenode1,namenode2", -        "dfs.namenode.rpc-address.aliDfs.namenode1": "", - "dfs.namenode.rpc-address.aliDfs.namenode2": "", - "dfs.client.failover.proxy.provider.testDfs": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" - } - ``` - - * 必选:否
- - * 默认值:无
- - + + * 描述:hadoopConfig里可以配置与Hadoop相关的一些高级参数,比如HA的配置。
+ + ``` + "dfs.nameservices": "testDfs", + "dfs.ha.namenodes.testDfs": "namenode1,namenode2", + "dfs.namenode.rpc-address.aliDfs.namenode1": "", + "dfs.namenode.rpc-address.aliDfs.namenode2": "", + "dfs.client.failover.proxy.provider.testDfs": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" + } + ``` + + * 必选:否
+ + * 默认值:无 diff --git a/docs/hdfswriter.md b/docs/hdfswriter.md index d28030fe09..b33ca831d2 100644 --- a/docs/hdfswriter.md +++ b/docs/hdfswriter.md @@ -4,169 +4,120 @@ ``` { - "job": { - "setting": { - "speed": { - "channel": 1 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { - "parameter": { - "password": "abc123", - "columnTypes": [ - "java.lang.Integer", - "java.lang.String" - ], - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?charset=utf8" - ], - "table": [ - "tb2" - ] - } - ], - "splitPk": "col1", - "username": "dtstack" - }, - "name": "mysqlreader" - }, - "writer": { - "name": "hdfswriter", - "parameter": { - "hadoopConfig": { - "dfs.nameservices":"ns1", - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" - }, - "defaultFS": "hdfs://ns1", - "fileType": "text", - "fileName": "hallo", - "column": [ - { - "name": "col1", - "type": "STRING" - }, - { - "name": "col2", - "type": "STRING" - } - ], - "path": "/hyf", - "writeMode": "append", - "fieldDelimiter": "\\001" - } - } - } - ] - } + "job": { + "setting": {}, + "content": [{ + "reader": {}, + "writer": { + "name": "hdfswriter", + "parameter": { + "hadoopConfig": { + "dfs.nameservices": "ns1", + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", + "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" + }, + "defaultFS": "hdfs://ns1", + "fileType": "text", + "fileName": "hello", + "column": [{ + "name": "col1", + "index": 0, + "type": "STRING" + }], + "rowGroupSize": 134217728, + "compress": "SNAPPY", + "path": "/test", + "writeMode": "append", + "fieldDelimiter": "\\001" + } + } + }] + } } ``` ## 2. 参数说明 * **defaultFS** - - * 描述:Hadoop hdfs文件系统namenode节点地址。格式:hdfs://ip:端口;例如:hdfs://127.0.0.1:9000
- - * 必选:是
- - * 默认值:无
+ + * 描述:Hadoop hdfs文件系统namenode节点地址。格式:hdfs://ip:端口;例如:hdfs://127.0.0.1:9000
+ + * 必选:是
+ + * 默认值:无
* **fileType** + + * 描述:文件的类型,目前只支持用户配置为"text"、"orc"、“parquet” + + * text:textfile文件格式 + + * orc:orcfile文件格式 + + * parquet:parquet文件格式 + + * 必选:是
+ + * 默认值:无
- * 描述:文件的类型,目前只支持用户配置为"text"或"orc"。
- - text表示textfile文件格式 - - orc表示orcfile文件格式 - - * 必选:是
- - * 默认值:无
- * **path** + + * 描述:存储到Hadoop hdfs文件系统的路径信息,HdfsWriter会根据并发配置在Path目录下写入多个文件。 + + * 必选:是
+ + * 默认值:无
+ +* **rowGroupSize** + + * 描述:写入parquet格式文件时指定,表示一个group的大小,如果字段数量很多,并且任务可使用内存有限,使用默认值可能会导致内存溢出,可以通过降低此参数的值来避免内存溢出,如果值很小,则会生产很多小的group,此时通过hive或者spark处理的话会降低效率,因此这个参数的调整要结合具体使用场景。 + + * 必选:否 + + * 默认值:134217728 - * 描述:存储到Hadoop hdfs文件系统的路径信息,HdfsWriter会根据并发配置在Path目录下写入多个文件。为与hive表关联,请填写hive表在hdfs上的存储路径。例:Hive上设置的数据仓库的存储路径为:/user/hive/warehouse/ ,已建立数据库:test,表:hello;则对应的存储路径为:/user/hive/warehouse/test.db/hello
- - * 必选:是
- - * 默认值:无
- -* **fileName** - - * 描述:HdfsWriter写入时的文件名
- - * 必选:是
- - * 默认值:无
- * **column** + + * 描述:写入数据的字段。 + + * name:指定字段名 + + * type:指定字段类型。 + + * 必选:是
+ + * 默认值:无
- * 描述:写入数据的字段,不支持对部分列写入。为与hive中表关联,需要指定表中所有字段名和字段类型,其中:name指定字段名,type指定字段类型。
- - ```json - "column": - [ - { - "name": "userName", - "type": "string" - }, - { - "name": "age", - "type": "long" - } - ] - ``` - - * 必选:是
- - * 默认值:无
- * **writeMode** - - * 描述:hdfswriter写入前数据清理处理模式:
- - * append,追加 - * overwrite,覆盖 - - * 必选:是
- - * 默认值:无
+ + * 描述:hdfswriter写入前数据清理处理模式:
+ * append,追加 + + * overwrite,覆盖 + * 注意:overwrite模式时会删除写入路径下的所有文件 + * 必选:否 + * 默认值:overwrite * **fieldDelimiter** - - * 描述:hdfswriter写入时的字段分隔符,**需要用户保证与创建的Hive表的字段分隔符一致,否则无法在Hive表中查到数据**
- - * 必选:是
- - * 默认值:\\001
+ + * 描述:hdfswriter写入时的字段分隔符,**需要用户保证与创建的Hive表的字段分隔符一致,否则无法在Hive表中查到数据**
+ + * 必选:是
+ + * 默认值:\\001
* **compress** - - * 描述:hdfs文件压缩类型,默认不填写意味着没有压缩。其中:text类型文件支持压缩类型有gzip、bzip2;orc类型文件支持的压缩类型有NONE、SNAPPY(需要用户安装SnappyCodec)。
- - * 必选:否
- - * 默认值:无压缩
+ + * 描述:hdfs文件压缩类型,默认不填写意味着没有压缩。其中:text类型文件支持压缩类型有gzip、bzip2;orc类型文件支持的压缩类型有NONE、SNAPPY(需要用户安装SnappyCodec)。
+ + * 必选:否
+ + * 默认值:无压缩
* **encoding** - - * 描述:写文件的编码配置。
- - * 必选:否
- - * 默认值:utf-8,**慎重修改**
+ + * 描述:写文件的编码配置。
+ * 必选:否 + * 默认值:utf-8 diff --git a/docs/mongodbreader.md b/docs/mongodbreader.md index 856a02f7f3..72f74cac66 100644 --- a/docs/mongodbreader.md +++ b/docs/mongodbreader.md @@ -1,103 +1,144 @@ # MongoDB读取插件(mongodbreader) ## 1. 配置样例 + ```json { - "job":{ - "content":[{ - "reader":{ - "parameter":{ - "hostPorts":"localhost:27017", - "username": "", - "password": "", - "database":"", - "collectionName": "", - "column": [ - { - "name":"id", - "type":"int", - "splitter":"," - }, - { - "name":"id", - "type":"string" - } - ], - "filter": "" - }, - "name":"mongodbreader" - }, - "writer":{} - }] - } + "job":{ + "content":[{ + "reader":{ + "parameter":{ + "hostPorts":"localhost:27017", + "username": "", + "password": "", + "database":"", + "collectionName": "", + "fetchSize":100, + "column": [ + { + "name":"id", + "type":"int", + "splitter":"," + } + ], + "filter": "" + }, + "name":"mongodbreader" + }, + "writer":{} + }] + } } ``` ## 2. 参数说明 * **name** - - * 描述:插件名,此处只能填mongodbreader,否则Flinkx将无法正常加载该插件包。 - - * 必选:是 - - * 默认值:无 + + * 描述:插件名,此处只能填mongodbreader,否则Flinkx将无法正常加载该插件包。 + + * 必选:是 + + * 默认值:无 * **hostPorts** - - * 描述:MongoDB的地址和端口,格式为 IP1:port,可填写多个地址,以英文逗号分隔。 - - * 必选:是 - - * 默认值:无 + + * 描述:MongoDB的地址和端口,格式为 IP1:port,可填写多个地址,以英文逗号分隔。 + + * 必选:是 + + * 默认值:无 * **username** - - * 描述:数据源的用户名 - - * 必选:否 - - * 默认值:无 + + * 描述:数据源的用户名 + + * 必选:否 + + * 默认值:无 * **password** + + * 描述:数据源指定用户名的密码 + + * 必选:否 + + * 默认值:无 - * 描述:数据源指定用户名的密码 - - * 必选:否 - - * 默认值:无 - * **database** + + * 描述:数据库名称 + + * 必选:是 + + * 默认值:无 - * 描述:数据库名称 - - * 必选:是 - - * 默认值:无 - * **collectionName** - - * 描述:集合名称 - - * 必选:是 - - * 默认值:无 + + * 描述:集合名称 + + * 必选:是 + + * 默认值:无 * **column** + + * 描述:需要读取的字段。 + + * 格式:支持3中格式 + + 1.读取全部字段,如果字段数量很多,可以使用下面的写法: + + ``` + "column":[*] + ``` + + 2.只指定字段名称: + + ``` + "column":["id","name"] + ``` + + 3.指定具体信息: + + ``` + "column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value", + "splitter":"," + }] + ``` + + * 属性说明: + + * name:字段名称 + + * type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + + * format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + + * value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + + * splitter:因为 MongoDB 支持数组类型,所以 MongoDB 读出来的数组类型要通过这个分隔符合并成字符串 + + * 必选:是 + + * 默认值:无 + +* **fetchSize** + + * 描述:每次读取的数据条数,通过调整此参数来优化读取速率 + + * 必选:否 + + * 默认值:100 - * 描述:MongoDB 的文档列名,配置为数组形式表示 MongoDB 的多个列。 - - name:Column 的名字。 - - type:Column 的类型。 - - splitter:因为 MongoDB 支持数组类型,所以 MongoDB 读出来的数组类型要通过这个分隔符合并成字符串。 - - * 必选:是 - - * 默认值:无 - * **filter** - - * 描述:过滤条件,通过该配置型来限制返回 MongoDB 数据范围,语法请参考[MongoDB查询语法](https://docs.mongodb.com/manual/crud/#read-operations) - - * 必选:否 - - * 默认值:无 \ No newline at end of file + + * 描述:过滤条件,通过该配置型来限制返回 MongoDB 数据范围,语法请参考[MongoDB查询语法](https://docs.mongodb.com/manual/crud/#read-operations) + + * 必选:否 + + * 默认值:无 diff --git a/docs/mongodbwriter.md b/docs/mongodbwriter.md index 80cdfbd886..838b34e412 100644 --- a/docs/mongodbwriter.md +++ b/docs/mongodbwriter.md @@ -1,124 +1,126 @@ # MongoDB写入插件(mongodbwriter) ## 1. 配置样例 + ```json { - "job":{ - "content":[{ - "reader":{}, - "writer":{ - "parameter":{ - "hostPorts":"localhost:27017", - "username": "", - "password": "", - "database":"test", - "collectionName": "test", - "writeMode": "insert", - "batchSize":1, - "column": [ - { - "name":"id", - "type":"int", - "splitter":"," - }, - { - "name":"id", - "type":"string", - "splitter":"," - } - ], - "replaceKey":"id" - }, - "name":"mongodbwriter" - } - }] - } + "job":{ + "content":[{ + "reader":{}, + "writer":{ + "parameter":{ + "hostPorts":"localhost:27017", + "username": "", + "password": "", + "database":"test", + "collectionName": "test", + "writeMode": "insert", + "batchSize":1, + "column": [ + { + "name":"id", + "type":"int", + "splitter":"," + }, + { + "name":"id", + "type":"string", + "splitter":"," + } + ], + "replaceKey":"id" + }, + "name":"mongodbwriter" + } + }] + } } ``` ## 2. 参数说明 * **name** - - * 描述:插件名,此处只能填 mongodbwriter,否则Flinkx将无法正常加载该插件包。 - - * 必选:是 - - * 默认值:无 + + * 描述:插件名,此处只能填 mongodbwriter,否则Flinkx将无法正常加载该插件包。 + + * 必选:是 + + * 默认值:无 * **hostPorts** - - * 描述:MongoDB的地址和端口,格式为 IP1:port,可填写多个地址,以英文逗号分隔。 - - * 必选:是 - - * 默认值:无 + + * 描述:MongoDB的地址和端口,格式为 IP1:port,可填写多个地址,以英文逗号分隔。 + + * 必选:是 + + * 默认值:无 * **username** - - * 描述:数据源的用户名 - - * 必选:否 - - * 默认值:无 + + * 描述:数据源的用户名 + + * 必选:否 + + * 默认值:无 * **password** + + * 描述:数据源指定用户名的密码 + + * 必选:否 + + * 默认值:无 - * 描述:数据源指定用户名的密码 - - * 必选:否 - - * 默认值:无 - * **database** + + * 描述:数据库名称 + + * 必选:是 + + * 默认值:无 - * 描述:数据库名称 - - * 必选:是 - - * 默认值:无 - * **collectionName** - - * 描述:集合名称 - - * 必选:是 - - * 默认值:无 + + * 描述:集合名称 + + * 必选:是 + + * 默认值:无 * **column** + + * 描述:MongoDB 的文档列名,配置为数组形式表示 MongoDB 的多个列。 + + - name:Column 的名字。 + - type:Column 的类型。 + - splitter:特殊分隔符,当且仅当要处理的字符串要用分隔符分隔为字符数组 Array 时,才使用这个参数。通过这个参数指定的分隔符,将字符串分隔存储到 MongoDB 的数组中。 + + * 必选:是 + + * 默认值:无 - * 描述:MongoDB 的文档列名,配置为数组形式表示 MongoDB 的多个列。 - - name:Column 的名字。 - - type:Column 的类型。 - - splitter:特殊分隔符,当且仅当要处理的字符串要用分隔符分隔为字符数组 Array 时,才使用这个参数。通过这个参数指定的分隔符,将字符串分隔存储到 MongoDB 的数组中。 - - * 必选:是 - - * 默认值:无 - * **replaceKey** + + * 描述:replaceKey 指定了每行记录的业务主键,用来做覆盖时使用(不支持 replaceKey为多个键,一般是指Monogo中的主键)。 + + * 必选:否 + + * 默认值:无 - * 描述:replaceKey 指定了每行记录的业务主键,用来做覆盖时使用(不支持 replaceKey为多个键,一般是指Monogo中的主键)。 - - * 必选:否 - - * 默认值:无 - * **writeMode** - - * 描述:写入模式,当 batchSize > 1 时不支持 replace 和 update 模式 - - * 必选:是 - - * 所有选项:insert/replace/update - - * 默认值:insert + + * 描述:写入模式,当 batchSize > 1 时不支持 replace 和 update 模式 + + * 必选:是 + + * 所有选项:insert/replace/update + + * 默认值:insert * **batchSize** - - * 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与MongoDB的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况。
- - * 必选:否 - - * 默认值:1 \ No newline at end of file + + * 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与MongoDB的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况。
+ + * 必选:否 + + * 默认值:1 diff --git a/docs/mysqldreader.md b/docs/mysqldreader.md deleted file mode 100644 index 4813fc02dc..0000000000 --- a/docs/mysqldreader.md +++ /dev/null @@ -1,145 +0,0 @@ -# MySQL分库分表读取插件(mysqldreader) - -## 1. 配置样例 - -``` -{ - "job": { - "setting": { - "speed": { - "channel": 4 - }, - "errorLimit": { - "record": 0, - "percentage": 10 - } - }, - "content": [ - { - "reader": { - "parameter": { - "password": "abc123", - "username": "dtstack", - "column": [ - "col1", - "col2" - ], - "where": "id > 1", - "connection": [ - { - "password": "abc123", - "username": "dtstack", - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?useUnicode=true&characterEncoding=utf8" - ], - "table": [ - "tb2" - ] - } - ], - "splitPk": "col1" - }, - "name": "mysqldreader" - }, - "writer": {} - } - ] - } -} - -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填mysqldreader,否则Flinkx将无法正常加载该插件包。 - - * 必选:是
- - * 默认值:无
- -* **connection** - - * 描述:需要读取的数据源数组。 - - * 必选:是 - - * 默认值:无 - - * 元素: - - * username:具体数据源的用户名,如果不填则使用全局的用户名。 - - * password:具体数据源的密码,如果不填则使用全局的密码。 - - * jdbcUrl:数据源连接url,只支持写单个连接。 - - * table:要查询的表名称,可写多张表,多张表的表结构必须一致。 - -* **jdbcUrl** - - * 描述:针对mysql数据库的jdbc连接字符串 - - jdbcUrl按照Mysql官方规范,并可以填写连接附件控制信息。具体请参看[Mysql官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html)。 - - * 必选:是
- - * 默认值:无
- -* **username** - - * 描述:全局数据源的用户名
- - * 必选:否
- - * 默认值:无
- -* **password** - - * 描述:全局数据源的密码
- - * 必选:否
- - * 默认值:无
- -* **where** - - * 描述:筛选条件,MysqldReader根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > $bizdate 。注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。
- - where条件可以有效地进行业务增量同步。如果不填写where语句,包括不提供where的key或者value,FlinkX均视作同步全量数据。 - - * 必选:否
- - * 默认值:无
- -* **splitPk** - - * 描述:MysqldReader进行数据抽取时,如果指定splitPk,表示用户希望使用splitPk代表的字段进行数据分片,FlinkX因此会启动并发任务进行数据同步,这样可以大大提供数据同步的效能。 - - 推荐splitPk用户使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 - -  目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,MysqldReader将报错! - -  如果splitPk不填写,包括不提供splitPk或者splitPk值为空,FlinkX视作使用单通道同步该表数据。 - - * 必选:否
- - * 默认值:空
- - - -* **column** - - * 描述:所配置的表中需要同步的列名集合。 - - 支持列裁剪,即列可以挑选部分列进行导出。 - - 支持列换序,即列可以不按照表schema信息进行导出。 - - 暂不支持常量列。 - - * 必选:是
- - * 默认值:无
- diff --git a/docs/mysqlreader.md b/docs/mysqlreader.md deleted file mode 100644 index 52e7c97420..0000000000 --- a/docs/mysqlreader.md +++ /dev/null @@ -1,151 +0,0 @@ -# MySQL读取插件(mysqlreader) - -## 1. 配置样例 - -``` -{ - "job": { - "setting": { - "speed": { - "channel": 3, - "bytes": 0 - }, - "errorLimit": { - "record": 10000, - "percentage": 100 - }, - "dirty": { - "path": "/tmp", - "hadoopConfig": { - - } - } - }, - "content": [ - { - "reader": { - "parameter": { - "password": "abc123" - "column": [ - "col1", - "col2" - ], - "where": "id > 1", - "connection": [ - { - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?useUnicode=true&characterEncoding=utf8" - ], - "table": [ - "tb2" - ] - } - ], - "splitPk": "col1", - "username": "dtstack" - }, - "name": "mysqlreader" - }, - "writer": { - "name": "sqlserverwriter", - "parameter": { - "batchSize": 2048, - "username": "sa", - "password": "Dtstack201610!", - "column": [ - "id", - "v" - ], - "writeMode": "replace", - "connection": [ - { - "jdbcUrl": "jdbc:jtds:sqlserver://172.16.10.46:1433;DatabaseName=dq", - "table": [ - "tb1" - ] - } - ] - } - } - } - ] - } -} - -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填mysqlreader,否则Flinkx将无法正常加载该插件包。 - * 必选:是
- - * 默认值:无
- -* **jdbcUrl** - - * 描述:针对mysql数据库的jdbc连接字符串 - - jdbcUrl按照Mysql官方规范,并可以填写连接附件控制信息。具体请参看[Mysql官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html)。 - - * 必选:是
- - * 默认值:无
- -* **username** - - * 描述:数据源的用户名
- - * 必选:是
- - * 默认值:无
- -* **password** - - * 描述:数据源指定用户名的密码
- - * 必选:是
- - * 默认值:无
- -* **where** - - * 描述:筛选条件,MysqlReader根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > $bizdate 。注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。
- - where条件可以有效地进行业务增量同步。如果不填写where语句,包括不提供where的key或者value,FlinkX均视作同步全量数据。 - - * 必选:否
- - * 默认值:无
- -* **splitPk** - - * 描述:MysqlReader进行数据抽取时,如果指定splitPk,表示用户希望使用splitPk代表的字段进行数据分片,FlinkX因此会启动并发任务进行数据同步,这样可以大大提供数据同步的效能。 - - 推荐splitPk用户使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 - -  目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,MysqlReader将报错! - -  如果splitPk不填写,包括不提供splitPk或者splitPk值为空,FlinkX视作使用单通道同步该表数据。 - - * 必选:否
- - * 默认值:空
- - - -* **column** - - * 描述:所配置的表中需要同步的列名集合。 - - 支持列裁剪,即列可以挑选部分列进行导出。 - - 支持列换序,即列可以不按照表schema信息进行导出。 - - 暂不支持常量列。 - - * 必选:是
- - * 默认值:无
- diff --git a/docs/mysqlwriter.md b/docs/mysqlwriter.md deleted file mode 100644 index 5c050e3b48..0000000000 --- a/docs/mysqlwriter.md +++ /dev/null @@ -1,171 +0,0 @@ -# MySQL写入插件(mysqlwriter) - -## 1. 配置样例 - -``` -{ - "job": { - "setting": { - "speed": { - "channel": 3, - "bytes": 0 - }, - "errorLimit": { - "record": 10000, - "percentage": 100 - }, - "dirty": { - "path": "/tmp", - "hadoopConfig": { - "fs.default.name": "hdfs://ns1", - "dfs.nameservices": "ns1", - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.ha.automatic-failover.enabled": "true", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "fs.hdfs.impl.disable.cache": "true" - } - } - }, - "content": [ - { - "reader": { - "name": "mysqlreader", - "parameter": { - "username": "dtstack", - "password": "abc123", - "column": [ - "id", - "v1" - ], - "where": "id > 1", - "connection": [ - { - "table": [ - "sb9" - ], - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true" - ] - } - ], - "splitPk": "id" - } - }, - "writer": { - "name": "mysqlwriter", - "parameter": { - "writeMode": "insert", - "username": "dtstack", - "password": "abc123", - "column": [ - "c1", - "c2" - ], - "batchSize": 1, - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true", - "table": [ - "tb3" - ] - } - ] - } - } - } - ] - } -} - -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填mysqlwriter,否则Flinkx将无法正常加载该插件包。 - * 必选:是
- - * 默认值:无
- -* **jdbcUrl** - - * 描述:针对mysql数据库的jdbc连接字符串 - - jdbcUrl按照Mysql官方规范,并可以填写连接附件控制信息。具体请参看[Mysql官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html)。 - - * 必选:是
- - * 默认值:无
- -* **username** - - * 描述:数据源的用户名
- - * 必选:是
- - * 默认值:无
- -* **password** - - * 描述:数据源指定用户名的密码
- - * 必选:是
- - * 默认值:无
- -* **column** - - * 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 - - * 必选:是
- - * 默认值:否
- - * 默认值:无
- -* **preSql** - - * 描述:写入数据到目的表前,会先执行这里的一组标准语句。 - - * 必选:否
- - * 默认值:无
- -* **postSql** - - * 描述:写入数据到目的表后,会执行这里的一组标准语句。 - - * 必选:否
- - * 默认值:无
- -* **table** - - * 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表。 - - 注意:table 和 jdbcUrl 必须包含在 connection 配置单元中 - - * 必选:是
- - * 默认值:无
- -* **writeMode** - - * 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句
- - * 必选:是
- - * 所有选项:insert/replace/update
- - * 默认值:insert
- -* **batchSize** - - * 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与Mysql的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况。
- - * 必选:否
- - * 默认值:1024
\ No newline at end of file diff --git a/docs/odpsreader.md b/docs/odpsreader.md index b92d5ebf98..6f0d838a6d 100644 --- a/docs/odpsreader.md +++ b/docs/odpsreader.md @@ -4,131 +4,111 @@ ``` { - "job": { - "setting": { - "speed": { - "channel": 3, - "bytes": 10000000 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "writer": { - "name": "mysqlwriter", - "parameter": { - "writeMode": "insert", - "username": "dtstack", - "password": "abc123", - "column": [ - "c1", - "c2" - ], - "batchSize": 1, - "session": [ - "set session sql_mode='ANSI'" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true", - "table": [ - "tb3" - ] - } - ] - } - }, - "reader": { - "name": "odpsreader", - "parameter": { - "odpsConfig": { - "accessId": "${odps.accessId}", - "accessKey": "${odps.accessKey}", - "project": "${odps.project}" - }, - "table": "tb252", - "partition": "pt='xxooxx'", - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - } - ] - } - } - } - ] - } + "job": { + "setting": {}, + "content": [{ + "writer": {}, + "reader": { + "name": "odpsreader", + "parameter": { + "odpsConfig": { + "accessId": "${odps.accessId}", + "accessKey": "${odps.accessKey}", + "project": "${odps.project}" + }, + "table": "tableTest", + "partition": "pt='xxooxx'", + "column": [{ + "name": "col1", + "type": "string", + “value”:"xx", + "format":"yyyy-MM-dd HH:mm:ss" + + }] + } + } + }] + } } - ``` ## 2. 参数说明 - * **accessId** - * 描述:ODPS系统登录ID
- - * 必选:是
- - * 默认值:无
+ + * 描述:ODPS系统登录ID
+ * 必选:是 + * 默认值:无 * **accessKey** - * 描述:ODPS系统登录Key
- - * 必选:是
- - * 默认值:无
+ + * 描述:ODPS系统登录Key
+ * 必选:是 + * 默认值:无 * **project** - - * 描述:读取数据表所在的 ODPS 项目名称(大小写不敏感)
- - * 必选:是
- - * 默认值:无
+ + * 描述:读取数据表所在的 ODPS 项目名称(大小写不敏感)
+ + * 必选:是
+ + * 默认值:无 * **table** + + * 描述:读取数据表的表名称(大小写不敏感)
+ + * 必选:是
+ + * 默认值:无
- * 描述:读取数据表的表名称(大小写不敏感)
- - * 必选:是
- - * 默认值:无
- * **partition** + + * 描述:读取数据所在的分区信息,支持linux shell通配符,包括 * 表示0个或多个字符,?代表任意一个字符。例如现在有分区表 test,其存在 pt=1,ds=hangzhou pt=1,ds=shanghai pt=2,ds=hangzhou pt=2,ds=beijing 四个分区,如果你想读取 pt=1,ds=shanghai 这个分区的数据,那么你应该配置为: `"partition":["pt=1,ds=shanghai"]`; 如果你想读取 pt=1下的所有分区,那么你应该配置为: `"partition":["pt=1,ds=* "]`;如果你想读取整个 test 表的所有分区的数据,那么你应该配置为: `"partition":["pt=*,ds=*"]`
+ + * 必选:如果表为分区表,则必填。如果表为非分区表,则不能填写
+ + * 默认值:无
- * 描述:读取数据所在的分区信息,支持linux shell通配符,包括 * 表示0个或多个字符,?代表任意一个字符。例如现在有分区表 test,其存在 pt=1,ds=hangzhou pt=1,ds=shanghai pt=2,ds=hangzhou pt=2,ds=beijing 四个分区,如果你想读取 pt=1,ds=shanghai 这个分区的数据,那么你应该配置为: `"partition":["pt=1,ds=shanghai"]`; 如果你想读取 pt=1下的所有分区,那么你应该配置为: `"partition":["pt=1,ds=* "]`;如果你想读取整个 test 表的所有分区的数据,那么你应该配置为: `"partition":["pt=*,ds=*"]`
- - * 必选:如果表为分区表,则必填。如果表为非分区表,则不能填写
- - * 默认值:无
- * **column** - - * 描述:读取 odps 源头表的列信息,包括需要选取的列,每列的格式如下: - * 根据字段名指定列 - - ``` -{ - "name": 'col1' //获取字段名为col1的字段 -} - ``` - - * 指定常量列 - - ``` -{ - "type": "string", - "value": "yesyoucan" //OdpsReader内部生成yesyoucan的字符串字段作为当前字段 -} - ``` - - + + * 描述:需要读取的字段。 + + * 格式:支持3中格式 + + 1.读取全部字段,如果字段数量很多,可以使用下面的写法: + + ``` + "column":[*] + ``` + + 2.只指定字段名称: + + ``` + "column":["id","name"] + ``` + + 3.指定具体信息: + + ``` + "column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" + }] + ``` + + * 属性说明: + + * name:字段名称 + + * type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + + * format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + + * value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + + * 必选:是 + + * 默认值:无 diff --git a/docs/odpswriter.md b/docs/odpswriter.md index 3d3faa7c11..baddb0f64e 100644 --- a/docs/odpswriter.md +++ b/docs/odpswriter.md @@ -4,113 +4,89 @@ ``` { - "job": { - "setting": { - "speed": { - "channel": 3, - "bytes": 10000000 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { - "name": "mysqlreader", - "parameter": { - "username": "dtstack", - "password": "abc123", - "column": [ - "col1", - "col2" - ], - // "splitPk": "col1", - "connection": [ - { - "table": [ - "tb2" - ], - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true" - ] + "job": { + "setting": {}, + "content": [{ + "reader": {}, + "writer": { + "name": "odpswriter", + "parameter": { + "odpsConfig": { + "accessId": "${odps.accessId}", + "accessKey": "${odps.accessKey}", + "project": "${odps.project}" + }, + "table": "tableTest", + "partition": "pt='xx'", + "writeMode": "append", + "bufferSize": 64, + "column": [{ + "name": "col1", + "type": "string" + }] } - ] - } - }, - "writer": { - "name": "odpswriter", - "parameter": { - "odpsConfig": { - "accessId": "${odps.accessId}", - "accessKey": "${odps.accessKey}", - "project": "${odps.project}" - }, - "table": "tb252", - "partition": "pt='xxooxx'", - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - } - ] - } - } - } - ] - } + } + }] + } } ``` ## 2. 参数说明 * **accessId** - * 描述:ODPS系统登录ID
- - * 必选:是
- - * 默认值:无
+ + * 描述:ODPS系统登录ID
+ * 必选:是 + * 默认值:无 * **accessKey** - * 描述:ODPS系统登录Key
- - * 必选:是
- - * 默认值:无
+ + * 描述:ODPS系统登录Key
+ * 必选:是 + * 默认值:无 * **project** - - * 描述:写入数据表所在的 ODPS 项目名称(大小写不敏感)
- - * 必选:是
- - * 默认值:无
+ + * 描述:读取数据表所在的 ODPS 项目名称(大小写不敏感)
+ + * 必选:是
+ + * 默认值:无 * **table** + + * 描述:读取数据表的表名称(大小写不敏感)
+ + * 必选:是
+ + * 默认值:无
- * 描述:写入数据表的表名称(大小写不敏感)
- - * 必选:是
- - * 默认值:无
- * **partition** + + * 描述:需要写入数据表的分区信息,必须指定到最后一级分区。把数据写入一个三级分区表,必须配置到最后一级分区,例如pt=20150101/type=1/biz=2。 + + * 必选:**如果是分区表,该选项必填,如果非分区表,该选项不可填写。** + + * 默认值:空
- * 描述:需要写入数据表的分区信息,必须指定到最后一级分区。把数据写入一个三级分区表,必须配置到最后一级分区,例如pt=20150101/type=1/biz=2。 -
- * 必选:**如果是分区表,该选项必填,如果非分区表,该选项不可填写。** - * 默认值:空
- * **column** - - * 描述:需要导入的字段列表,当导入全部字段时,可以配置为"column": ["*"], 当需要插入部分odps列填写部分列,例如"column": ["id", "name"]。ODPSWriter支持列筛选、列换序,例如表有a,b,c三个字段,用户只同步c,b两个字段。可以配置成["c","b"], 在导入过程中,字段a自动补空,设置为null。
- * 必选:否
- * 默认值:无
- - - + + * 描述:需要导入的字段列表,当导入全部字段时,可以配置为"column": ["*"], 当需要插入部分odps列填写部分列,例如"column": ["id", "name"]。ODPSWriter支持列筛选、列换序,例如表有a,b,c三个字段,用户只同步c,b两个字段。可以配置成["c","b"], 在导入过程中,字段a自动补空,设置为null。
+ * 必选:否
+ * 默认值:无
+ +* **writeMode** + + * 描述:写入模式,支持append和overwrite + + * 必填:否 + + * 默认值:append + +* **bufferSize** + + * 描述:写入缓存大小,单位兆,odps写入数据时会先缓存,达到一定值后才会写入数据,如果写入数据时出现内存溢出,可以降低此参数的值。 + + * 必填:否 + + * 默认值:64 diff --git a/docs/oraclereader.md b/docs/oraclereader.md deleted file mode 100644 index 870d9f8502..0000000000 --- a/docs/oraclereader.md +++ /dev/null @@ -1,143 +0,0 @@ -# Oracle读取插件(oraclereader) - -## 1. 配置样例 - -``` -{"job": { - "content": [ - { - "reader": { - "parameter": { - "password": "wujing", - "where": "3 > 1", - "column": [ - "ID1", - "C1", - "C2" - ], - "connection": [ - { - "jdbcUrl": [ - "jdbc:oracle:thin:@//172.16.8.121:1521/dtstack" - ], - "table": [ - "SB1" - ] - } - ], - "splitPk": "ID1", - "username": "wujing" - }, - "name": "oraclereader" - }, - "writer": { - "parameter": { - "password": "wujing", - "column": [ - "ID", - "C1", - "C2" - ], - "connection": [ - { - "jdbcUrl": "jdbc:oracle:thin:@//172.16.8.121:1521/dtstack", - "table": [ - "SB2" - ] - } - ], - "writeMode": "replace", - "username": "wujing" - }, - "name": "oraclewriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 100 - }, - "speed": { - "bytes": 1048576, - "channel": 2 - } - } - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填oraclereader,否则Flinkx将无法正常加载该插件包。 - * 必选:是
- - * 默认值:无
- -* **jdbcUrl** - - * 描述:针对oracle数据库的jdbc连接字符串 - - jdbcUrl按照Oracle官方规范,并可以填写连接附件控制信息。具体请参看[Oracle官方文档](http://www.oracle.com/technetwork/database/enterprise-edition/documentation/index.html)。 - - * 必选:是
- - * 默认值:无
- -* **username** - - * 描述:数据源的用户名
- - * 必选:是
- - * 默认值:无
- -* **password** - - * 描述:数据源指定用户名的密码
- - * 必选:是
- - * 默认值:无
- -* **where** - - * 描述:筛选条件,OracleReader根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > $bizdate 。注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。
- - where条件可以有效地进行业务增量同步。如果不填写where语句,包括不提供where的key或者value,FlinkX均视作同步全量数据。 - - * 必选:否
- - * 默认值:无
- -* **splitPk** - - * 描述:OracleReader进行数据抽取时,如果指定splitPk,表示用户希望使用splitPk代表的字段进行数据分片,FlinkX因此会启动并发任务进行数据同步,这样可以大大提供数据同步的效能。 - - 推荐splitPk用户使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 - -  目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,OracleReader将报错! - - 如果splitPk不填写,包括不提供splitPk或者splitPk值为空,FlinkX视作使用单通道同步该表数据。 - - * 必选:否
- - * 默认值:空
- - - -* **column** - - * 描述:所配置的表中需要同步的列名集合。 - - 支持列裁剪,即列可以挑选部分列进行导出。 - - 支持列换序,即列可以不按照表schema信息进行导出。 - - 暂不支持常量列。 - - * 必选:是
- - * 默认值:无
- diff --git a/docs/oraclewriter.md b/docs/oraclewriter.md deleted file mode 100644 index e9ac5fd64e..0000000000 --- a/docs/oraclewriter.md +++ /dev/null @@ -1,156 +0,0 @@ -# Oracle写入插件(oraclewriter) - -## 1. 配置样例 - -``` -{"job": { - "content": [ - { - "reader": { - "parameter": { - "password": "wujing", - "where": "3 > 1", - "column": [ - "ID1", - "C1", - "C2" - ], - "connection": [ - { - "jdbcUrl": [ - "jdbc:oracle:thin:@//172.16.8.121:1521/dtstack" - ], - "table": [ - "SB1" - ] - } - ], - "splitPk": "ID1", - "username": "wujing" - }, - "name": "oraclereader" - }, - "writer": { - "parameter": { - "password": "wujing", - "column": [ - "ID", - "C1", - "C2" - ], - "connection": [ - { - "jdbcUrl": "jdbc:oracle:thin:@//172.16.8.121:1521/dtstack", - "table": [ - "SB2" - ] - } - ], - "writeMode": "replace", - "username": "wujing" - }, - "name": "oraclewriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 100 - }, - "speed": { - "bytes": 1048576, - "channel": 2 - } - } - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填oraclewriter,否则Flinkx将无法正常加载该插件包。 - * 必选:是
- - * 默认值:无
- -* **jdbcUrl** - - * 描述:针对mysql数据库的jdbc连接字符串 - - jdbcUrl按照Oracle官方规范,并可以填写连接附件控制信息。具体请参看[Oracle官方文档](http://www.oracle.com/technetwork/database/enterprise-edition/documentation/index.html)。 - - * 必选:是
- - * 默认值:无
- -* **username** - - * 描述:数据源的用户名
- - * 必选:是
- - * 默认值:无
- -* **password** - - * 描述:数据源指定用户名的密码
- - * 必选:是
- - * 默认值:无
- -* **column** - - * 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 - - * 必选:是
- - * 默认值:否
- - * 默认值:无
- -* **preSql** - - * 描述:写入数据到目的表前,会先执行这里的一组标准语句。 - - * 必选:否
- - * 默认值:无
- -* **postSql** - - * 描述:写入数据到目的表后,会执行这里的一组标准语句。 - - * 必选:否
- - * 默认值:无
- -* **table** - - * 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表。 - - 注意:table 和 jdbcUrl 必须包含在 connection 配置单元中 - - * 必选:是
- - * 默认值:无
- -* **writeMode** - - * 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句
- ** 在oracle中, 用merge into模拟后两种插入语义。 ** - * 必选:是
- - * 所有选项:insert/replace/update
- - * 默认值:insert
- -* **batchSize** - - * 描述:一次性批量提交的记录数大小 - - * 必选:否
- - * 默认值:1024
\ No newline at end of file diff --git a/docs/postgresqlreader.md b/docs/postgresqlreader.md deleted file mode 100644 index 1870cbb673..0000000000 --- a/docs/postgresqlreader.md +++ /dev/null @@ -1,101 +0,0 @@ -# PostgreSQL读取插件(postgresqlreader) - -## 1. 配置样例 -```json -{ - "job":{ - "content":[{ - "reader":{ - "parameter":{ - "username": "postgres", - "password": "postgres", - "connection": [{ - "jdbcUrl": ["jdbc:postgresql://localhost:5432/postgres"], - "table": ["tableTest"] - }], - "splitPk": "id", - "column": ["id", "tenant_id"], - "where": "id > 0" - }, - "name":"postgresqlreader" - }, - "writer":{} - }] - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填postgresqlreader,否则Flinkx将无法正常加载该插件包。 - * 必选:是
- - * 默认值:无
- -* **jdbcUrl** - - * 描述:针对PostgreSQL数据库的jdbc连接字符串 - - jdbcUrl按照PostgreSQL官方规范,并可以填写连接附件控制信息。具体请参看[PostgreSQL官方文档](https://jdbc.postgresql.org/documentation/head/connect.html)。 - - * 必选:是
- - * 默认值:无
- -* **username** - - * 描述:数据源的用户名
- - * 必选:是
- - * 默认值:无
- -* **password** - - * 描述:数据源指定用户名的密码
- - * 必选:是
- - * 默认值:无
- -* **where** - - * 描述:筛选条件,PostgreSQLReader根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > $bizdate 。注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。
- - where条件可以有效地进行业务增量同步。如果不填写where语句,包括不提供where的key或者value,FlinkX均视作同步全量数据。 - - * 必选:否
- - * 默认值:无
- -* **splitPk** - - * 描述:PostgreSQLReader进行数据抽取时,如果指定splitPk,表示用户希望使用splitPk代表的字段进行数据分片,FlinkX因此会启动并发任务进行数据同步,这样可以大大提供数据同步的效能。 - - 推荐splitPk用户使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 - -  目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,PostgreSQLReader将报错! - -  如果splitPk不填写,包括不提供splitPk或者splitPk值为空,FlinkX视作使用单通道同步该表数据。 - - * 必选:否
- - * 默认值:空
- - - -* **column** - - * 描述:所配置的表中需要同步的列名集合。 - - 支持列裁剪,即列可以挑选部分列进行导出。 - - 支持列换序,即列可以不按照表schema信息进行导出。 - - 暂不支持常量列。 - - * 必选:是
- - * 默认值:无
\ No newline at end of file diff --git a/docs/postgresqlwriter.md b/docs/postgresqlwriter.md deleted file mode 100644 index d769abcaff..0000000000 --- a/docs/postgresqlwriter.md +++ /dev/null @@ -1,120 +0,0 @@ -# PostgreSQL写入插件(postgresqlwriter) - -## 1. 配置样例 -```json -{ - "job":{ - "content":[{ - "reader":{}, - "writer":{ - "parameter":{ - "postSql": [], - "password": "postgres", - "session": [], - "column": ["id", "data_name"], - "connection": [{ - "jdbcUrl": "jdbc:postgresql://localhost:5432/postgres", - "table": ["table1"] - }], - "writeMode": "insert", - "preSql": [], - "username": "postgres" - }, - "name":"postgresqlwriter" - } - }] - } -} -``` - - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填postgresqlwriter,否则Flinkx将无法正常加载该插件包。 - * 必选:是
- - * 默认值:无
- -* **jdbcUrl** - - * 描述:针对PostgreSQL数据库的jdbc连接字符串 - - jdbcUrl按照PostgreSQL官方规范,并可以填写连接附件控制信息。具体请参看[PostgreSQL官方文档](https://jdbc.postgresql.org/documentation/head/connect.html)。 - - * 必选:是
- - * 默认值:无
- -* **username** - - * 描述:数据源的用户名
- - * 必选:是
- - * 默认值:无
- -* **password** - - * 描述:数据源指定用户名的密码
- - * 必选:是
- - * 默认值:无
- -* **column** - - * 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 - - * 必选:是
- - * 默认值:否
- - * 默认值:无
- -* **preSql** - - * 描述:写入数据到目的表前,会先执行这里的一组标准语句。 - - * 必选:否
- - * 默认值:无
- -* **postSql** - - * 描述:写入数据到目的表后,会执行这里的一组标准语句。 - - * 必选:否
- - * 默认值:无
- -* **table** - - * 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表。 - - 注意:table 和 jdbcUrl 必须包含在 connection 配置单元中 - - * 必选:是
- - * 默认值:无
- -* **writeMode** - - * 描述:控制写入数据到目标表采用 `insert into` 或者 `insert into .... on conflict(id) do update set ..` 语句。
- - 注:PostgreSQL 9.5之前的版本不支持`insert into .... on conflict(id) do update set ..`语法,因此如果PostgreSQL的版本小于9.5,PostgreSQLWriter插件的update和replace模式将无法使用 - - * 必选:是
- - * 所有选项:insert/replace/update
- - * 默认值:insert
- -* **batchSize** - - * 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与PostgreSQL的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况。
- - * 必选:否
- - * 默认值:1024
\ No newline at end of file diff --git a/docs/rdbdreader.md b/docs/rdbdreader.md new file mode 100644 index 0000000000..364e9258dd --- /dev/null +++ b/docs/rdbdreader.md @@ -0,0 +1,158 @@ +# 分库分表读取插件(**dreader) + +## 1. 配置样例 + +``` +{ + "job": { + "setting": {}, + "content": [ + { + "reader": { + "parameter": { + "password": "abc123", + "username": "dtstack", + "column": [ + "col1", + "col2" + ], + "where": "id > 1", + "connection": [ + { + "password": "abc123", + "username": "dtstack", + "jdbcUrl": [ + "jdbc:mysql://127.0.0.1:3306/test?useUnicode=true&characterEncoding=utf8" + ], + "table": [ + "tb2" + ] + } + ], + "splitPk": "id" + }, + "name": "mysqldreader" + }, + "writer": {} + } + ] + } +} +``` + +## 2. 参数说明 + +* **name** + + * 描述:插件名,目前只支持mysql的分库分表读取,mysqldreader。 + + * 必选:是
+ + * 默认值:无
+ +* **connection** + + * 描述:需要读取的数据源数组。 + + * 必选:是 + + * 默认值:无 + + * 元素: + + * username:具体数据源的用户名,如果不填则使用全局的用户名。 + + * password:具体数据源的密码,如果不填则使用全局的密码。 + + * jdbcUrl:数据源连接url,只支持写单个连接。 + + * table:要查询的表名称,可写多张表,多张表的表结构必须一致。 + +* **jdbcUrl** + + * 描述:针对关系型数据库的jdbc连接字符串 + + * 必选:是
+ + * 默认值:无
+ +* **username** + + * 描述:全局数据源的用户名
+ + * 必选:否
+ + * 默认值:无
+ +* **password** + + * 描述:全局数据源的密码
+ + * 必选:否
+ + * 默认值:无
+ +* **where** + + * 描述:筛选条件,MysqldReader根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > $bizdate 。注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。where条件可以有效地进行业务增量同步。如果不填写where语句,包括不提供where的key或者value,FlinkX均视作同步全量数据 + + * 必选:否
+ + * 默认值:无
+ +* **splitPk** + + * 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + + * 注意: + + * 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + * 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,MysqlReader将报错! + * 如果channel大于1但是没有配置此参数,任务将置为失败。 + + * 必选:否 + + * 默认值:空 + +* **column** + + * 描述:需要读取的字段。 + + * 格式:支持3中格式 + + 1.读取全部字段,如果字段数量很多,可以使用下面的写法: + + ``` + "column":[*] + ``` + + 2.只指定字段名称: + + ``` + "column":["id","name"] + ``` + + 3.指定具体信息: + + ``` + "column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" + }] + ``` + + * 属性说明: + + * name:字段名称 + + * type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + + * format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + + * value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + + * 必选:是 + + * 默认值:无 diff --git a/docs/rdbreader.md b/docs/rdbreader.md new file mode 100644 index 0000000000..f30a661b58 --- /dev/null +++ b/docs/rdbreader.md @@ -0,0 +1,241 @@ +# 关系数据库读取插件(*reader) + +## 1. 配置样例 + +``` +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "username": "username", + "password": "password", + "connection": [{ + "jdbcUrl": [ + "jdbc:mysql://127.0.0.1:3306/test?useUnicode=true&characterEncoding=utf8" + ], + "table": [ + "tableTest" + ] + }], + "column": [{ + "name": "id", + "type": "int", + "values": 123 + },{ + "name":"", + "index":1, + "type":"", + "value":"", + "format":"" + }], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "select * from tableTest", + "requestAccumulatorInterval": 2, + "increColumn": "id", + "startLocation": null, + "useMaxFunc": true + }, + "name": "mysqlreader" + }, + "writer": { + + } + }] + }, + "setting": { + + } +} +``` + +## 2. 参数说明 + +* **name** + + * 描述:插件名,此处填写插件名称,当前支持的关系数据库插件包括:mysqlreader,oraclereader,sqlserverreader,postgresqlreader,db2reader。 + * 必选:是 + + * 默认值:无 + +* **jdbcUrl** + + * 描述:针对关系型数据库的jdbc连接字符串 + + jdbcUrl参考文档: + + - [Mysql官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) + + - [Oracle官方文档](http://www.oracle.com/technetwork/database/enterprise-edition/documentation/index.html) + + - [SqlServer官方文档](https://docs.microsoft.com/zh-cn/sql/connect/jdbc/overview-of-the-jdbc-driver?view=sql-server-2017) + + - [PostgreSql官方文档](https://jdbc.postgresql.org/documentation/head/connect.html) + + - [Db2官方文档](https://www.ibm.com/analytics/us/en/db2/) + + * 必选:是 + + * 默认值:无 + +* **username** + + * 描述:数据源的用户名 + + * 必选:是 + + * 默认值:无 + +* **password** + + * 描述:数据源指定用户名的密码 + + * 必选:是 + + * 默认值:无 + +* **where** + + * 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + + * 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + + * 必选:否 + + * 默认值:无 + +* **splitPk** + + * 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + + * 注意: + + * 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + * 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,MysqlReader将报错! + * 如果channel大于1但是没有配置此参数,任务将置为失败。 + + * 必选:否 + + * 默认值:空 + +* **fetchSize** + + * 描述:读取时每批次读取的数据条数。 + + * 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + + * 必选:否 + + * 默认值:mysql为0,表示流式读取,其它数据库为1000 + +* **queryTimeOut** + + * 描述:查询超时时间,单位秒。 + + * 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + + * 必选:否 + + * 默认值:1000s + +* **customSql** + + * 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + + * 注意: + + * 只能是查询语句,否则会导致任务失败; + + * 查询语句返回的字段需要和column列表里的字段对应; + + * 当指定了此参数时,connection里指定的table无效,但是在一些情况下依然必须指定,比如使用增量同步的时候; + + * 必选:否 + + * 默认值:null + +* **increColumn** + + * 描述:当需要增量同步时指定此参数,任务运行过程中会把此字段的值存储到flink的Accumulator里,如果配置了指标,名称为:endLocation,类型为string,日期类型会转为时间戳,精度最多到纳秒,数值类型的为字段的值,程序结束时由外部应用获取。 + + * 注意: + + * 指定的字段必须在column列表里存在,否则任务会失败; + + * 增量字段支持数值类型和日期类型,并且是升序的,推荐使用表主键; + + * 必选:否 + + * 默认值:无 + +* **startLocation** + + * 描述:此配置参数和increColumn参数配合使用,表示本次任务获取数据的开始位置。 + + * 注意: + + * 此参数为空时进行全量同步 + + * 必选:否 + + * 默认值:无 + +* **useMaxFunc** + + * 描述:进行增量同步任务时,如果指定的字段值存在重复值,比如字段类型为时间,精度到秒,就可能出现重复的时间,需要指定此字段为true,读取数据前会获取增量字段的最大值作为此次任务的结束位置,防止数据丢失。 + + * 注意: + + * 此参数设为true时,会执行select max(increCol) from tb语句,会影响数据库负载,配置时需要考虑数据库的使用情况; + + * 此参数设置为true时,本次任务不会读取 increCol = max(increCol)的记录,会在任务下次运行时读取; + + * 必选:否 + + * 默认:false + +* **column** + + * 描述:需要读取的字段。 + + * 格式:支持3中格式 + + 1.读取全部字段,如果字段数量很多,可以使用下面的写法: + + ``` + "column":[*] + ``` + + 2.只指定字段名称: + + ``` + "column":["id","name"] + ``` + + 3.指定具体信息: + + ``` + "column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" + }] + ``` + + * 属性说明: + + * name:字段名称 + + * type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + + * format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + + * value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + + * 必选:是 + + * 默认值:无 diff --git a/docs/rdbwriter.md b/docs/rdbwriter.md new file mode 100644 index 0000000000..092fccdd2a --- /dev/null +++ b/docs/rdbwriter.md @@ -0,0 +1,134 @@ +# MySQL写入插件(*writer) + +## 1. 配置样例 + +``` +{ + "job": { + "content": [{ + "reader": {}, + "writer": { + "name": "*writer", + + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:mysql://127.0.0.1:3306/test?useCursorFetch=true", + "table": [ + "tableTest" + ] + }], + "username": "username", + "password": "password", + "column": [], + + "writeMode": "insert", + "batchSize": 1024, + "preSql": "", + "postSql": "", + "updateKey": "" + } + } + }] + }, + "setting": {} +} +``` + +## 2. 参数说明 + +* **name** + + * 描述:插件名,此处可填写:mysqlwriter,oraclewriter,sqlserverwriter,postgresqlwriter,db2writer + * 必选:是 + + 默认值:无 + +* **jdbcUrl** + + * 描述:针对关系型数据库的jdbc连接字符串 + + * 必选:是 + + * 默认值:无 + +* **username** + + * 描述:数据源的用户名 + + * 必选:是 + + * 默认值:无 + +* **password** + + * 描述:数据源指定用户名的密码 + + * 必选:是 + + * 默认值:无 + +* **column** + + * 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 + + * 必选:是 + + * 默认值:否 + + * 默认值:无 + +* **preSql** + + * 描述:写入数据到目的表前,会先执行这里的一组标准语句。 + + * 必选:否 + + * 默认值:无 + +* **postSql** + + * 描述:写入数据到目的表后,会执行这里的一组标准语句。 + + * 必选:否 + + * 默认值:无 + +* **table** + + * 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表。 + + * 必选:是 + + * 默认值:无 + +* **writeMode** + + * 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句 + + * 必选:是 + + * 所有选项:insert/replace/update + + * 默认值:insert + +* **batchSize** + + * 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况。 + + * 必选:否 + + * 默认值:1024 + +* **updateKey** + + * 描述:当写入模式为update和replace时,需要指定此参数的值为唯一索引字段。 + + * 注意: + + * 如果此参数为空,并且写入模式为update和replace时,应用会自动获取数据库中的唯一索引; + + * 如果数据表没有唯一索引,但是写入模式配置为update和replace,应用会以insert的方式写入数据; + + * 必选:否 + + * 默认值:无 diff --git a/docs/rediswriter.md b/docs/rediswriter.md index e02a2b5264..366664ab63 100644 --- a/docs/rediswriter.md +++ b/docs/rediswriter.md @@ -1,147 +1,145 @@ # Redis写入插件(rediswriter) ## 1. 配置样例 + ```json { - "job":{ - "content":[{ - "reader":{}, - "writer":{ - "parameter":{ - "hostPort":"localhost:6379", - "password": "密码", - "database":1, - "keyIndexes": [0,2], - "writeMode":"", - "keyFieldDelimiter": "\u0001", - "expireTime": 1000, - "timeout": 10000, - "dateFormat": "yyyy-MM-dd HH:mm:ss", - "type": "string", - "mode": "set", - "valueFieldDelimiter": "\u0001" - }, - "name":"rediswriter" - } - }] - } + "job":{ + "content":[{ + "reader":{}, + "writer":{ + "parameter":{ + "hostPort":"localhost:6379", + "password": "密码", + "database":1, + "keyIndexes": [0,2], + "writeMode":"", + "keyFieldDelimiter": "\u0001", + "expireTime": 1000, + "timeout": 10000, + "dateFormat": "yyyy-MM-dd HH:mm:ss", + "type": "string", + "mode": "set", + "valueFieldDelimiter": "\u0001" + }, + "name":"rediswriter" + } + }] + } } ``` ## 2. 参数说明 * **name** + + * 描述:插件名,此处只能填rediswriter,否则Flinkx将无法正常加载该插件包。 + * 必选:是 + * 默认值:无 - * 描述:插件名,此处只能填rediswriter,否则Flinkx将无法正常加载该插件包。 - * 必选:是
- - * 默认值:无
- * **hostPort** + + * 描述:Redis的IP地址和端口 + + * 必选:是 + + * 默认值:localhost:6379 - * 描述:Redis的IP地址和端口 - - * 必选:是 - - * 默认值:localhost:6379 - * **password** + + * 描述:数据源指定用户名的密码 + + * 必选:是 + + * 默认值:无 - * 描述:数据源指定用户名的密码 - - * 必选:是 - - * 默认值:无 - * **database** + + * 描述:要写入Redis数据库 + + * 必选:否 + + * 默认值:0 - * 描述:要写入Redis数据库 - - * 必选:否 - - * 默认值:0 - * **keyIndexes** + + * 描述:keyIndexes 表示源端哪几列需要作为 key(第一列是从 0 开始)。如果是第一列和第二列需要组合作为 key,那么 keyIndexes 的值则为 [0,1]。 + + * 注意:配置 keyIndexes 后,Redis Writer 会将其余的列作为 value,如果您只想同步源表的某几列作为 key,某几列作为 value,不需要同步所有字段,那么在 Reader 插件端就指定好 column 作好列筛选即可。 + + * 必选:是 + + * 默认值:无 - * 描述:keyIndexes 表示源端哪几列需要作为 key(第一列是从 0 开始)。如果是第一列和第二列需要组合作为 key, - 那么 keyIndexes 的值则为 [0,1]。 - - 注意:配置 keyIndexes 后,Redis Writer 会将其余的列作为 value,如果您只想同步源表的某几列作为 key,某几列作为 value,不需要同步所有字段,那么在 Reader 插件端就指定好 column 作好列筛选即可。 - - * 必选:是 - - * 默认值:无 - * **keyFieldDelimiter** + + * 描述:写入 Redis 的 key 分隔符。比如: key=key1\u0001id,如果 key 有多个需要拼接时,该值为必填项,如果 key 只有一个则可以忽略该配置项。 + + * 必选:否 + + * 默认值:\u0001 - * 描述:写入 Redis 的 key 分隔符。比如: key=key1\u0001id,如果 key 有多个需要拼接时,该值为必填项,如果 key 只有一个则可以忽略该配置项。 - - * 必选:否 - - * 默认值:\u0001 - * **expireTime** + + * 描述:Redis value 值缓存失效时间(如果需要永久有效则可以不填该配置项)。 + + * 注意:如果过期时间的秒数大于 60*60*24*30(即 30 天),则服务端认为是 Unix 时间,该时间指定了到未来某个时刻数据失效。否则为相对当前时间的秒数,该时间指定了从现在开始多长时间后数据失效。 + + * 必选:否 + + * 默认值:0(0 表示永久有效) - * 描述:Redis value 值缓存失效时间(如果需要永久有效则可以不填该配置项)。 - - 注意:如果过期时间的秒数大于 60*60*24*30(即 30 天),则服务端认为是 Unix 时间,该时间指定了到未来某个时刻数据失效。否则为相对当前时间的秒数,该时间指定了从现在开始多长时间后数据失效。 - - * 必选:否 - - * 默认值:0(0 表示永久有效) - * **timeout** + + * 描述:写入 Redis 的超时时间。 + + * 单位:毫秒 + + * 必选:否 + + * 默认值:30000 - * 描述:写入 Redis 的超时时间。 - - * 单位:毫秒 - - * 必选:否 - - * 默认值:30000 - * **dateFormat** + + * 描述:写入 Redis 时,Date 的时间格式:”yyyy-MM-dd HH:mm:ss” + + * 必选:否 + + * 默认值:将日期以long类型写入 - * 描述:写入 Redis 时,Date 的时间格式:”yyyy-MM-dd HH:mm:ss” - - * 必选:否 - - * 默认值:将日期以long类型写入 - * **writeMode** - - * 描述:写入模式,由于 Redis 的数据结构为key-value模式,因此只要key相同,就会覆盖value值 - - * 必选:是 - - * 所有选项:insert - - * 默认值:insert - + + * 描述:写入模式,由于 Redis 的数据结构为key-value模式,因此只要key相同,就会覆盖value值 + + * 必选:是 + + * 所有选项:insert + + * 默认值:insert * **valueFieldDelimiter** + + * 描述:该配置项是考虑了当源数据每行超过两列的情况(如果您的源数据只有两列即 key 和 value 时,那么可以忽略该配置项,不用填写),value 类型是 string 时,value 之间的分隔符,比如 value1\u0001value2\u0001value3。 + + * 必选:否 + + * 默认值:\u0001 - * 描述:该配置项是考虑了当源数据每行超过两列的情况(如果您的源数据只有两列即 key 和 value 时,那么可以忽略该配置项,不用填写),value 类型是 string 时,value 之间的分隔符,比如 value1\u0001value2\u0001value3。 - - * 必选:否 - - * 默认值:\u0001 - * **type和mode** - - * 描述:type 表示 value 的类型,mode 表示在选定的数据类型下的写入模式。 - - * 选项:string/list/set/zset/hash - - | type | 描述 | mode | 说明 | 注意 | - | --- | --- | --- | --- | --- | - | string | 字符串 | set | 存储这个数据,如果已经存在则覆盖 | | - | list | 字符串列表 | lpush | 在 list 最左边存储这个数据 | | - | list | 字符串列表 | rpush | 在 list 最右边存储这个数据 | | - | set | 字符串集合 | sadd | 向 set 集合中存储这个数据,如果已经存在则覆盖 | | - | zset | 有序字符串集合 | zadd | 向 zset 有序集合中存储这个数据,如果已经存在则覆盖 | 当 value 类型是 zset 时,数据源的每一行记录需要遵循相应的规范,即每一行记录除 key 以外,只能有一对 score 和 value,并且 score 必须在 value 前面,rediswriter 方能解析出哪一个 column 是 score,哪一个 column 是 value。 | - | hash | 哈希 | hset | 向 hash 有序集合中存储这个数据,如果已经存在则覆盖 | 当 value 类型是 hash 时,数据源的每一行记录需要遵循相应的规范,即每一行记录除 key 以外,只能有一对 attribute 和 value,并且 attribute 必须在 value 前面,Rediswriter 方能解析出哪一个 column 是 attribute,哪一个 column 是 value。 | - - * 必选:是 - - * 默认值:无 \ No newline at end of file + + * 描述:type 表示 value 的类型,mode 表示在选定的数据类型下的写入模式。 + + * 选项:string/list/set/zset/hash + + | type | 描述 | mode | 说明 | 注意 | + | ------ | ------- | ----- | ---------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------ | + | string | 字符串 | set | 存储这个数据,如果已经存在则覆盖 | | + | list | 字符串列表 | lpush | 在 list 最左边存储这个数据 | | + | list | 字符串列表 | rpush | 在 list 最右边存储这个数据 | | + | set | 字符串集合 | sadd | 向 set 集合中存储这个数据,如果已经存在则覆盖 | | + | zset | 有序字符串集合 | zadd | 向 zset 有序集合中存储这个数据,如果已经存在则覆盖 | 当 value 类型是 zset 时,数据源的每一行记录需要遵循相应的规范,即每一行记录除 key 以外,只能有一对 score 和 value,并且 score 必须在 value 前面,rediswriter 方能解析出哪一个 column 是 score,哪一个 column 是 value。 | + | hash | 哈希 | hset | 向 hash 有序集合中存储这个数据,如果已经存在则覆盖 | 当 value 类型是 hash 时,数据源的每一行记录需要遵循相应的规范,即每一行记录除 key 以外,只能有一对 attribute 和 value,并且 attribute 必须在 value 前面,Rediswriter 方能解析出哪一个 column 是 attribute,哪一个 column 是 value。 | + + * 必选:是 + + * 默认值:无 diff --git a/docs/sqlserverreader.md b/docs/sqlserverreader.md deleted file mode 100644 index 1977bd9c65..0000000000 --- a/docs/sqlserverreader.md +++ /dev/null @@ -1,161 +0,0 @@ -# SQLServer读取插件(sqlserverreader) - -## 1. 配置样例 - -``` -{ - "job": { - "setting": { - "speed": { - "channel": 3, - "bytes": 0 - }, - "errorLimit": { - "record": 10000, - "percentage": 100 - }, - "dirty": { - "path": "/tmp", - "hadoopConfig": { - "fs.default.name": "hdfs://ns1", - "dfs.nameservices": "ns1", - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.ha.automatic-failover.enabled": "true", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "fs.hdfs.impl.disable.cache": "true" - } - } - }, - "content": [ - { - "reader": { - "name": "sqlserverreader", - "parameter": { - "username": "sa", - "password": "Dtstack201610!", - "column": [ - "id", - "v" - ], - "where": "id > 1", - "connection": [ - { - "table": [ - "tb1" - ], - "jdbcUrl": [ - "jdbc:jtds:sqlserver://172.16.10.46:1433;DatabaseName=dq" - ] - } - ], - "splitPk": "id" - } - }, - "writer": { - "name": "mysqlwriter", - "parameter": { - "writeMode": "insert", - "username": "dtstack", - "password": "abc123", - "column": [ - "c1", - "c2" - ], - "batchSize": 1, - "session": [ - "set session sql_mode='ANSI'" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true", - "table": [ - "tb3" - ] - } - ] - } - } - } - ] - } -} - -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填sqlserverreader,否则FlinkX将无法正常加载该插件包。 - * 必选:是
- - * 默认值:无
- -* **jdbcUrl** - - * 描述:针对SQLServer数据库的jdbc连接字符串 - - jdbcUrl按照SqlServer官方规范,并可以填写连接附件控制信息。具体请参看[SqlServer官方文档](http://technet.microsoft.com/zh-cn/library/ms378749(v=SQL.110).aspx)。 - - * 必选:是
- - * 默认值:无
- -* **username** - - * 描述:数据源的用户名
- - * 必选:是
- - * 默认值:无
- -* **password** - - * 描述:数据源指定用户名的密码
- - * 必选:是
- - * 默认值:无
- -* **where** - - * 描述:筛选条件,sqlserverreader根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > $bizdate 。注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。
- - where条件可以有效地进行业务增量同步。如果不填写where语句,包括不提供where的key或者value,FlinkX均视作同步全量数据。 - - * 必选:否
- - * 默认值:无
- -* **splitPk** - - * 描述:sqlserverreader进行数据抽取时,如果指定splitPk,表示用户希望使用splitPk代表的字段进行数据分片,FlinkX因此会启动并发任务进行数据同步,这样可以大大提供数据同步的效能。 - - 推荐splitPk用户使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 - -  目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,MysqlReader将报错! - - 如果splitPk不填写,包括不提供splitPk或者splitPk值为空,FlinkX视作使用单通道同步该表数据。 - - * 必选:否
- - * 默认值:空
- - - -* **column** - - * 描述:所配置的表中需要同步的列名集合。 - - 支持列裁剪,即列可以挑选部分列进行导出。 - - 支持列换序,即列可以不按照表schema信息进行导出。 - - 暂不支持常量列。 - - * 必选:是
- - * 默认值:无
- diff --git a/docs/sqlserverwriter.md b/docs/sqlserverwriter.md deleted file mode 100644 index f6519d8ebc..0000000000 --- a/docs/sqlserverwriter.md +++ /dev/null @@ -1,159 +0,0 @@ -# SQLServer写入插件(sqlserverwriter) - -## 1. 配置样例 - -``` -{ - "job": { - "setting": { - "speed": { - "channel": 4 - }, - "errorLimit": { - "record": 0, - "percentage": 10 - } - }, - "content": [ - { - "reader": { - "parameter": { - "password": "abc123" - "column": [ - "col1", - "col2" - ], - "where": "id > 1", - "connection": [ - { - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?charset=utf8" - ], - "table": [ - "tb2" - ] - } - ], - "splitPk": "col1", - "username": "dtstack" - }, - "name": "mysqlreader" - }, - "writer": { - "name": "sqlserverwriter", - "parameter": { - "batchSize": 2048, - "username": "sa", - "password": "Dtstack201610!", - "column": [ - "id", - "v" - ], - "preSql": [], - "postSql": [], - "writeMode": "replace", - "connection": [ - { - "jdbcUrl": "jdbc:jtds:sqlserver://172.16.10.46:1433;DatabaseName=dq", - "table": [ - "tb1" - ] - } - ] - } - } - } - ] - } -} - -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填sqlserverwriter,否则Flinkx将无法正常加载该插件包。 - * 必选:是
- - * 默认值:无
- -* **jdbcUrl** - - * 描述:针对SQLServer数据库的jdbc连接字符串 - - jdbcUrl按照SqlServer官方规范,并可以填写连接附件控制信息。具体请参看[SqlServer官方文档](http://technet.microsoft.com/zh-cn/library/ms378749(v=SQL.110).aspx)。 - - * 必选:是
- - * 默认值:无
- -* **username** - - * 描述:数据源的用户名
- - * 必选:是
- - * 默认值:无
- -* **password** - - * 描述:数据源指定用户名的密码
- - * 必选:是
- - * 默认值:无
- -* **column** - - * 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 - - * 必选:是
- - * 默认值:否
- - * 默认值:无
- -* **preSql** - - * 描述:写入数据到目的表前,会先执行这里的一组标准语句。 - - * 必选:否
- - * 默认值:无
- -* **postSql** - - * 描述:写入数据到目的表后,会执行这里的一组标准语句。 - - * 必选:否
- - * 默认值:无
- -* **table** - - * 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表。 - - 注意:table 和 jdbcUrl 必须包含在 connection 配置单元中 - - * 必选:是
- - * 默认值:无
- -* **writeMode** - - * 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句
- ** 在sqlserver中, 用merge into模拟后两种插入语义。 ** - * 必选:是
- - * 所有选项:insert/replace/update
- - * 默认值:insert
- -* **batchSize** - - * 描述:一次性批量提交的记录数大小 - - * 必选:否
- - * 默认值:1024
\ No newline at end of file diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java index b987f04037..c91b32a0d8 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java @@ -57,7 +57,7 @@ public EsReader(DataTransferConfig config, StreamExecutionEnvironment env) { address = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_ADDRESS); index = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_INDEX); type = readerConfig.getParameter().getStringVal(EsConfigKeys.KEY_TYPE); - batchSize = readerConfig.getParameter().getIntVal(EsConfigKeys.KEY_BATCH_SIZE, 0); + batchSize = readerConfig.getParameter().getIntVal(EsConfigKeys.KEY_BATCH_SIZE, 10); clientConfig = new HashMap<>(); clientConfig.put(EsConfigKeys.KEY_TIMEOUT, readerConfig.getParameter().getVal(EsConfigKeys.KEY_TIMEOUT)); diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java index eafa081221..b61f298ed6 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java @@ -38,10 +38,6 @@ public class HdfsConfigKeys { public static final String KEY_WRITE_MODE = "writeMode"; - public static final String KEY_USERNAME = "username"; - - public static final String KEY_PASSWORD = "password"; - public static final String KEY_FULL_COLUMN_NAME_LIST = "fullColumnName"; public static final String KEY_FULL_COLUMN_TYPE_LIST = "fullColumnType"; @@ -52,8 +48,6 @@ public class HdfsConfigKeys { public static final String KEY_COMPRESS = "compress"; - public static final String KEY_PARTITION = "partition"; - public static final String KEY_FILE_NAME = "fileName"; public static final String KEY_ENCODING = "encoding"; diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsWriter.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsWriter.java index e41871f899..0912b94a94 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsWriter.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsWriter.java @@ -75,17 +75,6 @@ public class HdfsWriter extends DataWriter { protected List fullColumnType; - /** hive config **/ - protected String partition; - - protected String dbUrl; - - protected String username; - - protected String password; - - protected String table; - protected static final String DATA_SUBDIR = ".data"; protected static final String FINISHED_SUBDIR = ".finished"; diff --git a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsWriter.java b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsWriter.java index f71d3e39e9..a129691e5b 100644 --- a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsWriter.java +++ b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsWriter.java @@ -49,8 +49,6 @@ public class OdpsWriter extends DataWriter { protected String projectName; - protected String writeMode; - protected long bufferSize; public OdpsWriter(DataTransferConfig config) { @@ -61,7 +59,6 @@ public OdpsWriter(DataTransferConfig config) { partition = writerConfig.getParameter().getStringVal(OdpsConfigKeys.KEY_PARTITION); mode = writerConfig.getParameter().getStringVal(OdpsConfigKeys.KEY_WRITE_MODE); projectName = writerConfig.getParameter().getStringVal(OdpsConfigKeys.KEY_PROJECT); - writeMode = writerConfig.getParameter().getStringVal(OdpsConfigKeys.KEY_MODE); bufferSize = writerConfig.getParameter().getLongVal(OdpsConfigKeys.KEY_BUFFER_SIZE, 0); if (bufferSize == 0){ @@ -89,7 +86,7 @@ public DataStreamSink writeData(DataStream dataSet) { builder.setPartition(partition); builder.setColumnNames(columnName); builder.setColumnTypes(columnType); - builder.setWriteMode(writeMode); + builder.setWriteMode(mode); builder.setTableName(tableName); builder.setOdpsConfig(odpsConfig); builder.setDirtyPath(dirtyPath); From e0fb86ba82794106c324104d10ee6774bc4ce91b Mon Sep 17 00:00:00 2001 From: lijiangbo <245730400@qq.com> Date: Thu, 9 May 2019 11:41:21 +0800 Subject: [PATCH 96/96] =?UTF-8?q?=E6=9B=B4=E6=96=B0=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/streamreader.md | 60 ++++++++++++++++++++++++++++++++++++++++++++ docs/streamwriter.md | 41 ++++++++++++++++++++++++++++++ 2 files changed, 101 insertions(+) create mode 100644 docs/streamreader.md create mode 100644 docs/streamwriter.md diff --git a/docs/streamreader.md b/docs/streamreader.md new file mode 100644 index 0000000000..68886faa7a --- /dev/null +++ b/docs/streamreader.md @@ -0,0 +1,60 @@ +# Stream读取插件(streamreader) + +## 1. 配置样例 + +``` +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "column": [ + { + "type": "int", + "value":"xxx" + } + ], + "sliceRecordCount":10000 + }, + "name": "streamreader" + }, + "writer": {} + } + ], + "setting": {} + } +} +``` + +## 2. 参数说明 + +* **name** + + * 描述:插件名,此处填写插件名称,streamreader。 + + * 必选:是 + + * 默认值:无 + +* **sliceRecordCount** + + * 描述:每个通道生成的数据条数,不配置此参数或者配置为0,程序会持续生成数据,不会停止 + + * 必选:否 + + * 默认值:0 + +* **column** + + * 描述:需要生成的字段。 + + * 属性说明: + + * type:字段类型,程序根据指定的字段类型生成模拟数据,支持基本数据类型以及基本类型的数组,"int[]"表示生成一个长度随机的整形数组; + + * value:常量值,程序使用此字段的值直接返回; + + * 必选:是 + + * 默认值:无 diff --git a/docs/streamwriter.md b/docs/streamwriter.md new file mode 100644 index 0000000000..0d786d2f6b --- /dev/null +++ b/docs/streamwriter.md @@ -0,0 +1,41 @@ +# Stream写入插件(streamwriter) + +## 1. 配置样例 + +``` +{ + "job": { + "content": [ + { + "reader": {}, + "writer": { + "parameter": { + "print":true + }, + "name": "streamwriter" + } + } + ], + "setting": {} + } +} +``` + +## 2. 参数说明 + +* **name** + + * 描述:插件名,此处填写插件名称,streamwriter,此插件用来单独测试reader插件,对读到的数据不做任务处理; + + * 必选:是 + + * 默认值:无 + +* **print** + + * 描述:是否在控制台打印数据 + + * 必选:否 + + * 默认值:false +