You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2018/01/17 19:14:22 UTC

[GitHub] keith-turner closed pull request #360: ACCUMULO-4780 Add overflow check to seq num in CommitSession

keith-turner closed pull request #360: ACCUMULO-4780 Add overflow check to seq num in CommitSession
URL: https://github.com/apache/accumulo/pull/360
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java b/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java
index 7709a98e41..95a969a447 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java
@@ -25,7 +25,6 @@
 import java.io.File;
 import java.io.FileNotFoundException;
 
-import com.google.common.collect.Sets;
 import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.commons.io.FileUtils;
@@ -36,6 +35,8 @@
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.collect.Sets;
+
 public class AccumuloTest {
   private FileSystem fs;
   private Path path;
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
index 21734f9937..6a65097927 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletMutations.java
@@ -23,11 +23,11 @@
 
 public class TabletMutations {
   private final int tid;
-  private final int seq;
+  private final long seq;
   private final List<Mutation> mutations;
   private final Durability durability;
 
-  public TabletMutations(int tid, int seq, List<Mutation> mutations, Durability durability) {
+  public TabletMutations(int tid, long seq, List<Mutation> mutations, Durability durability) {
     this.tid = tid;
     this.seq = seq;
     this.mutations = mutations;
@@ -42,7 +42,7 @@ public int getTid() {
     return tid;
   }
 
-  public int getSeq() {
+  public long getSeq() {
     return seq;
   }
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 6c585d0ef9..10d7cad436 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -2998,13 +2998,13 @@ private Durability getMincEventDurability(KeyExtent extent) {
     return durability;
   }
 
-  public void minorCompactionFinished(CommitSession tablet, String newDatafile, int walogSeq) throws IOException {
+  public void minorCompactionFinished(CommitSession tablet, String newDatafile, long walogSeq) throws IOException {
     Durability durability = getMincEventDurability(tablet.getExtent());
     totalMinorCompactions.incrementAndGet();
     logger.minorCompactionFinished(tablet, newDatafile, walogSeq, durability);
   }
 
-  public void minorCompactionStarted(CommitSession tablet, int lastUpdateSequence, String newMapfileLocation) throws IOException {
+  public void minorCompactionStarted(CommitSession tablet, long lastUpdateSequence, String newMapfileLocation) throws IOException {
     Durability durability = getMincEventDurability(tablet.getExtent());
     logger.minorCompactionStarted(tablet, lastUpdateSequence, newMapfileLocation, durability);
   }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index c6a2593ac5..8fd2b7a0cf 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -518,7 +518,7 @@ public void close() throws IOException {
       }
   }
 
-  public synchronized void defineTablet(int seq, int tid, KeyExtent tablet) throws IOException {
+  public synchronized void defineTablet(long seq, int tid, KeyExtent tablet) throws IOException {
     // write this log to the METADATA table
     final LogFileKey key = new LogFileKey();
     key.event = DEFINE_TABLET;
@@ -539,7 +539,7 @@ private synchronized void write(LogFileKey key, LogFileValue value) throws IOExc
     encryptingLogFile.flush();
   }
 
-  public LoggerOperation log(int seq, int tid, Mutation mutation, Durability durability) throws IOException {
+  public LoggerOperation log(long seq, int tid, Mutation mutation, Durability durability) throws IOException {
     return logManyTablets(Collections.singletonList(new TabletMutations(tid, seq, Collections.singletonList(mutation), durability)));
   }
 
@@ -601,7 +601,7 @@ static Durability chooseDurabilityForGroupCommit(List<TabletMutations> mutations
     return result;
   }
 
-  public LoggerOperation minorCompactionFinished(int seq, int tid, String fqfn, Durability durability) throws IOException {
+  public LoggerOperation minorCompactionFinished(long seq, int tid, String fqfn, Durability durability) throws IOException {
     LogFileKey key = new LogFileKey();
     key.event = COMPACTION_FINISH;
     key.seq = seq;
@@ -609,7 +609,7 @@ public LoggerOperation minorCompactionFinished(int seq, int tid, String fqfn, Du
     return logFileData(Collections.singletonList(new Pair<>(key, EMPTY)), durability);
   }
 
-  public LoggerOperation minorCompactionStarted(int seq, int tid, String fqfn, Durability durability) throws IOException {
+  public LoggerOperation minorCompactionStarted(long seq, int tid, String fqfn, Durability durability) throws IOException {
     LogFileKey key = new LogFileKey();
     key.event = COMPACTION_START;
     key.seq = seq;
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 1cde8a67f8..3aa0264e3d 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -395,7 +395,7 @@ public LoggerOperation write(DfsLogger logger, int ignored) throws Exception {
     });
   }
 
-  public int log(final CommitSession commitSession, final int tabletSeq, final Mutation m, final Durability durability) throws IOException {
+  public int log(final CommitSession commitSession, final long tabletSeq, final Mutation m, final Durability durability) throws IOException {
     if (durability == Durability.NONE) {
       return -1;
     }
@@ -446,7 +446,7 @@ public LoggerOperation write(DfsLogger logger, int ignored) throws Exception {
     return seq;
   }
 
-  public void minorCompactionFinished(final CommitSession commitSession, final String fullyQualifiedFileName, final int walogSeq, final Durability durability)
+  public void minorCompactionFinished(final CommitSession commitSession, final String fullyQualifiedFileName, final long walogSeq, final Durability durability)
       throws IOException {
 
     long t1 = System.currentTimeMillis();
@@ -464,7 +464,7 @@ public LoggerOperation write(DfsLogger logger, int ignored) throws Exception {
     log.debug(" wrote MinC finish  {}: writeTime:{}ms  durability:{}", seq, (t2 - t1), durability);
   }
 
-  public int minorCompactionStarted(final CommitSession commitSession, final int seq, final String fullyQualifiedFileName, final Durability durability)
+  public long minorCompactionStarted(final CommitSession commitSession, final long seq, final String fullyQualifiedFileName, final Durability durability)
       throws IOException {
     write(commitSession, false, new Writer() {
       @Override
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
index d908f1d8b9..c62c8d5c84 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
@@ -30,21 +30,21 @@
 
   private static final Logger log = LoggerFactory.getLogger(CommitSession.class);
 
-  private final int seq;
+  private final long seq;
   private final InMemoryMap memTable;
   private final TabletCommitter committer;
 
   private int commitsInProgress;
   private long maxCommittedTime = Long.MIN_VALUE;
 
-  CommitSession(TabletCommitter committer, int seq, InMemoryMap imm) {
+  CommitSession(TabletCommitter committer, long seq, InMemoryMap imm) {
     this.seq = seq;
     this.memTable = imm;
     this.committer = committer;
     commitsInProgress = 0;
   }
 
-  public int getWALogSeq() {
+  public long getWALogSeq() {
     return seq;
   }
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletMemory.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletMemory.java
index 86f9543b2c..0278e2fefe 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletMemory.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletMemory.java
@@ -35,7 +35,7 @@
   private InMemoryMap memTable;
   private InMemoryMap otherMemTable;
   private InMemoryMap deletingMemTable;
-  private int nextSeq = 1;
+  private long nextSeq = 1L;
   private CommitSession commitSession;
 
   TabletMemory(TabletCommitter tablet) {


 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services