accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ktur...@apache.org
Subject [accumulo] 01/01: Merge branch '1.9'
Date Fri, 29 Jun 2018 17:18:53 GMT
This is an automated email from the ASF dual-hosted git repository.

kturner pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git

commit 13115207ff721f881cf1f67fca3f2ef6e79002f5
Merge: 99851cd 05e49e1
Author: Keith Turner <kturner@apache.org>
AuthorDate: Fri Jun 29 13:17:04 2018 -0400

    Merge branch '1.9'

 .../org/apache/accumulo/tserver/TabletServer.java  |  85 +++++++++++++--
 .../accumulo/tserver/log/SortedLogRecovery.java    |  60 +++++------
 .../org/apache/accumulo/tserver/tablet/Tablet.java |  21 +++-
 .../accumulo/tserver/WalRemovalOrderTest.java      | 115 +++++++++++++++++++++
 .../tserver/log/SortedLogRecoveryTest.java         |  48 ++++-----
 5 files changed, 253 insertions(+), 76 deletions(-)

diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
index 6442991,642064f..f9da274
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
@@@ -152,9 -153,14 +152,9 @@@ public class SortedLogRecovery 
        return next;
      }
  
 -    @Override
 -    public void remove() {
 -      throw new UnsupportedOperationException("remove");
 -    }
 -
    }
  
-   private long findLastStartToFinish(List<Path> recoveryLogs, Set<String> tabletFiles,
int tabletId)
+   private long findRecoverySeq(List<Path> recoveryLogs, Set<String> tabletFiles,
int tabletId)
        throws IOException {
      HashSet<String> suffixes = new HashSet<>();
      for (String path : tabletFiles)
diff --cc server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java
index fa178e3,53d6851..2bb0e47
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/SortedLogRecoveryTest.java
@@@ -397,8 -394,8 +397,8 @@@ public class SortedLogRecoveryTest 
          createKeyValue(DEFINE_TABLET, 1, 1, extent),
          createKeyValue(COMPACTION_START, 3, 1, "/t1/f1"), createKeyValue(MUTATION, 2, 1,
ignored),
          createKeyValue(MUTATION, 4, 1, m),};
 -    KeyValue entries2[] = new KeyValue[] {createKeyValue(OPEN, 5, -1, "1"),
 +    KeyValue entries2[] = {createKeyValue(OPEN, 5, -1, "1"),
-         createKeyValue(DEFINE_TABLET, 6, 1, extent), createKeyValue(COMPACTION_FINISH, 7,
1, null),
+         createKeyValue(DEFINE_TABLET, 4, 1, extent), createKeyValue(COMPACTION_FINISH, 4,
1, null),
          createKeyValue(MUTATION, 8, 1, m2),};
      Map<String,KeyValue[]> logs = new TreeMap<>();
      logs.put("entries", entries);
@@@ -420,11 -417,11 +420,11 @@@
      m.put(cf, cq, value);
      Mutation m2 = new ServerMutation(new Text("row2"));
      m2.put(cf, cq, new Value("123".getBytes()));
 -    KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"),
 +    KeyValue entries[] = {createKeyValue(OPEN, 0, -1, "1"),
          createKeyValue(DEFINE_TABLET, 1, 1, extent), createKeyValue(COMPACTION_FINISH, 2,
1, null),
          createKeyValue(COMPACTION_START, 4, 1, "/t1/f1"),
-         createKeyValue(COMPACTION_FINISH, 6, 1, null), createKeyValue(MUTATION, 3, 1, ignored),
-         createKeyValue(MUTATION, 5, 1, m), createKeyValue(MUTATION, 7, 1, m2),};
+         createKeyValue(COMPACTION_FINISH, 5, 1, null), createKeyValue(MUTATION, 3, 1, ignored),
+         createKeyValue(MUTATION, 5, 1, m), createKeyValue(MUTATION, 5, 1, m2),};
      Map<String,KeyValue[]> logs = new TreeMap<>();
      logs.put("entries", entries);
      // Recover
@@@ -450,11 -447,11 +450,11 @@@
          createKeyValue(DEFINE_TABLET, 1, 1, extent),
          createKeyValue(COMPACTION_START, 3, 1, "/t1/f1"), createKeyValue(MUTATION, 2, 1,
ignored),
          createKeyValue(MUTATION, 4, 1, m),};
 -    KeyValue entries2[] = new KeyValue[] {createKeyValue(OPEN, 5, -1, "1"),
 +    KeyValue entries2[] = {createKeyValue(OPEN, 5, -1, "1"),
-         createKeyValue(DEFINE_TABLET, 6, 1, extent), createKeyValue(MUTATION, 7, 1, m2),};
+         createKeyValue(DEFINE_TABLET, 4, 1, extent), createKeyValue(MUTATION, 4, 1, m2),};
 -    KeyValue entries3[] = new KeyValue[] {createKeyValue(OPEN, 8, -1, "1"),
 +    KeyValue entries3[] = {createKeyValue(OPEN, 8, -1, "1"),
-         createKeyValue(DEFINE_TABLET, 9, 1, extent), createKeyValue(COMPACTION_FINISH, 10,
1, null),
-         createKeyValue(MUTATION, 11, 1, m3),};
+         createKeyValue(DEFINE_TABLET, 4, 1, extent), createKeyValue(COMPACTION_FINISH, 4,
1, null),
+         createKeyValue(MUTATION, 4, 1, m3),};
      Map<String,KeyValue[]> logs = new TreeMap<>();
      logs.put("entries", entries);
      logs.put("entries2", entries2);
