SAMZA-2578: Excessive trimming during transactional state restore (#1413)

diff --git a/samza-api/src/main/java/org/apache/samza/system/BoundedSSPIterator.java b/samza-api/src/main/java/org/apache/samza/system/BoundedSSPIterator.java
new file mode 100644
index 0000000..b92d6cf
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/BoundedSSPIterator.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.system;
+
+import com.google.common.collect.ImmutableSet;
+import java.util.ArrayDeque;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.Set;
+import org.apache.samza.SamzaException;
+
+/**
+ * Iterates over messages in the provided {@link SystemStreamPartition} using the provided
+ * {@link SystemConsumer} until all messages with offsets up to and including the {@code endOffset} have been consumed.
+ * If {@code endOffset} is null, the iterator will return all messages until caught up to head.
+ */
+public class BoundedSSPIterator implements Iterator<IncomingMessageEnvelope> {
+
+  protected final SystemAdmin admin;
+
+  private final SystemConsumer systemConsumer;
+  private final String endOffset;
+  private final Set<SystemStreamPartition> fetchSet;
+
+  private Queue<IncomingMessageEnvelope> peeks;
+
+  public BoundedSSPIterator(SystemConsumer systemConsumer,
+      SystemStreamPartition systemStreamPartition, String endOffset, SystemAdmin admin) {
+    this.systemConsumer = systemConsumer;
+    this.endOffset = endOffset;
+    this.admin = admin;
+    this.fetchSet = ImmutableSet.of(systemStreamPartition);
+    this.peeks = new ArrayDeque<>();
+  }
+
+  public boolean hasNext() {
+    refresh();
+
+    return peeks.size() > 0 && (endOffset == null || admin.offsetComparator(peeks.peek().getOffset(), endOffset) <= 0);
+  }
+
+  public IncomingMessageEnvelope next() {
+    refresh();
+
+    if (peeks.size() == 0 || (endOffset != null && admin.offsetComparator(peeks.peek().getOffset(), endOffset) > 0)) {
+      throw new NoSuchElementException();
+    }
+
+    return peeks.poll();
+  }
+
+  private void refresh() {
+    if (peeks.size() == 0) {
+      try {
+        Map<SystemStreamPartition, List<IncomingMessageEnvelope>> envelopes = systemConsumer.poll(fetchSet, SystemConsumer.BLOCK_ON_OUTSTANDING_MESSAGES);
+
+        for (List<IncomingMessageEnvelope> systemStreamPartitionEnvelopes : envelopes.values()) {
+          peeks.addAll(systemStreamPartitionEnvelopes);
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new SamzaException(e);
+      }
+    }
+  }
+}
diff --git a/samza-api/src/main/java/org/apache/samza/system/ChangelogSSPIterator.java b/samza-api/src/main/java/org/apache/samza/system/ChangelogSSPIterator.java
index 892eba8..ea44b9d 100644
--- a/samza-api/src/main/java/org/apache/samza/system/ChangelogSSPIterator.java
+++ b/samza-api/src/main/java/org/apache/samza/system/ChangelogSSPIterator.java
@@ -19,73 +19,51 @@
 
 package org.apache.samza.system;
 
-import java.util.ArrayDeque;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Queue;
-import java.util.Set;
-import org.apache.samza.SamzaException;
-
 /**
  * Iterates over messages in the provided changelog {@link SystemStreamPartition} using the provided
  * {@link SystemConsumer} until all messages have been consumed.
  *
  * The iterator has a {@link Mode} that depends on its position in the changelog SSP. If trim mode
  * is enabled, the mode switches to {@code TRIM} if the current message offset is greater than the
- * provided {@code endOffset}, or if {@code endOffset} is null.
+ * provided {@code restoreOffset}, or if {@code restoreOffset} is null.
  *
  * The iterator mode is used during transactional state restore to determine which changelog SSP entries
  * should be restored and which ones need to be reverted / trimmed from the changelog topic.
  */
-public class ChangelogSSPIterator {
+public class ChangelogSSPIterator extends BoundedSSPIterator {
   public enum Mode {
     RESTORE,
     TRIM
   }
 
-  private final SystemConsumer systemConsumer;
-  private final String endOffset;
-  private final SystemAdmin admin;
-  private final Set<SystemStreamPartition> fetchSet;
+  private final String restoreOffset;
   private final boolean trimEnabled;
-  private Queue<IncomingMessageEnvelope> peeks;
   private Mode mode = Mode.RESTORE;
 
-  // endOffset is inclusive when restoring. endOffset == null means trim from staring offset to head.
-  public ChangelogSSPIterator(SystemConsumer systemConsumer,
-      SystemStreamPartition systemStreamPartition, String endOffset, SystemAdmin admin, boolean trimEnabled) {
-    this.systemConsumer = systemConsumer;
-    this.endOffset = endOffset;
+  public ChangelogSSPIterator(SystemConsumer systemConsumer, SystemStreamPartition systemStreamPartition,
+      String restoreOffset, SystemAdmin admin, boolean trimEnabled) {
+    this(systemConsumer, systemStreamPartition, restoreOffset, admin, trimEnabled, null);
+  }
+
+  // restoreOffset is inclusive when restoring. restoreOffset == null means trim from starting offset to head.
+  public ChangelogSSPIterator(SystemConsumer systemConsumer, SystemStreamPartition systemStreamPartition,
+      String restoreOffset, SystemAdmin admin, boolean trimEnabled, String endOffset) {
+    super(systemConsumer, systemStreamPartition, endOffset, admin);
+
+    this.restoreOffset = restoreOffset;
     this.trimEnabled = trimEnabled;
-    if (this.trimEnabled && endOffset == null) {
+    if (this.trimEnabled && restoreOffset == null) {
       mode = Mode.TRIM;
     }
-    this.admin = admin;
-    this.fetchSet = new HashSet<>();
-    this.fetchSet.add(systemStreamPartition);
-    this.peeks = new ArrayDeque<>();
   }
 
-  public boolean hasNext() {
-    refresh();
-
-    return peeks.size() > 0;
-  }
-
+  @Override
   public IncomingMessageEnvelope next() {
-    refresh();
+    IncomingMessageEnvelope envelope = super.next();
 
-    if (peeks.size() == 0) {
-      throw new NoSuchElementException();
-    }
-
-    IncomingMessageEnvelope envelope = peeks.poll();
-
-    // if trimming changelog is enabled, then switch to trim mode if if we've consumed past the end offset
-    // (i.e., endOffset was null or current offset is > endOffset)
-    if (this.trimEnabled && (endOffset == null || admin.offsetComparator(envelope.getOffset(), endOffset) > 0)) {
+    // if trimming changelog is enabled, then switch to trim mode if if we've consumed past the restore offset
+    // (i.e., restoreOffset was null or current offset is > restoreOffset)
+    if (this.trimEnabled && (restoreOffset == null || admin.offsetComparator(envelope.getOffset(), restoreOffset) > 0)) {
       mode = Mode.TRIM;
     }
 
@@ -95,19 +73,4 @@
   public Mode getMode() {
     return this.mode;
   }
-
-  private void refresh() {
-    if (peeks.size() == 0) {
-      try {
-        Map<SystemStreamPartition, List<IncomingMessageEnvelope>> envelopes = systemConsumer.poll(fetchSet, SystemConsumer.BLOCK_ON_OUTSTANDING_MESSAGES);
-
-        for (List<IncomingMessageEnvelope> systemStreamPartitionEnvelopes : envelopes.values()) {
-          peeks.addAll(systemStreamPartitionEnvelopes);
-        }
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new SamzaException(e);
-      }
-    }
-  }
 }
diff --git a/samza-api/src/test/java/org/apache/samza/system/TestBoundedSSPIterator.java b/samza-api/src/test/java/org/apache/samza/system/TestBoundedSSPIterator.java
new file mode 100644
index 0000000..2939131
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/TestBoundedSSPIterator.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.system;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import org.apache.samza.Partition;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.stubbing.OngoingStubbing;
+
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyLong;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestBoundedSSPIterator {
+  private static final SystemStreamPartition SSP = new SystemStreamPartition("test", "test", new Partition(0));
+
+  @Test
+  public void testHasNextFalseWhenEnvelopeOutOfBounds() throws InterruptedException {
+    SystemConsumer mockConsumer = mock(SystemConsumer.class);
+    SystemAdmin mockAdmin = buildMockSystemAdmin();
+
+    int numMessages = 10;
+    long endOffset = 5;
+
+    OngoingStubbing<Map<SystemStreamPartition, List<IncomingMessageEnvelope>>> stubbing =
+        when(mockConsumer.poll(any(), anyLong()));
+    for (int i = 0; i < numMessages; i++) {
+      IncomingMessageEnvelope ime = new IncomingMessageEnvelope(SSP, String.valueOf(i), null, i);
+      stubbing = stubbing.thenReturn(ImmutableMap.of(SSP, ImmutableList.of(ime)));
+    }
+    stubbing.thenReturn(ImmutableMap.of(SSP, ImmutableList.of()));
+
+    BoundedSSPIterator iter = new BoundedSSPIterator(mockConsumer, SSP, String.valueOf(endOffset), mockAdmin);
+
+    int consumed = 0;
+    while (iter.hasNext()) {
+      iter.next();
+      consumed++;
+    }
+    Assert.assertEquals(consumed, endOffset + 1);
+
+    try {
+      iter.next();
+      Assert.fail("Iterator next call should have failed due to bound check");
+    } catch (NoSuchElementException e) {
+    }
+  }
+
+  @Test
+  public void testConsumeAllWithNullBound() throws InterruptedException {
+    SystemConsumer mockConsumer = mock(SystemConsumer.class);
+    SystemAdmin mockAdmin = buildMockSystemAdmin();
+
+    int numMessages = 10;
+    String endOffset = null;
+
+    OngoingStubbing<Map<SystemStreamPartition, List<IncomingMessageEnvelope>>> stubbing =
+        when(mockConsumer.poll(any(), anyLong()));
+    for (int i = 0; i < numMessages; i++) {
+      IncomingMessageEnvelope ime = new IncomingMessageEnvelope(SSP, String.valueOf(i), null, i);
+      stubbing = stubbing.thenReturn(ImmutableMap.of(SSP, ImmutableList.of(ime)));
+    }
+    stubbing.thenReturn(ImmutableMap.of(SSP, ImmutableList.of()));
+
+    BoundedSSPIterator iter = new BoundedSSPIterator(mockConsumer, SSP, endOffset, mockAdmin);
+
+    int consumed = 0;
+    while (iter.hasNext()) {
+      iter.next();
+      consumed++;
+    }
+
+    Assert.assertEquals(consumed, numMessages);
+
+    Assert.assertFalse(iter.hasNext());
+    try {
+      iter.next();
+      Assert.fail("Iterator next call should have failed due to bound check");
+    } catch (NoSuchElementException e) {
+    }
+  }
+
+  private SystemAdmin buildMockSystemAdmin() {
+    SystemAdmin mockAdmin = mock(SystemAdmin.class);
+    when(mockAdmin.offsetComparator(any(), any())).thenAnswer(invocation -> {
+      String offset1 = invocation.getArgumentAt(0, String.class);
+      String offset2 = invocation.getArgumentAt(1, String.class);
+
+      if (offset1 == null || offset2 == null) {
+        return -1;
+      }
+
+      return Long.valueOf(offset1).compareTo(Long.valueOf(offset2));
+    });
+    return mockAdmin;
+  }
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java
index e4633b7..4b6ac1f 100644
--- a/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java
+++ b/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java
@@ -74,6 +74,7 @@
   private final FileUtil fileUtil;
 
   private StoreActions storeActions; // available after init
+  private Map<SystemStreamPartition, SystemStreamPartitionMetadata> currentChangelogOffsets;
 
   public TransactionalStateTaskRestoreManager(
       TaskModel taskModel,
@@ -104,8 +105,7 @@
 
   @Override
   public void init(Map<SystemStreamPartition, String> checkpointedChangelogOffsets) {
-    Map<SystemStreamPartition, SystemStreamPartitionMetadata> currentChangelogOffsets =
-        getCurrentChangelogOffsets(taskModel, storeChangelogs, sspMetadataCache);
+    currentChangelogOffsets = getCurrentChangelogOffsets(taskModel, storeChangelogs, sspMetadataCache);
 
     this.storeActions = getStoreActions(taskModel, storeEngines, storeChangelogs,
         checkpointedChangelogOffsets, currentChangelogOffsets, systemAdmins, storageManagerUtil,
@@ -113,7 +113,8 @@
 
     setupStoreDirs(taskModel, storeEngines, storeActions, storageManagerUtil, fileUtil,
         loggedStoreBaseDirectory, nonLoggedStoreBaseDirectory);
-    registerStartingOffsets(taskModel, storeActions, storeChangelogs, systemAdmins, storeConsumers, currentChangelogOffsets);
+    registerStartingOffsets(taskModel, storeActions, storeChangelogs, systemAdmins, storeConsumers,
+        currentChangelogOffsets);
   }
 
   @Override
@@ -129,7 +130,8 @@
       SystemStreamPartition changelogSSP = new SystemStreamPartition(systemStream, taskModel.getChangelogPartition());
 
       ChangelogSSPIterator changelogSSPIterator =
-          new ChangelogSSPIterator(systemConsumer, changelogSSP, endOffset, systemAdmin, true);
+          new ChangelogSSPIterator(systemConsumer, changelogSSP, endOffset, systemAdmin, true,
+              currentChangelogOffsets.get(changelogSSP).getNewestOffset());
       StorageEngine taskStore = storeEngines.get(storeName);
 
       LOG.info("Restoring store: {} for task: {}", storeName, taskModel.getTaskName());
diff --git a/samza-core/src/test/java/org/apache/samza/system/MockSystemFactory.java b/samza-core/src/test/java/org/apache/samza/system/MockSystemFactory.java
index 73cae4e..6797fa5 100644
--- a/samza-core/src/test/java/org/apache/samza/system/MockSystemFactory.java
+++ b/samza-core/src/test/java/org/apache/samza/system/MockSystemFactory.java
@@ -145,7 +145,7 @@
           Map<Partition, SystemStreamMetadata.SystemStreamPartitionMetadata> partitionMetaMap =
               v.stream().<Map<Partition, SystemStreamMetadata.SystemStreamPartitionMetadata>>collect(HashMap::new,
                 (m, p) -> {
-                  m.put(p, new SystemStreamMetadata.SystemStreamPartitionMetadata("", "", ""));
+                  m.put(p, new SystemStreamMetadata.SystemStreamPartitionMetadata("0", "0", "1"));
                 }, (m1, m2) -> m1.putAll(m2));
 
           metadataMap.put(k, new SystemStreamMetadata(k, partitionMetaMap));