Skip to content

Commit ea559d9

Browse files
author
dapeng
committed
解决冲突
1 parent 720e0d1 commit ea559d9

File tree

2 files changed

+22
-20
lines changed

2 files changed

+22
-20
lines changed

redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java

Lines changed: 9 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@
3030
import org.apache.commons.pool2.impl.GenericObjectPoolConfig;
3131
import org.apache.flink.api.java.typeutils.RowTypeInfo;
3232
import com.google.common.collect.Maps;
33+
import org.apache.flink.table.runtime.types.CRow;
3334
import org.apache.flink.types.Row;
3435
import org.apache.flink.util.Collector;
3536
import org.slf4j.Logger;
@@ -93,15 +94,15 @@ protected void reloadCache() {
9394
}
9495

9596
@Override
96-
public void flatMap(Row row, Collector<Row> out) throws Exception {
97+
public void flatMap(CRow input, Collector<CRow> out) throws Exception {
9798
Map<String, String> inputParams = Maps.newHashMap();
9899
for (int i = 0; i < sideInfo.getEqualValIndex().size(); i++) {
99100
Integer conValIndex = sideInfo.getEqualValIndex().get(i);
100-
Object equalObj = row.getField(conValIndex);
101+
Object equalObj = input.row().getField(conValIndex);
101102
if(equalObj == null){
102103
if(sideInfo.getJoinType() == JoinType.LEFT){
103-
Row data = fillData(row, null);
104-
out.collect(data);
104+
Row data = fillData(input.row(), null);
105+
out.collect(new CRow(data, input.change()));
105106
}
106107
return;
107108
}
@@ -117,13 +118,13 @@ public void flatMap(Row row, Collector<Row> out) throws Exception {
117118
if(sideInfo.getJoinType() != JoinType.LEFT){
118119
return;
119120
}
120-
Row data = fillData(row, null);
121-
out.collect(data);
121+
Row data = fillData(input.row(), null);
122+
out.collect(new CRow(data, input.change()));
122123
return;
123124

124125
}
125-
Row newRow = fillData(row, cacheMap);
126-
out.collect(newRow);
126+
Row newRow = fillData(input.row(), cacheMap);
127+
out.collect(new CRow(newRow, input.change()));
127128
}
128129

129130
private String buildCacheKey(Map<String, String> refData) {

redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java

Lines changed: 13 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,7 @@
3838
import com.google.common.collect.Maps;
3939
import org.apache.flink.configuration.Configuration;
4040
import org.apache.flink.streaming.api.functions.async.ResultFuture;
41+
import org.apache.flink.table.runtime.types.CRow;
4142
import org.apache.flink.types.Row;
4243

4344
import java.util.Collections;
@@ -120,14 +121,14 @@ public Row fillData(Row input, Object sideInput) {
120121
}
121122

122123
@Override
123-
public void asyncInvoke(Row input, ResultFuture<Row> resultFuture) throws Exception {
124-
Row inputRow = Row.copy(input);
124+
public void asyncInvoke(CRow input, ResultFuture<CRow> resultFuture) throws Exception {
125+
CRow inputCopy = new CRow(input.row(), input.change());
125126
Map<String, Object> refData = Maps.newHashMap();
126127
for (int i = 0; i < sideInfo.getEqualValIndex().size(); i++) {
127128
Integer conValIndex = sideInfo.getEqualValIndex().get(i);
128-
Object equalObj = inputRow.getField(conValIndex);
129+
Object equalObj = input.row().getField(conValIndex);
129130
if(equalObj == null){
130-
dealMissKey(inputRow, resultFuture);
131+
dealMissKey(inputCopy, resultFuture);
131132
return;
132133
}
133134
refData.put(sideInfo.getEqualFieldList().get(i), equalObj);
@@ -141,14 +142,14 @@ public void asyncInvoke(Row input, ResultFuture<Row> resultFuture) throws Except
141142
CacheObj val = getFromCache(key);
142143
if(val != null){
143144
if(ECacheContentType.MissVal == val.getType()){
144-
dealMissKey(inputRow, resultFuture);
145+
dealMissKey(inputCopy, resultFuture);
145146
return;
146147
}else if(ECacheContentType.MultiLine == val.getType()){
147148
try {
148-
Row row = fillData(inputRow, val.getContent());
149-
resultFuture.complete(Collections.singleton(row));
149+
Row row = fillData(input.row(), val.getContent());
150+
resultFuture.complete(Collections.singleton(new CRow(row, inputCopy.change())));
150151
} catch (Exception e) {
151-
dealFillDataError(resultFuture, e, inputRow);
152+
dealFillDataError(resultFuture, e, inputCopy);
152153
}
153154
}else{
154155
RuntimeException exception = new RuntimeException("not support cache obj type " + val.getType());
@@ -164,14 +165,14 @@ public void asyncInvoke(Row input, ResultFuture<Row> resultFuture) throws Except
164165
public void accept(Map<String, String> values) {
165166
if (MapUtils.isNotEmpty(values)) {
166167
try {
167-
Row row = fillData(inputRow, values);
168+
Row row = fillData(input.row(), values);
168169
dealCacheData(key,CacheObj.buildCacheObj(ECacheContentType.MultiLine, values));
169-
resultFuture.complete(Collections.singleton(row));
170+
resultFuture.complete(Collections.singleton(new CRow(row, inputCopy.change())));
170171
} catch (Exception e) {
171-
dealFillDataError(resultFuture, e, inputRow);
172+
dealFillDataError(resultFuture, e, inputCopy);
172173
}
173174
} else {
174-
dealMissKey(inputRow, resultFuture);
175+
dealMissKey(inputCopy, resultFuture);
175176
dealCacheData(key,CacheMissVal.getMissKeyObj());
176177
}
177178
}

0 commit comments

Comments
 (0)