fhueske commented on a change in pull request #6621: [FLINK-8686] [sql-client]
Limit result size for prototyping modes
URL: https://github.com/apache/flink/pull/6621#discussion_r213359097
##########
File path:
flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/result/MaterializedCollectStreamResult.java
##########
@@ -112,31 +155,64 @@ else if (!isRetrieving()) {
@Override
protected void processRecord(Tuple2<Boolean, Row> change) {
synchronized (resultLock) {
- final Row row = change.f1;
// insert
if (change.f0) {
- materializedTable.add(row);
- rowPositionCache.put(row,
materializedTable.size() - 1);
+ processInsert(change.f1);
}
// delete
else {
- // delete the newest record first to minimize
per-page changes
- final Integer cachedPos =
rowPositionCache.get(row);
- final int startSearchPos;
- if (cachedPos != null) {
- startSearchPos = Math.min(cachedPos,
materializedTable.size() - 1);
- } else {
- startSearchPos =
materializedTable.size() - 1;
- }
-
- for (int i = startSearchPos; i >= 0; i--) {
- if
(materializedTable.get(i).equals(row)) {
- materializedTable.remove(i);
- rowPositionCache.remove(row);
- break;
- }
- }
+ processDelete(change.f1);
}
}
}
+
+ @VisibleForTesting
+ protected List<Row> getMaterializedTable() {
+ return materializedTable;
+ }
+
+ //
--------------------------------------------------------------------------------------------
+
+ private void processInsert(Row row) {
+ // limit the materialized table
+ if (materializedTable.size() - validRowPosition >= maxRowCount)
{
+ cleanUp();
+ }
+ materializedTable.add(row);
+ rowPositionCache.put(row, materializedTable.size() - 1);
+ }
+
+ private void processDelete(Row row) {
+ // delete the newest record first to minimize per-page changes
+ final Integer cachedPos = rowPositionCache.get(row);
+ final int startSearchPos;
+ if (cachedPos != null) {
+ startSearchPos = Math.min(cachedPos,
materializedTable.size() - 1);
+ } else {
+ startSearchPos = materializedTable.size() - 1;
+ }
+
+ for (int i = startSearchPos; i >= validRowPosition; i--) {
+ if (materializedTable.get(i).equals(row)) {
+ materializedTable.remove(i);
+ rowPositionCache.remove(row);
+ break;
+ }
+ }
+ }
+
+ private void cleanUp() {
+ // invalidate row
+ final Row deleteRow = materializedTable.get(0);
+ rowPositionCache.remove(deleteRow);
+ materializedTable.set(0, null);
+
+ validRowPosition++;
+
+ // perform clean up in batches
+ if (validRowPosition >= overcommitThreshold) {
+ materializedTable.subList(0, validRowPosition).clear();
Review comment:
We could update all indexes with a pass over the cache and subtracting
`overcommitThreshold` from all cached indexes.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services