wuchong commented on code in PR #2016:
URL: https://github.com/apache/fluss/pull/2016#discussion_r2572918209
##########
fluss-client/src/main/java/org/apache/fluss/client/write/Sender.java:
##########
@@ -602,4 +607,31 @@ public void initiateClose() {
accumulator.close();
running = false;
}
+
+ private static final class TableKey {
+ private final long tableId;
+ private final TableInfo tableInfo;
+
+ public TableKey(long tableId, TableInfo tableInfo) {
+ this.tableId = tableId;
+ this.tableInfo = tableInfo;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ TableKey tableKey = (TableKey) o;
+ return tableId == tableKey.tableId;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(tableId);
Review Comment:
The `TableKey` is super hack, because it is used as a map key, but the
TableInfo is not considered for comparison. If we want to carry the TableInfo
in the Map, it should belong to the values. However, I think we don't need the
TableInfo to check log/kv. I made a change for this.
##########
fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlussSinkITCase.java:
##########
@@ -93,11 +92,6 @@ public void setup() throws Exception {
env.setParallelism(2);
}
- @AfterAll
- protected static void afterAll() throws Exception {
- conn.close();
- }
Review Comment:
I think we can revert all the test changes.
##########
fluss-client/src/main/java/org/apache/fluss/client/write/ArrowLogWriteBatch.java:
##########
@@ -52,14 +53,15 @@ public class ArrowLogWriteBatch extends WriteBatch {
public ArrowLogWriteBatch(
int bucketId,
PhysicalTablePath physicalTablePath,
- int schemaId,
+ TableInfo tableInfo,
Review Comment:
We should avoid passing large context objects; pass only the required data.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]