提交 43b8e57b 编写于 作者: G Gyula Fora

[FLINK-2924] [streaming] Improve compacting logic

上级 c254bda3
......@@ -96,6 +96,8 @@ public class LazyDbKvState<K, V> implements KvState<K, V, DbStateBackend>, Check
private long nextTs;
private Map<Long, Long> completedCheckpoints = new HashMap<>();
private long lastCompactedTs;
// ------------------------------------------------------
/**
......@@ -104,14 +106,15 @@ public class LazyDbKvState<K, V> implements KvState<K, V, DbStateBackend>, Check
*/
public LazyDbKvState(String kvStateId, boolean compact, ShardedConnection cons, DbBackendConfig conf,
TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, V defaultValue) throws IOException {
this(kvStateId, compact, cons, conf, keySerializer, valueSerializer, defaultValue, 1);
this(kvStateId, compact, cons, conf, keySerializer, valueSerializer, defaultValue, 1, 0);
}
/**
* Initialize the {@link LazyDbKvState} from a snapshot.
*/
public LazyDbKvState(String kvStateId, boolean compact, ShardedConnection cons, final DbBackendConfig conf,
TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, V defaultValue, long nextTs)
TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, V defaultValue, long nextTs,
long lastCompactedTs)
throws IOException {
this.kvStateId = kvStateId;
......@@ -130,6 +133,7 @@ public class LazyDbKvState<K, V> implements KvState<K, V, DbStateBackend>, Check
this.sqlRetrySleep = conf.getSleepBetweenSqlRetries();
this.nextTs = nextTs;
this.lastCompactedTs = lastCompactedTs;
this.cache = new StateCache(conf.getKvCacheSize(), conf.getNumElementsToEvict());
......@@ -192,7 +196,7 @@ public class LazyDbKvState<K, V> implements KvState<K, V, DbStateBackend>, Check
nextTs = timestamp + 1;
completedCheckpoints.put(checkpointId, timestamp);
return new DbKvStateSnapshot<K, V>(kvStateId, timestamp);
return new DbKvStateSnapshot<K, V>(kvStateId, timestamp, lastCompactedTs);
}
/**
......@@ -245,8 +249,9 @@ public class LazyDbKvState<K, V> implements KvState<K, V, DbStateBackend>, Check
if (compactEvery > 0 && compact && checkpointId % compactEvery == 0) {
try {
for (Connection c : connections.connections()) {
dbAdapter.compactKvStates(kvStateId, c, 0, ts);
dbAdapter.compactKvStates(kvStateId, c, lastCompactedTs, ts);
}
lastCompactedTs = ts;
if (LOG.isDebugEnabled()) {
LOG.debug("State succesfully compacted for {}.", kvStateId);
}
......@@ -294,10 +299,10 @@ public class LazyDbKvState<K, V> implements KvState<K, V, DbStateBackend>, Check
}
/**
* Snapshot that stores a specific checkpoint id and state id, and also
* Snapshot that stores a specific checkpoint timestamp and state id, and also
* rolls back the database to that point upon restore. The rollback is done
* by removing all state checkpoints that have ids between the checkpoint
* and recovery id.
* by removing all state checkpoints that have timestamps between the checkpoint
* and recovery timestamp.
*
*/
private static class DbKvStateSnapshot<K, V> implements KvStateSnapshot<K, V, DbStateBackend> {
......@@ -306,10 +311,12 @@ public class LazyDbKvState<K, V> implements KvState<K, V, DbStateBackend>, Check
private final String kvStateId;
private final long checkpointTimestamp;
private final long lastCompactedTimestamp;
public DbKvStateSnapshot(String kvStateId, long checkpointTimestamp) {
public DbKvStateSnapshot(String kvStateId, long checkpointTimestamp, long lastCompactedTs) {
this.checkpointTimestamp = checkpointTimestamp;
this.kvStateId = kvStateId;
this.lastCompactedTimestamp = lastCompactedTs;
}
@Override
......@@ -346,7 +353,7 @@ public class LazyDbKvState<K, V> implements KvState<K, V, DbStateBackend>, Check
// Restore the KvState
LazyDbKvState<K, V> restored = new LazyDbKvState<K, V>(kvStateId, cleanup,
stateBackend.getConnections(), stateBackend.getConfiguration(), keySerializer, valueSerializer,
defaultValue, recoveryTimestamp);
defaultValue, recoveryTimestamp, lastCompactedTimestamp);
if (LOG.isDebugEnabled()) {
LOG.debug("KV state({},{}) restored.", kvStateId, recoveryTimestamp);
......
......@@ -141,6 +141,7 @@ public class MySqlAdapter implements DbAdapter {
return "SELECT v"
+ " FROM kvstate_" + stateId
+ " WHERE k = ?"
+ " AND timestamp <= ?"
+ " ORDER BY timestamp DESC LIMIT 1";
}
......@@ -148,6 +149,7 @@ public class MySqlAdapter implements DbAdapter {
public byte[] lookupKey(String stateId, PreparedStatement lookupStatement, byte[] key, long lookupTs)
throws SQLException {
lookupStatement.setBytes(1, key);
lookupStatement.setLong(2, lookupTs);
ResultSet res = lookupStatement.executeQuery();
......
......@@ -98,7 +98,8 @@ public class DerbyAdapter extends MySqlAdapter {
validateStateId(stateId);
return "SELECT v " + "FROM kvstate_" + stateId
+ " WHERE k = ? "
+ "ORDER BY timestamp DESC";
+ " AND timestamp <= ?"
+ " ORDER BY timestamp DESC";
}
@Override
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册