@@@ -499,11 -496,11 +499,11 @@@
      m2.put(cf, cq, value);
      Mutation m3 = new ServerMutation(new Text("row3"));
      m3.put(cf, cq, value);
 -    KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"),
 +    KeyValue entries[] = {createKeyValue(OPEN, 0, -1, "1"),
          createKeyValue(DEFINE_TABLET, 1, 1, extent),
-         createKeyValue(COMPACTION_START, 2, 1, "/t1/f1"),
-         createKeyValue(COMPACTION_FINISH, 4, 1, null), createKeyValue(MUTATION, 3, 1, m),};
+         createKeyValue(COMPACTION_START, 3, 1, "/t1/f1"),
+         createKeyValue(COMPACTION_FINISH, 4, 1, null), createKeyValue(MUTATION, 4, 1, m),};
 -    KeyValue entries2[] = new KeyValue[] {createKeyValue(OPEN, 5, -1, "1"),
 +    KeyValue entries2[] = {createKeyValue(OPEN, 5, -1, "1"),
          createKeyValue(DEFINE_TABLET, 6, 1, extent),
          createKeyValue(COMPACTION_START, 8, 1, "/t1/f1"), createKeyValue(MUTATION, 7, 1,
m2),
          createKeyValue(MUTATION, 9, 1, m3),};
@@@ -628,10 -618,10 +628,10 @@@
      Mutation m = new ServerMutation(new Text("row1"));
      m.put("foo", "bar", "v1");
  
 -    KeyValue entries[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"),
 +    KeyValue entries[] = {createKeyValue(OPEN, 0, -1, "1"),
          createKeyValue(DEFINE_TABLET, 1, 1, extent), createKeyValue(DEFINE_TABLET, 1, 2,
extent),
-         createKeyValue(MUTATION, 2, 2, ignored), createKeyValue(COMPACTION_START, 3, 2,
"/t1/f1"),
-         createKeyValue(MUTATION, 4, 2, m), createKeyValue(COMPACTION_FINISH, 6, 2, null),};
+         createKeyValue(MUTATION, 2, 2, ignored), createKeyValue(COMPACTION_START, 5, 2,
"/t1/f1"),
+         createKeyValue(MUTATION, 6, 2, m), createKeyValue(COMPACTION_FINISH, 6, 2, null),};
  
      Arrays.sort(entries);
  
@@@ -737,10 -727,10 +737,10 @@@
      Mutation m4 = new ServerMutation(new Text("s"));
      m4.put("f1", "q2", "v4");
  
 -    KeyValue entries1[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"),
 +    KeyValue entries1[] = {createKeyValue(OPEN, 0, -1, "1"),
          createKeyValue(DEFINE_TABLET, 7, 10, e1), createKeyValue(DEFINE_TABLET, 5, 11, e2),
          createKeyValue(MUTATION, 8, 10, m1), createKeyValue(COMPACTION_START, 9, 10, "/t/f1"),
-         createKeyValue(MUTATION, 10, 10, m2), createKeyValue(COMPACTION_FINISH, 11, 10,
null),
+         createKeyValue(MUTATION, 10, 10, m2), createKeyValue(COMPACTION_FINISH, 10, 10,
null),
          createKeyValue(MUTATION, 6, 11, m3), createKeyValue(COMPACTION_START, 7, 11, "/t/f2"),
          createKeyValue(MUTATION, 8, 11, m4)};
  
@@@ -758,8 -748,8 +758,8 @@@
      Assert.assertEquals(m3, mutations2.get(0));
      Assert.assertEquals(m4, mutations2.get(1));
  
 -    KeyValue entries2[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"),
 +    KeyValue entries2[] = {createKeyValue(OPEN, 0, -1, "1"),
-         createKeyValue(DEFINE_TABLET, 9, 11, e2), createKeyValue(COMPACTION_FINISH, 10,
11, null)};
+         createKeyValue(DEFINE_TABLET, 9, 11, e2), createKeyValue(COMPACTION_FINISH, 8, 11,
null)};
      Arrays.sort(entries2);
      logs.put("entries2", entries2);
  
@@@ -836,12 -826,9 +836,9 @@@
      // The presence of only a compaction finish event indicates the write ahead logs are
incomplete
      // in some way. This should cause an exception.
  
 -    KeyValue entries1[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"),
 +    KeyValue entries1[] = {createKeyValue(OPEN, 0, -1, "1"),
-         createKeyValue(DEFINE_TABLET, 100, 10, extent), createKeyValue(MUTATION, 100, 10,
m1),
-         createKeyValue(COMPACTION_FINISH, 102, 10, null), createKeyValue(MUTATION, 105,
10, m2)};
+         createKeyValue(DEFINE_TABLET, 100, 10, extent),
+         createKeyValue(COMPACTION_FINISH, 102, 10, null), createKeyValue(MUTATION, 102,
10, m1)};
  
      Arrays.sort(entries1);
  
@@@ -891,11 -877,11 +887,11 @@@
  
      // Duplicate consecutive compaction finish events should not cause an exception.
  
 -    KeyValue entries1[] = new KeyValue[] {createKeyValue(OPEN, 0, -1, "1"),
 +    KeyValue entries1[] = {createKeyValue(OPEN, 0, -1, "1"),
          createKeyValue(DEFINE_TABLET, 100, 10, extent), createKeyValue(MUTATION, 100, 10,
m1),
          createKeyValue(COMPACTION_START, 102, 10, "/t/f1"),
-         createKeyValue(COMPACTION_FINISH, 104, 10, null),
-         createKeyValue(COMPACTION_FINISH, 104, 10, null), createKeyValue(MUTATION, 103,
10, m2)};
+         createKeyValue(COMPACTION_FINISH, 103, 10, null),
+         createKeyValue(COMPACTION_FINISH, 103, 10, null), createKeyValue(MUTATION, 103,
10, m2)};
  
      Arrays.sort(entries1);
  


Mime
View raw message