You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mm...@apache.org on 2022/02/08 13:13:01 UTC

[accumulo] branch main updated: Remove some generics from Fate implementation code (#2470)

This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/main by this push:
     new e3e8a9e  Remove some generics from Fate implementation code (#2470)
e3e8a9e is described below

commit e3e8a9e1164231897a10a04c9e76f7cfe908c771
Author: Mike Miller <mm...@apache.org>
AuthorDate: Tue Feb 8 08:10:11 2022 -0500

    Remove some generics from Fate implementation code (#2470)
    
    * Replace generic with ZooStore in AgeOffStore and ReadOnlyStore
    * Rename SimleStore to TestStore and make it extend ZooStore
    
    Co-authored-by: Christopher Tubbs <ct...@apache.org>
---
 .../java/org/apache/accumulo/fate/AgeOffStore.java |  8 +---
 .../org/apache/accumulo/fate/ReadOnlyStore.java    |  4 +-
 .../java/org/apache/accumulo/fate/ZooStore.java    |  5 +++
 .../org/apache/accumulo/fate/AgeOffStoreTest.java  | 51 +++++++++++-----------
 .../apache/accumulo/fate/ReadOnlyStoreTest.java    |  2 +-
 .../fate/{SimpleStore.java => TestStore.java}      | 50 +--------------------
 .../java/org/apache/accumulo/manager/Manager.java  |  7 ++-
 .../accumulo/test/fate/zookeeper/FateIT.java       |  3 +-
 8 files changed, 44 insertions(+), 86 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/fate/AgeOffStore.java b/core/src/main/java/org/apache/accumulo/fate/AgeOffStore.java
index 38a6498..42abba8 100644
--- a/core/src/main/java/org/apache/accumulo/fate/AgeOffStore.java
+++ b/core/src/main/java/org/apache/accumulo/fate/AgeOffStore.java
@@ -43,7 +43,7 @@ public class AgeOffStore<T> implements TStore<T> {
 
   private static final Logger log = LoggerFactory.getLogger(AgeOffStore.class);
 
-  private TStore<T> store;
+  private final ZooStore<T> store;
   private Map<Long,Long> candidates;
   private long ageOffTime;
   private long minTime;
@@ -112,7 +112,7 @@ public class AgeOffStore<T> implements TStore<T> {
     }
   }
 
-  public AgeOffStore(TStore<T> store, long ageOffTime, TimeSource timeSource) {
+  public AgeOffStore(ZooStore<T> store, long ageOffTime, TimeSource timeSource) {
     this.store = store;
     this.ageOffTime = ageOffTime;
     this.timeSource = timeSource;
@@ -139,10 +139,6 @@ public class AgeOffStore<T> implements TStore<T> {
     }
   }
 
-  public AgeOffStore(TStore<T> store, long ageOffTime) {
-    this(store, ageOffTime, System::currentTimeMillis);
-  }
-
   @Override
   public long create() {
     long txid = store.create();
diff --git a/core/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java b/core/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
index c53a73b..dd599c8 100644
--- a/core/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
+++ b/core/src/main/java/org/apache/accumulo/fate/ReadOnlyStore.java
@@ -32,13 +32,13 @@ import java.util.List;
  */
 public class ReadOnlyStore<T> implements ReadOnlyTStore<T> {
 
-  private final TStore<T> store;
+  private final ZooStore<T> store;
 
   /**
    * @param store
    *          may not be null
    */
-  public ReadOnlyStore(TStore<T> store) {
+  public ReadOnlyStore(ZooStore<T> store) {
     requireNonNull(store);
     this.store = store;
   }
diff --git a/core/src/main/java/org/apache/accumulo/fate/ZooStore.java b/core/src/main/java/org/apache/accumulo/fate/ZooStore.java
index 3ba26d0..9bcc132 100644
--- a/core/src/main/java/org/apache/accumulo/fate/ZooStore.java
+++ b/core/src/main/java/org/apache/accumulo/fate/ZooStore.java
@@ -111,6 +111,11 @@ public class ZooStore<T> implements TStore<T> {
     zk.putPersistentData(path, new byte[0], NodeExistsPolicy.SKIP);
   }
 
+  /**
+   * For testing only
+   */
+  ZooStore() {}
+
   @Override
   public long create() {
     while (true) {
diff --git a/core/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java b/core/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
index 7373baf..291a97d 100644
--- a/core/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
+++ b/core/src/test/java/org/apache/accumulo/fate/AgeOffStoreTest.java
@@ -25,6 +25,7 @@ import java.util.Set;
 
 import org.apache.accumulo.fate.AgeOffStore.TimeSource;
 import org.apache.accumulo.fate.ReadOnlyTStore.TStatus;
+import org.apache.zookeeper.KeeperException;
 import org.junit.Test;
 
 public class AgeOffStoreTest {
@@ -40,11 +41,11 @@ public class AgeOffStoreTest {
   }
 
   @Test
-  public void testBasic() {
+  public void testBasic() throws InterruptedException, KeeperException {
 
     TestTimeSource tts = new TestTimeSource();
-    SimpleStore<String> sstore = new SimpleStore<>();
-    AgeOffStore<String> aoStore = new AgeOffStore<>(sstore, 10, tts);
+    TestStore testStore = new TestStore();
+    AgeOffStore<String> aoStore = new AgeOffStore<>(testStore, 10, tts);
 
     aoStore.ageOff();
 
@@ -92,31 +93,31 @@ public class AgeOffStoreTest {
   }
 
   @Test
-  public void testNonEmpty() {
+  public void testNonEmpty() throws InterruptedException, KeeperException {
     // test age off when source store starts off non empty
 
     TestTimeSource tts = new TestTimeSource();
-    SimpleStore<String> sstore = new SimpleStore<>();
-    long txid1 = sstore.create();
-    sstore.reserve(txid1);
-    sstore.setStatus(txid1, TStatus.IN_PROGRESS);
-    sstore.unreserve(txid1, 0);
-
-    long txid2 = sstore.create();
-    sstore.reserve(txid2);
-    sstore.setStatus(txid2, TStatus.IN_PROGRESS);
-    sstore.setStatus(txid2, TStatus.FAILED);
-    sstore.unreserve(txid2, 0);
-
-    long txid3 = sstore.create();
-    sstore.reserve(txid3);
-    sstore.setStatus(txid3, TStatus.IN_PROGRESS);
-    sstore.setStatus(txid3, TStatus.SUCCESSFUL);
-    sstore.unreserve(txid3, 0);
-
-    Long txid4 = sstore.create();
-
-    AgeOffStore<String> aoStore = new AgeOffStore<>(sstore, 10, tts);
+    TestStore testStore = new TestStore();
+    long txid1 = testStore.create();
+    testStore.reserve(txid1);
+    testStore.setStatus(txid1, TStatus.IN_PROGRESS);
+    testStore.unreserve(txid1, 0);
+
+    long txid2 = testStore.create();
+    testStore.reserve(txid2);
+    testStore.setStatus(txid2, TStatus.IN_PROGRESS);
+    testStore.setStatus(txid2, TStatus.FAILED);
+    testStore.unreserve(txid2, 0);
+
+    long txid3 = testStore.create();
+    testStore.reserve(txid3);
+    testStore.setStatus(txid3, TStatus.IN_PROGRESS);
+    testStore.setStatus(txid3, TStatus.SUCCESSFUL);
+    testStore.unreserve(txid3, 0);
+
+    Long txid4 = testStore.create();
+
+    AgeOffStore<String> aoStore = new AgeOffStore<>(testStore, 10, tts);
 
     assertEquals(Set.of(txid1, txid2, txid3, txid4), new HashSet<>(aoStore.list()));
     assertEquals(4, new HashSet<>(aoStore.list()).size());
diff --git a/core/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java b/core/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java
index de6260e..b8255a3 100644
--- a/core/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java
+++ b/core/src/test/java/org/apache/accumulo/fate/ReadOnlyStoreTest.java
@@ -39,7 +39,7 @@ public class ReadOnlyStoreTest {
     EasyMock.expect(repo.getDescription()).andReturn("description");
     EasyMock.expect(repo.isReady(0xdeadbeefL, null)).andReturn(0x0L);
 
-    TStore<String> mock = EasyMock.createNiceMock(TStore.class);
+    ZooStore<String> mock = EasyMock.createNiceMock(ZooStore.class);
     EasyMock.expect(mock.reserve()).andReturn(0xdeadbeefL);
     mock.reserve(0xdeadbeefL);
     EasyMock.expect(mock.top(0xdeadbeefL)).andReturn(repo);
diff --git a/core/src/test/java/org/apache/accumulo/fate/SimpleStore.java b/core/src/test/java/org/apache/accumulo/fate/TestStore.java
similarity index 68%
rename from core/src/test/java/org/apache/accumulo/fate/SimpleStore.java
rename to core/src/test/java/org/apache/accumulo/fate/TestStore.java
index 0bdd017..8616d91 100644
--- a/core/src/test/java/org/apache/accumulo/fate/SimpleStore.java
+++ b/core/src/test/java/org/apache/accumulo/fate/TestStore.java
@@ -18,9 +18,7 @@
  */
 package org.apache.accumulo.fate;
 
-import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -30,7 +28,7 @@ import java.util.Set;
 /**
  * Transient in memory store for transactions.
  */
-public class SimpleStore<T> implements TStore<T> {
+public class TestStore extends ZooStore<String> {
 
   private long nextId = 1;
   private Map<Long,TStatus> statuses = new HashMap<>();
@@ -43,11 +41,6 @@ public class SimpleStore<T> implements TStore<T> {
   }
 
   @Override
-  public long reserve() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
   public void reserve(long tid) {
     if (reserved.contains(tid))
       throw new IllegalStateException(); // zoo store would wait, but do not expect test to reserve
@@ -63,21 +56,6 @@ public class SimpleStore<T> implements TStore<T> {
   }
 
   @Override
-  public Repo<T> top(long tid) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void push(long tid, Repo<T> repo) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void pop(long tid) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
   public org.apache.accumulo.fate.TStore.TStatus getStatus(long tid) {
     if (!reserved.contains(tid))
       throw new IllegalStateException();
@@ -98,22 +76,6 @@ public class SimpleStore<T> implements TStore<T> {
   }
 
   @Override
-  public org.apache.accumulo.fate.TStore.TStatus waitForStatusChange(long tid,
-      EnumSet<org.apache.accumulo.fate.TStore.TStatus> expected) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setProperty(long tid, String prop, Serializable val) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Serializable getProperty(long tid, String prop) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
   public void delete(long tid) {
     if (!reserved.contains(tid))
       throw new IllegalStateException();
@@ -125,14 +87,4 @@ public class SimpleStore<T> implements TStore<T> {
     return new ArrayList<>(statuses.keySet());
   }
 
-  @Override
-  public long timeCreated(long tid) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public List<ReadOnlyRepo<T>> getStack(long tid) {
-    throw new UnsupportedOperationException();
-  }
-
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
index 3114277..d4d43ac 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
@@ -1142,8 +1142,11 @@ public class Manager extends AbstractServer
     }
 
     try {
-      final AgeOffStore<Manager> store = new AgeOffStore<>(new org.apache.accumulo.fate.ZooStore<>(
-          getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter()), 1000 * 60 * 60 * 8);
+      final AgeOffStore<Manager> store =
+          new AgeOffStore<>(
+              new org.apache.accumulo.fate.ZooStore<>(getZooKeeperRoot() + Constants.ZFATE,
+                  context.getZooReaderWriter()),
+              TimeUnit.HOURS.toMillis(8), System::currentTimeMillis);
 
       fate = new Fate<>(this, store, TraceRepo::toLogString);
       fate.startTransactionRunners(getConfiguration());
diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java
index afafc89..2f27d87 100644
--- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java
@@ -118,7 +118,8 @@ public class FateIT {
     zk.mkdirs(ZK_ROOT + Constants.ZTABLES + "/" + TID.canonical());
 
     ZooStore<Manager> zooStore = new ZooStore<Manager>(ZK_ROOT + Constants.ZFATE, zk);
-    final AgeOffStore<Manager> store = new AgeOffStore<Manager>(zooStore, 1000 * 60 * 60 * 8);
+    final AgeOffStore<Manager> store =
+        new AgeOffStore<Manager>(zooStore, 1000 * 60 * 60 * 8, System::currentTimeMillis);
 
     Manager manager = createMock(Manager.class);
     ServerContext sctx = createMock(ServerContext.class);