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

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


The following commit(s) were added to refs/heads/master by this push:
     new b915947  Remove unused tserver code (#863)
b915947 is described below

commit b915947c0c22d9db717067b601c62829205d1505
Author: Mike Miller <mmil...@apache.org>
AuthorDate: Wed Jan 2 14:23:16 2019 -0500

    Remove unused tserver code (#863)
---
 .../org/apache/accumulo/tserver/InMemoryMap.java   | 55 --------------------
 .../org/apache/accumulo/tserver/NativeMap.java     |  6 ---
 .../org/apache/accumulo/tserver/TabletServer.java  | 10 +---
 .../tserver/TabletServerResourceManager.java       | 26 ----------
 .../tserver/compaction/MajorCompactionRequest.java |  2 +
 .../TooManyDeletesCompactionStrategy.java          |  4 --
 .../accumulo/tserver/tablet/CommitSession.java     |  4 +-
 .../accumulo/tserver/tablet/ScanOptions.java       |  8 ---
 .../org/apache/accumulo/tserver/tablet/Tablet.java | 16 +-----
 .../accumulo/tserver/tablet/TabletCommitter.java   | 60 ----------------------
 .../apache/accumulo/tserver/tablet/TabletData.java |  4 --
 .../accumulo/tserver/tablet/TabletMemory.java      | 10 +---
 12 files changed, 7 insertions(+), 198 deletions(-)

diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
index b9e2167..58cb4a1 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
@@ -23,12 +23,10 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.SortedMap;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.TimeUnit;
@@ -186,9 +184,6 @@ public class InMemoryMap {
   }
 
   private interface SimpleMap {
-    Value get(Key key);
-
-    Iterator<Entry<Key,Value>> iterator(Key startKey);
 
     int size();
 
@@ -212,16 +207,6 @@ public class InMemoryMap {
     }
 
     @Override
-    public Value get(Key key) {
-      return map.get(key);
-    }
-
-    @Override
-    public Iterator<Entry<Key,Value>> iterator(Key startKey) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
     public int size() {
       return map.size();
     }
@@ -327,16 +312,6 @@ public class InMemoryMap {
     }
 
     @Override
-    public Value get(Key key) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public Iterator<Entry<Key,Value>> iterator(Key startKey) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
     public int size() {
       int sum = 0;
       for (SimpleMap map : maps)
@@ -415,18 +390,6 @@ public class InMemoryMap {
     }
 
     @Override
-    public Value get(Key key) {
-      return map.get(key);
-    }
-
-    @Override
-    public Iterator<Entry<Key,Value>> iterator(Key startKey) {
-      Key lk = new Key(startKey);
-      SortedMap<Key,Value> tm = map.tailMap(lk);
-      return tm.entrySet().iterator();
-    }
-
-    @Override
     public int size() {
       return size.get();
     }
@@ -482,16 +445,6 @@ public class InMemoryMap {
     }
 
     @Override
-    public Value get(Key key) {
-      return nativeMap.get(key);
-    }
-
-    @Override
-    public Iterator<Entry<Key,Value>> iterator(Key startKey) {
-      return nativeMap.iterator(startKey);
-    }
-
-    @Override
     public int size() {
       return nativeMap.size();
     }
@@ -558,10 +511,6 @@ public class InMemoryMap {
     return map.getMemoryUsed();
   }
 
-  Iterator<Map.Entry<Key,Value>> iterator(Key startKey) {
-    return map.iterator(startKey);
-  }
-
   public synchronized long getNumEntries() {
     if (map == null)
       return 0;
@@ -887,8 +836,4 @@ public class InMemoryMap {
       iter.next();
     }
   }
-
-  public ServerContext getContext() {
-    return context;
-  }
 }
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/NativeMap.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/NativeMap.java
index f3224af..7d238a4 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/NativeMap.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/NativeMap.java
@@ -20,7 +20,6 @@ import java.io.File;
 import java.util.AbstractMap.SimpleImmutableEntry;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.ConcurrentModificationException;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -528,11 +527,6 @@ public class NativeMap implements 
Iterable<Map.Entry<Key,Value>> {
     return mutationCount;
   }
 
-  @VisibleForTesting
-  public void mutate(Mutation mutation, int mutationCount) {
-    mutate(Collections.singletonList(mutation), mutationCount);
-  }
-
   void mutate(List<Mutation> mutations, int mutationCount) {
     Iterator<Mutation> iter = mutations.iterator();
 
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 08c5495..3a2e615 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
@@ -332,7 +332,6 @@ public class TabletServer implements Runnable {
 
   private Thread majorCompactorThread;
 
-  private HostAndPort replicationAddress;
   private HostAndPort clientAddress;
 
   private volatile boolean serverStopRequested = false;
@@ -2833,7 +2832,7 @@ public class TabletServer implements Runnable {
 
     // Start the thrift service listening for incoming replication requests
     try {
-      replicationAddress = startReplicationService();
+      startReplicationService();
     } catch (UnknownHostException e) {
       throw new RuntimeException("Failed to start replication service", e);
     }
@@ -3118,13 +3117,6 @@ public class TabletServer implements Runnable {
     return clientAddress.getHost() + ":" + clientAddress.getPort();
   }
 
-  public String getReplicationAddressSTring() {
-    if (replicationAddress == null) {
-      return null;
-    }
-    return replicationAddress.getHost() + ":" + replicationAddress.getPort();
-  }
-
   public TServerInstance getTabletSession() {
     String address = getClientAddressString();
     if (address == null)
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
index f7dc2e4..38806f6 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
@@ -724,32 +724,6 @@ public class TabletServerResourceManager {
     }
   }
 
-  public void close() {
-    for (ExecutorService executorService : threadPools.values()) {
-      executorService.shutdown();
-    }
-
-    if (this.cacheManager != null) {
-      try {
-        this.cacheManager.stop();
-      } catch (Exception ex) {
-        log.error("Error stopping BlockCacheManager", ex);
-      }
-    }
-
-    for (Entry<String,ExecutorService> entry : threadPools.entrySet()) {
-      while (true) {
-        try {
-          if (entry.getValue().awaitTermination(60, TimeUnit.SECONDS))
-            break;
-          log.info("Waiting for thread pool {} to shutdown", entry.getKey());
-        } catch (InterruptedException e) {
-          log.warn("Interrupted waiting for executor to terminate", e);
-        }
-      }
-    }
-  }
-
   public synchronized TabletResourceManager 
createTabletResourceManager(KeyExtent extent,
       AccumuloConfiguration conf) {
     return new TabletResourceManager(extent, conf);
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
index 211fe51..16fe49a 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
@@ -51,6 +51,7 @@ import 
org.apache.accumulo.tserver.compaction.strategies.TooManyDeletesCompactio
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.cache.Cache;
 
@@ -94,6 +95,7 @@ public class MajorCompactionRequest implements Cloneable {
     this.files = mcr.files;
   }
 
+  @VisibleForTesting
   public TabletId getTabletId() {
     return new TabletIdImpl(extent);
   }
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/TooManyDeletesCompactionStrategy.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/TooManyDeletesCompactionStrategy.java
index 8e88a11..1298262 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/TooManyDeletesCompactionStrategy.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/TooManyDeletesCompactionStrategy.java
@@ -36,8 +36,6 @@ import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.tserver.compaction.CompactionPlan;
 import org.apache.accumulo.tserver.compaction.DefaultCompactionStrategy;
 import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * This compaction strategy works in concert with the {@link 
DeletesSummarizer}. Using the
@@ -84,8 +82,6 @@ public class TooManyDeletesCompactionStrategy extends 
DefaultCompactionStrategy
 
   private boolean shouldCompact = false;
 
-  Logger log = LoggerFactory.getLogger(TooManyDeletesCompactionStrategy.class);
-
   private double threshold;
 
   private boolean proceed_bns;
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 1082ade..ea4d8bc 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
@@ -31,12 +31,12 @@ public class CommitSession {
 
   private final long seq;
   private final InMemoryMap memTable;
-  private final TabletCommitter committer;
+  private final Tablet committer;
 
   private int commitsInProgress;
   private long maxCommittedTime = Long.MIN_VALUE;
 
-  CommitSession(TabletCommitter committer, long seq, InMemoryMap imm) {
+  CommitSession(Tablet committer, long seq, InMemoryMap imm) {
     this.seq = seq;
     this.memTable = imm;
     this.committer = committer;
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanOptions.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanOptions.java
index a5b3223..e1e2024 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanOptions.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanOptions.java
@@ -90,10 +90,6 @@ final class ScanOptions {
     return isolated;
   }
 
-  public SamplerConfiguration getSamplerConfiguration() {
-    return samplerConfig;
-  }
-
   public SamplerConfigurationImpl getSamplerConfigurationImpl() {
     if (samplerConfig == null)
       return null;
@@ -108,10 +104,6 @@ final class ScanOptions {
     return classLoaderContext;
   }
 
-  public void setClassLoaderContext(String context) {
-    this.classLoaderContext = context;
-  }
-
   @Override
   public String toString() {
     StringBuilder buf = new StringBuilder();
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index d128f02..de94ede 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -165,7 +165,7 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 /**
  * Provide access to a single row range in a living TabletServer.
  */
-public class Tablet implements TabletCommitter {
+public class Tablet {
   private static final Logger log = LoggerFactory.getLogger(Tablet.class);
 
   private final TabletServer tabletServer;
@@ -259,7 +259,6 @@ public class Tablet implements TabletCommitter {
 
   private final int logId;
 
-  @Override
   public int getLogId() {
     return logId;
   }
@@ -1254,7 +1253,6 @@ public class Tablet implements TabletCommitter {
     return finishPreparingMutations(time);
   }
 
-  @Override
   public synchronized void abortCommit(CommitSession commitSession) {
     if (writesInProgress <= 0) {
       throw new IllegalStateException("waitingForLogs <= 0 " + 
writesInProgress);
@@ -1270,7 +1268,6 @@ public class Tablet implements TabletCommitter {
       this.notifyAll();
   }
 
-  @Override
   public void commit(CommitSession commitSession, List<Mutation> mutations) {
 
     int totalCount = 0;
@@ -2174,7 +2171,6 @@ public class Tablet implements TabletCommitter {
     return majCStats;
   }
 
-  @Override
   public KeyExtent getExtent() {
     return extent;
   }
@@ -2553,7 +2549,6 @@ public class Tablet implements TabletCommitter {
   // clean up by calling finishUpdatingLogsUsed()
   @SuppressFBWarnings(value = "UL_UNRELEASED_LOCK",
       justification = "lock is released by caller calling 
finishedUpdatingLogsUsed method")
-  @Override
   public boolean beginUpdatingLogsUsed(InMemoryMap memTable, DfsLogger more, 
boolean mincFinish) {
 
     boolean releaseLock = true;
@@ -2634,7 +2629,6 @@ public class Tablet implements TabletCommitter {
     }
   }
 
-  @Override
   public void finishUpdatingLogsUsed() {
     logLock.unlock();
   }
@@ -2719,17 +2713,14 @@ public class Tablet implements TabletCommitter {
     }
   }
 
-  @Override
   public TableConfiguration getTableConfiguration() {
     return tableConfiguration;
   }
 
-  @Override
   public Durability getDurability() {
     return 
DurabilityImpl.fromString(getTableConfiguration().get(Property.TABLE_DURABILITY));
   }
 
-  @Override
   public void updateMemoryUsageStats(long size, long mincSize) {
     getTabletResources().updateMemoryUsageStats(this, size, mincSize);
   }
@@ -2817,11 +2808,6 @@ public class Tablet implements TabletCommitter {
       this.lastCompactID = compactionId;
   }
 
-  public void removeMajorCompactionQueuedReason(MajorCompactionReason reason) {
-    majorCompactionQueued.remove(reason);
-
-  }
-
   public void minorCompactionWaitingToStart() {
     minorCompactionState = CompactionState.WAITING_TO_START;
   }
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
deleted file mode 100644
index 5a089dc..0000000
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.accumulo.tserver.tablet;
-
-import java.util.List;
-
-import org.apache.accumulo.core.client.Durability;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.server.conf.TableConfiguration;
-import org.apache.accumulo.tserver.InMemoryMap;
-import org.apache.accumulo.tserver.log.DfsLogger;
-
-/**
- * A partial interface of Tablet to allow for testing of CommitSession without 
needing a real
- * Tablet.
- */
-public interface TabletCommitter {
-
-  void abortCommit(CommitSession commitSession);
-
-  void commit(CommitSession commitSession, List<Mutation> mutations);
-
-  /**
-   * If this method returns true, the caller must call {@link 
#finishUpdatingLogsUsed()} to clean up
-   */
-  boolean beginUpdatingLogsUsed(InMemoryMap memTable, DfsLogger copy, boolean 
mincFinish);
-
-  void finishUpdatingLogsUsed();
-
-  TableConfiguration getTableConfiguration();
-
-  /**
-   * Returns a KeyExtent object representing this tablet's key range.
-   *
-   * @return extent
-   */
-  KeyExtent getExtent();
-
-  int getLogId();
-
-  Durability getDurability();
-
-  void updateMemoryUsageStats(long estimatedSizeInBytes, long 
estimatedSizeInBytes2);
-
-}
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
index d33429b..c4de67c 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
@@ -187,10 +187,6 @@ public class TabletData {
     this.splitTime = System.currentTimeMillis();
   }
 
-  public static Logger getLog() {
-    return log;
-  }
-
   public String getTime() {
     return time;
   }
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 c812356..3e30a81 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
@@ -32,7 +32,7 @@ import org.slf4j.LoggerFactory;
 class TabletMemory implements Closeable {
   private static final Logger log = 
LoggerFactory.getLogger(TabletMemory.class);
 
-  private final TabletCommitter tablet;
+  private final Tablet tablet;
   private InMemoryMap memTable;
   private InMemoryMap otherMemTable;
   private InMemoryMap deletingMemTable;
@@ -184,16 +184,8 @@ class TabletMemory implements Closeable {
     return commitSession;
   }
 
-  public ServerContext getContext() {
-    return context;
-  }
-
   @Override
   public void close() {
     commitSession = null;
   }
-
-  public boolean isClosed() {
-    return commitSession == null;
-  }
 }

Reply via email to