|
64 | 64 |
|
65 | 65 | import javax.annotation.Nullable; |
66 | 66 |
|
| 67 | +import java.time.Duration; |
67 | 68 | import java.util.ArrayList; |
68 | 69 | import java.util.Arrays; |
69 | 70 | import java.util.Collections; |
|
80 | 81 | import static com.alibaba.fluss.record.TestData.DATA1_KEY_TYPE; |
81 | 82 | import static com.alibaba.fluss.record.TestData.DATA1_ROW_TYPE; |
82 | 83 | import static com.alibaba.fluss.record.TestData.DATA1_SCHEMA; |
| 84 | +import static com.alibaba.fluss.record.TestData.DATA1_SCHEMA_PK; |
83 | 85 | import static com.alibaba.fluss.record.TestData.DATA1_TABLE_ID; |
84 | 86 | import static com.alibaba.fluss.record.TestData.DATA1_TABLE_ID_PK; |
85 | 87 | import static com.alibaba.fluss.record.TestData.DATA1_TABLE_PATH; |
|
102 | 104 | import static com.alibaba.fluss.testutils.DataTestUtils.genMemoryLogRecordsByObject; |
103 | 105 | import static com.alibaba.fluss.testutils.DataTestUtils.getKeyValuePairs; |
104 | 106 | import static com.alibaba.fluss.testutils.DataTestUtils.row; |
| 107 | +import static com.alibaba.fluss.testutils.common.CommonTestUtils.retry; |
105 | 108 | import static org.assertj.core.api.Assertions.assertThat; |
106 | 109 | import static org.assertj.core.api.Assertions.assertThatThrownBy; |
107 | 110 |
|
@@ -1163,6 +1166,51 @@ void testStopReplica() throws Exception { |
1163 | 1166 | replicaManager.getReplicaOrException(tb); |
1164 | 1167 | } |
1165 | 1168 |
|
| 1169 | + @Test |
| 1170 | + void testKvDataVisibility() throws Exception { |
| 1171 | + // The CDC log is only visible after the KV has been flushed to RocksDB. In other words, |
| 1172 | + // when we can read the CDC log, the associated kv record must have been |
| 1173 | + // inserted/updated/deleted in RocksDB. The reason for ensuring this visibility is that we |
| 1174 | + // first buffer the data in memory before flushing it to RocksDB. Thus, we need to guarantee |
| 1175 | + // visibility. |
| 1176 | + TableBucket tb = new TableBucket(DATA1_TABLE_ID_PK, 1); |
| 1177 | + makeKvTableAsLeader(DATA1_TABLE_ID_PK, DATA1_TABLE_PATH_PK, tb.getBucket()); |
| 1178 | + Replica replica = replicaManager.getReplicaOrException(tb); |
| 1179 | + |
| 1180 | + // retry send kv records to kv store, if the highWatermark increased, the kv record must be |
| 1181 | + // visible in rocksdb. |
| 1182 | + int round = 1000; |
| 1183 | + CompactedKeyEncoder keyEncoder = new CompactedKeyEncoder(DATA1_ROW_TYPE, new int[] {0}); |
| 1184 | + CompletableFuture<List<PutKvResultForBucket>> future; |
| 1185 | + for (int i = 0; i < round; i++) { |
| 1186 | + future = new CompletableFuture<>(); |
| 1187 | + Object[] key = {i}; |
| 1188 | + Object[] value = {i, "a"}; |
| 1189 | + byte[] keyBytes = keyEncoder.encodeKey(row(key)); |
| 1190 | + byte[] valueBytes = |
| 1191 | + ValueEncoder.encodeValue( |
| 1192 | + DEFAULT_SCHEMA_ID, compactedRow(DATA1_SCHEMA_PK.getRowType(), value)); |
| 1193 | + List<byte[]> lookups = replica.lookups(Collections.singletonList(keyBytes)); |
| 1194 | + assertThat(lookups.size()).isEqualTo(1); |
| 1195 | + assertThat(lookups.get(0)).isNull(); |
| 1196 | + replicaManager.putRecordsToKv( |
| 1197 | + 20000, |
| 1198 | + 1, |
| 1199 | + Collections.singletonMap( |
| 1200 | + tb, genKvRecordBatch(Collections.singletonList(Tuple2.of(key, value)))), |
| 1201 | + null, |
| 1202 | + future::complete); |
| 1203 | + assertThat(future.get()).containsOnly(new PutKvResultForBucket(tb, i + 1)); |
| 1204 | + |
| 1205 | + int expectedHw = i + 1; |
| 1206 | + retry( |
| 1207 | + Duration.ofMinutes(1), |
| 1208 | + () -> assertThat(replica.getLogHighWatermark()).isEqualTo(expectedHw)); |
| 1209 | + assertThat(replica.lookups(Collections.singletonList(keyBytes))) |
| 1210 | + .containsExactlyInAnyOrder(valueBytes); |
| 1211 | + } |
| 1212 | + } |
| 1213 | + |
1166 | 1214 | @Test |
1167 | 1215 | void testSnapshotKvReplicas() throws Exception { |
1168 | 1216 | // create multiple kv replicas and all do the snapshot operation |
|
0 commit comments