Skip to content

Commit 298fbd3

Browse files
committed
Merge branch '1.10_release_4.0.x' into hotfix_1.10_4.0.x_34672
2 parents b4d57b1 + 68f519b commit 298fbd3

File tree

5 files changed

+117
-67
lines changed

5 files changed

+117
-67
lines changed

core/src/main/java/com/dtstack/flink/sql/side/BaseSideInfo.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -118,7 +118,7 @@ public void parseSelectFields(JoinInfo joinInfo){
118118
public String getTargetFieldType(String fieldName){
119119
int fieldIndex = sideTableInfo.getFieldList().indexOf(fieldName);
120120
if(fieldIndex == -1){
121-
throw new RuntimeException(sideTableInfo.getName() + "can't find field: " + fieldName);
121+
throw new RuntimeException(sideTableInfo.getName() + " can't find field: " + fieldName);
122122
}
123123

124124
return sideTableInfo.getFieldTypes()[fieldIndex];

core/src/main/java/com/dtstack/flink/sql/table/AbstractTableParser.java

Lines changed: 44 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -19,12 +19,15 @@
1919

2020
package com.dtstack.flink.sql.table;
2121

22+
import com.dtstack.flink.sql.side.AbstractSideTableInfo;
2223
import com.dtstack.flink.sql.util.ClassUtil;
2324
import com.dtstack.flink.sql.util.DtStringUtil;
2425
import com.google.common.base.Preconditions;
2526
import com.google.common.collect.Maps;
2627
import org.apache.commons.lang3.StringUtils;
2728
import org.apache.flink.api.java.tuple.Tuple2;
29+
import org.slf4j.Logger;
30+
import org.slf4j.LoggerFactory;
2831

2932
import java.util.Arrays;
3033
import java.util.List;
@@ -43,6 +46,8 @@
4346

4447
public abstract class AbstractTableParser {
4548

49+
private static final Logger LOG = LoggerFactory.getLogger(AbstractTableParser.class);
50+
4651
private static final String PRIMARY_KEY = "primaryKey";
4752
private static final String NEST_JSON_FIELD_KEY = "nestFieldKey";
4853
private static final String CHAR_TYPE_NO_LENGTH = "CHAR";
@@ -105,30 +110,50 @@ public void parseFieldsInfo(String fieldsInfo, AbstractTableInfo tableInfo) {
105110
continue;
106111
}
107112

108-
Tuple2<String, String> t = extractType(fieldRow, tableInfo.getName());
109-
String fieldName = t.f0;
110-
String fieldType = t.f1;
113+
handleKeyNotHaveAlias(fieldRow, tableInfo);
114+
}
111115

112-
Class fieldClass;
113-
AbstractTableInfo.FieldExtraInfo fieldExtraInfo = null;
116+
/*
117+
* check whether filed list contains pks and then add pks into field list.
118+
* because some no-sql database is not primary key. eg :redis、hbase etc...
119+
*/
120+
if (tableInfo instanceof AbstractSideTableInfo) {
121+
tableInfo.getPrimaryKeys().stream()
122+
.filter(pk -> (!tableInfo.getFieldList().contains(pk)))
123+
.forEach(pk -> {
124+
try {
125+
handleKeyNotHaveAlias(String.format("%s varchar", pk.trim()), tableInfo);
126+
} catch (Exception e) {
127+
LOG.error(String.format("Add primary key to field list failed. Reason: %s", e.getMessage()));
128+
}
129+
});
130+
}
114131

115-
Matcher matcher = charTypePattern.matcher(fieldType);
116-
if (matcher.find()) {
117-
fieldClass = dbTypeConvertToJavaType(CHAR_TYPE_NO_LENGTH);
118-
fieldExtraInfo = new AbstractTableInfo.FieldExtraInfo();
119-
fieldExtraInfo.setLength(Integer.parseInt(matcher.group(1)));
120-
} else {
121-
fieldClass = dbTypeConvertToJavaType(fieldType);
122-
}
132+
tableInfo.finish();
133+
}
123134

124-
tableInfo.addPhysicalMappings(fieldName, fieldName);
125-
tableInfo.addField(fieldName);
126-
tableInfo.addFieldClass(fieldClass);
127-
tableInfo.addFieldType(fieldType);
128-
tableInfo.addFieldExtraInfo(fieldExtraInfo);
135+
private void handleKeyNotHaveAlias(String fieldRow, AbstractTableInfo tableInfo) {
136+
Tuple2<String, String> t = extractType(fieldRow, tableInfo.getName());
137+
String fieldName = t.f0;
138+
String fieldType = t.f1;
139+
140+
Class fieldClass;
141+
AbstractTableInfo.FieldExtraInfo fieldExtraInfo = null;
142+
143+
Matcher matcher = charTypePattern.matcher(fieldType);
144+
if (matcher.find()) {
145+
fieldClass = dbTypeConvertToJavaType(CHAR_TYPE_NO_LENGTH);
146+
fieldExtraInfo = new AbstractTableInfo.FieldExtraInfo();
147+
fieldExtraInfo.setLength(Integer.parseInt(matcher.group(1)));
148+
} else {
149+
fieldClass = dbTypeConvertToJavaType(fieldType);
129150
}
130151

131-
tableInfo.finish();
152+
tableInfo.addPhysicalMappings(fieldName, fieldName);
153+
tableInfo.addField(fieldName);
154+
tableInfo.addFieldClass(fieldClass);
155+
tableInfo.addFieldType(fieldType);
156+
tableInfo.addFieldExtraInfo(fieldExtraInfo);
132157
}
133158

134159
private Tuple2<String, String> extractType(String fieldRow, String tableName) {

hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -110,6 +110,12 @@ private String dealOneRow(ArrayList<ArrayList<KeyValue>> args, String rowKeyStr,
110110
//The order of the fields defined in the data conversion table
111111
List<Object> sideVal = Lists.newArrayList();
112112
for (String key : colNames) {
113+
114+
if (!sideMap.containsKey(key)) {
115+
sideVal.add(rowKeyStr);
116+
continue;
117+
}
118+
113119
Object val = sideMap.get(key);
114120
if (val == null) {
115121
LOG.error("can't get data with column {}", key);

hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -84,6 +84,12 @@ public void asyncGetData(String tableName, String rowKeyStr, BaseRow input, Resu
8484
//The order of the fields defined in the data conversion table
8585
List<Object> sideVal = Lists.newArrayList();
8686
for(String key : colNames){
87+
88+
if (!sideMap.containsKey(key)) {
89+
sideVal.add(rowKeyStr);
90+
continue;
91+
}
92+
8793
Object val = sideMap.get(key);
8894
if(val == null){
8995
LOG.error("can't get data with column {}", key);

kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaDeserializationMetricWrapper.java

Lines changed: 60 additions & 47 deletions
Original file line numberDiff line numberDiff line change
@@ -19,23 +19,24 @@
1919
package com.dtstack.flink.sql.source.kafka;
2020

2121
import com.dtstack.flink.sql.format.DeserializationMetricWrapper;
22+
import com.dtstack.flink.sql.util.ReflectionUtils;
2223
import org.apache.flink.api.common.serialization.DeserializationSchema;
2324
import org.apache.flink.api.common.typeinfo.TypeInformation;
2425
import org.apache.flink.metrics.Gauge;
2526
import org.apache.flink.metrics.MetricGroup;
2627
import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread;
2728
import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
29+
import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
30+
import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState;
2831
import org.apache.flink.types.Row;
2932
import org.apache.kafka.clients.consumer.KafkaConsumer;
3033
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
3134
import org.apache.kafka.common.TopicPartition;
3235
import org.slf4j.Logger;
3336
import org.slf4j.LoggerFactory;
3437

35-
import java.io.IOException;
3638
import java.lang.reflect.Field;
37-
import java.util.Set;
38-
import java.util.concurrent.atomic.AtomicBoolean;
39+
import java.util.List;
3940

4041
import static com.dtstack.flink.sql.metric.MetricConstant.DT_PARTITION_GROUP;
4142
import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_GROUP;
@@ -52,77 +53,89 @@ public class KafkaDeserializationMetricWrapper extends DeserializationMetricWrap
5253

5354
private static final Logger LOG = LoggerFactory.getLogger(KafkaDeserializationMetricWrapper.class);
5455

55-
private AbstractFetcher<Row, ?> fetcher;
56-
57-
private AtomicBoolean firstMsg = new AtomicBoolean(true);
58-
5956
private Calculate calculate;
6057

6158
public KafkaDeserializationMetricWrapper(TypeInformation<Row> typeInfo, DeserializationSchema<Row> deserializationSchema, Calculate calculate) {
6259
super(typeInfo, deserializationSchema);
6360
this.calculate = calculate;
6461
}
6562

66-
@Override
67-
protected void beforeDeserialize() throws IOException {
68-
super.beforeDeserialize();
69-
if (firstMsg.compareAndSet(true, false)) {
70-
try {
71-
registerPtMetric(fetcher);
72-
} catch (Exception e) {
73-
LOG.error("register topic partition metric error.", e);
74-
}
75-
}
76-
}
77-
7863
protected void registerPtMetric(AbstractFetcher<Row, ?> fetcher) throws Exception {
79-
Field consumerThreadField = getConsumerThreadField(fetcher);
64+
Field consumerThreadField = ReflectionUtils.getDeclaredField(fetcher, "consumerThread");
8065
consumerThreadField.setAccessible(true);
8166
KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher);
8267

8368
Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions");
8469
hasAssignedPartitionsField.setAccessible(true);
8570

86-
//wait until assignedPartitions
87-
88-
boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread);
71+
// get subtask unassigned kafka topic partition
72+
Field subscribedPartitionStatesField = ReflectionUtils.getDeclaredField(fetcher, "subscribedPartitionStates");
73+
subscribedPartitionStatesField.setAccessible(true);
74+
List<KafkaTopicPartitionState<KafkaTopicPartition>> subscribedPartitionStates = (List<KafkaTopicPartitionState<KafkaTopicPartition>>) subscribedPartitionStatesField.get(fetcher);
8975

90-
if (!hasAssignedPartitions) {
91-
throw new RuntimeException("wait 50 secs, but not assignedPartitions");
92-
}
76+
// init partition lag metric
77+
for (KafkaTopicPartitionState<KafkaTopicPartition> kafkaTopicPartitionState : subscribedPartitionStates) {
78+
KafkaTopicPartition kafkaTopicPartition = kafkaTopicPartitionState.getKafkaTopicPartition();
79+
MetricGroup topicMetricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, kafkaTopicPartition.getTopic());
9380

94-
Field consumerField = consumerThread.getClass().getDeclaredField("consumer");
95-
consumerField.setAccessible(true);
96-
97-
KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread);
98-
Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions");
99-
subscriptionStateField.setAccessible(true);
100-
101-
//topic partitions lag
102-
SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer);
103-
Set<TopicPartition> assignedPartitions = subscriptionState.assignedPartitions();
104-
105-
for (TopicPartition topicPartition : assignedPartitions) {
106-
MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic())
107-
.addGroup(DT_PARTITION_GROUP, topicPartition.partition() + "");
81+
MetricGroup metricGroup = topicMetricGroup.addGroup(DT_PARTITION_GROUP, kafkaTopicPartition.getPartition() + "");
10882
metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new Gauge<Long>() {
83+
// tmp variable
84+
boolean initLag = true;
85+
int partitionIndex;
86+
SubscriptionState subscriptionState;
87+
TopicPartition topicPartition;
88+
10989
@Override
11090
public Long getValue() {
111-
return calculate.calc(subscriptionState, topicPartition);
91+
// first time register metrics
92+
if (initLag) {
93+
partitionIndex = kafkaTopicPartition.getPartition();
94+
initLag = false;
95+
return -1L;
96+
}
97+
// when kafka topic partition assigned calc metrics
98+
if (subscriptionState == null) {
99+
try {
100+
Field consumerField = consumerThread.getClass().getDeclaredField("consumer");
101+
consumerField.setAccessible(true);
102+
103+
KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread);
104+
Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions");
105+
subscriptionStateField.setAccessible(true);
106+
107+
boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread);
108+
109+
if (!hasAssignedPartitions) {
110+
LOG.error("wait 50 secs, but not assignedPartitions");
111+
}
112+
113+
subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer);
114+
115+
topicPartition = subscriptionState
116+
.assignedPartitions()
117+
.stream()
118+
.filter(x -> x.partition() == partitionIndex)
119+
.findFirst()
120+
.get();
121+
122+
} catch (Exception e) {
123+
LOG.error(e.getMessage());
124+
}
125+
return -1L;
126+
} else {
127+
return calculate.calc(subscriptionState, topicPartition);
128+
}
112129
}
113130
});
114131
}
115132
}
116133

117134
public void setFetcher(AbstractFetcher<Row, ?> fetcher) {
118-
this.fetcher = fetcher;
119-
}
120-
121-
private Field getConsumerThreadField(AbstractFetcher fetcher) throws NoSuchFieldException {
122135
try {
123-
return fetcher.getClass().getDeclaredField("consumerThread");
136+
registerPtMetric(fetcher);
124137
} catch (Exception e) {
125-
return fetcher.getClass().getSuperclass().getDeclaredField("consumerThread");
138+
LOG.error("register topic partition metric error.", e);
126139
}
127140
}
128141
}

0 commit comments

Comments
 (0)