You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@omid.apache.org by fp...@apache.org on 2016/06/08 18:37:53 UTC

[01/14] incubator-omid git commit: [OMID-36] Use SystemExitPanicker for killing TSO process

Repository: incubator-omid
Updated Branches:
  refs/heads/master 90cea3263 -> d0ac6890a


[OMID-36] Use SystemExitPanicker for killing TSO process

Exit the JVM when an exception occurs in the TSO processors instead
of throwing a RuntimeException. With the Runtime exception, the process
hangs without exiting, which avoids the process to be relaunched in
both HA and non-HA scenarios.

Also terminate (close) all the TSO processor in an orderly manner when
shutdown.

Finally, log info has been added for both the startup and the shutdown
phases.

Change-Id: I0800f4fae98de29cbf02502db42700b0a1cdb280


Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/08163880
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/08163880
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/08163880

Branch: refs/heads/master
Commit: 0816388011104e6b7b08cf3f4a3aa5b4c619d205
Parents: 90cea32
Author: Francisco Perez-Sorrosal <fp...@yahoo-inc.com>
Authored: Thu May 26 09:50:53 2016 -0700
Committer: Francisco Perez-Sorrosal <fp...@yahoo-inc.com>
Committed: Fri Jun 3 16:46:55 2016 -0700

----------------------------------------------------------------------
 .../apache/omid/tso/PersistenceProcessor.java   |  3 +-
 .../omid/tso/PersistenceProcessorImpl.java      | 32 +++++++++++++--
 .../org/apache/omid/tso/ReplyProcessor.java     |  4 +-
 .../org/apache/omid/tso/ReplyProcessorImpl.java | 34 ++++++++++++++--
 .../org/apache/omid/tso/RequestProcessor.java   |  3 +-
 .../apache/omid/tso/RequestProcessorImpl.java   | 42 +++++++++++++++-----
 .../org/apache/omid/tso/RetryProcessor.java     |  4 +-
 .../org/apache/omid/tso/RetryProcessorImpl.java | 35 +++++++++++++---
 .../org/apache/omid/tso/SystemExitPanicker.java | 38 ++++++++++++++++++
 .../java/org/apache/omid/tso/TSOModule.java     |  2 +-
 .../java/org/apache/omid/tso/TSOServer.java     | 33 ++++++++++++++-
 11 files changed, 201 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/08163880/tso-server/src/main/java/org/apache/omid/tso/PersistenceProcessor.java
----------------------------------------------------------------------
diff --git a/tso-server/src/main/java/org/apache/omid/tso/PersistenceProcessor.java b/tso-server/src/main/java/org/apache/omid/tso/PersistenceProcessor.java
index 07893f6..b96945d 100644
--- a/tso-server/src/main/java/org/apache/omid/tso/PersistenceProcessor.java
+++ b/tso-server/src/main/java/org/apache/omid/tso/PersistenceProcessor.java
@@ -19,9 +19,10 @@ package org.apache.omid.tso;
 
 import org.jboss.netty.channel.Channel;
 
+import java.io.Closeable;
 import java.util.concurrent.Future;
 
-interface PersistenceProcessor {
+interface PersistenceProcessor extends Closeable {
 
     void addCommitToBatch(long startTimestamp, long commitTimestamp, Channel c, MonitoringContext monCtx)
             throws Exception;

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/08163880/tso-server/src/main/java/org/apache/omid/tso/PersistenceProcessorImpl.java
----------------------------------------------------------------------
diff --git a/tso-server/src/main/java/org/apache/omid/tso/PersistenceProcessorImpl.java b/tso-server/src/main/java/org/apache/omid/tso/PersistenceProcessorImpl.java
index 5204332..7735c6b 100644
--- a/tso-server/src/main/java/org/apache/omid/tso/PersistenceProcessorImpl.java
+++ b/tso-server/src/main/java/org/apache/omid/tso/PersistenceProcessorImpl.java
@@ -23,7 +23,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.lmax.disruptor.BusySpinWaitStrategy;
 import com.lmax.disruptor.EventFactory;
 import com.lmax.disruptor.RingBuffer;
-import com.lmax.disruptor.WorkerPool;
 import com.lmax.disruptor.dsl.Disruptor;
 import org.apache.commons.pool2.ObjectPool;
 import org.apache.omid.committable.CommitTable;
@@ -41,6 +40,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
 import static com.lmax.disruptor.dsl.ProducerType.SINGLE;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.omid.metrics.MetricsUtils.name;
 import static org.apache.omid.tso.PersistenceProcessorImpl.PersistBatchEvent.EVENT_FACTORY;
 import static org.apache.omid.tso.PersistenceProcessorImpl.PersistBatchEvent.makePersistBatch;
@@ -49,6 +49,9 @@ class PersistenceProcessorImpl implements PersistenceProcessor {
 
     private static final Logger LOG = LoggerFactory.getLogger(PersistenceProcessorImpl.class);
 
+    // Disruptor-related attributes
+    private final ExecutorService disruptorExec;
+    private final Disruptor<PersistBatchEvent> disruptor;
     private final RingBuffer<PersistBatchEvent> persistRing;
 
     private final ObjectPool<Batch> batchPool;
@@ -78,10 +81,10 @@ class PersistenceProcessorImpl implements PersistenceProcessor {
         // ------------------------------------------------------------------------------------------------------------
 
         ThreadFactoryBuilder threadFactory = new ThreadFactoryBuilder().setNameFormat("persist-%d");
-        ExecutorService requestExec = Executors.newFixedThreadPool(config.getNumConcurrentCTWriters(), threadFactory.build());
+        this.disruptorExec = Executors.newFixedThreadPool(config.getNumConcurrentCTWriters(), threadFactory.build());
 
-        Disruptor<PersistBatchEvent> disruptor = new Disruptor<>(EVENT_FACTORY, 1 << 20, requestExec , SINGLE, new BusySpinWaitStrategy());
-        disruptor.handleExceptionsWith(new FatalExceptionHandler(panicker));
+        this.disruptor = new Disruptor<>(EVENT_FACTORY, 1 << 20, disruptorExec , SINGLE, new BusySpinWaitStrategy());
+        disruptor.handleExceptionsWith(new FatalExceptionHandler(panicker)); // This must be before handleEventsWith()
         disruptor.handleEventsWithWorkerPool(handlers);
         this.persistRing = disruptor.start();
 
@@ -101,6 +104,8 @@ class PersistenceProcessorImpl implements PersistenceProcessor {
         // Metrics config
         this.lwmWriteTimer = metrics.timer(name("tso", "lwmWriter", "latency"));
 
+        LOG.info("PersistentProcessor initialized");
+
     }
 
     @Override
@@ -176,6 +181,25 @@ class PersistenceProcessorImpl implements PersistenceProcessor {
 
     }
 
+    @Override
+    public void close() throws IOException {
+
+        LOG.info("Terminating Persistence Processor...");
+        disruptor.halt();
+        disruptor.shutdown();
+        LOG.info("\tPersistence Processor Disruptor shutdown");
+        disruptorExec.shutdownNow();
+        try {
+            disruptorExec.awaitTermination(3, SECONDS);
+            LOG.info("\tPersistence Processor Disruptor executor shutdown");
+        } catch (InterruptedException e) {
+            LOG.error("Interrupted whilst finishing Persistence Processor Disruptor executor");
+            Thread.currentThread().interrupt();
+        }
+        LOG.info("Persistence Processor terminated");
+
+    }
+
     final static class PersistBatchEvent {
 
         private long batchSequence;

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/08163880/tso-server/src/main/java/org/apache/omid/tso/ReplyProcessor.java
----------------------------------------------------------------------
diff --git a/tso-server/src/main/java/org/apache/omid/tso/ReplyProcessor.java b/tso-server/src/main/java/org/apache/omid/tso/ReplyProcessor.java
index 820630b..f196c42 100644
--- a/tso-server/src/main/java/org/apache/omid/tso/ReplyProcessor.java
+++ b/tso-server/src/main/java/org/apache/omid/tso/ReplyProcessor.java
@@ -19,7 +19,9 @@ package org.apache.omid.tso;
 
 import org.jboss.netty.channel.Channel;
 
-interface ReplyProcessor {
+import java.io.Closeable;
+
+interface ReplyProcessor extends Closeable {
 
     /**
      * The each reply to a transactional operation for a client is contained in a batch. The batch must be ordered

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/08163880/tso-server/src/main/java/org/apache/omid/tso/ReplyProcessorImpl.java
----------------------------------------------------------------------
diff --git a/tso-server/src/main/java/org/apache/omid/tso/ReplyProcessorImpl.java b/tso-server/src/main/java/org/apache/omid/tso/ReplyProcessorImpl.java
index d5e4c00..276f803 100644
--- a/tso-server/src/main/java/org/apache/omid/tso/ReplyProcessorImpl.java
+++ b/tso-server/src/main/java/org/apache/omid/tso/ReplyProcessorImpl.java
@@ -41,15 +41,20 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import static com.codahale.metrics.MetricRegistry.name;
 import static com.lmax.disruptor.dsl.ProducerType.MULTI;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.omid.tso.ReplyProcessorImpl.ReplyBatchEvent.EVENT_FACTORY;
 
 class ReplyProcessorImpl implements EventHandler<ReplyProcessorImpl.ReplyBatchEvent>, ReplyProcessor {
 
     private static final Logger LOG = LoggerFactory.getLogger(ReplyProcessorImpl.class);
 
-    private final ObjectPool<Batch> batchPool;
-
+    // Disruptor-related attributes
+    private final ExecutorService disruptorExec;
+    private final Disruptor<ReplyBatchEvent> disruptor;
     private final RingBuffer<ReplyBatchEvent> replyRing;
 
+    private final ObjectPool<Batch> batchPool;
+
     @VisibleForTesting
     AtomicLong nextIDToHandle = new AtomicLong();
 
@@ -69,9 +74,9 @@ class ReplyProcessorImpl implements EventHandler<ReplyProcessorImpl.ReplyBatchEv
         // ------------------------------------------------------------------------------------------------------------
 
         ThreadFactoryBuilder threadFactory = new ThreadFactoryBuilder().setNameFormat("reply-%d");
-        ExecutorService requestExec = Executors.newSingleThreadExecutor(threadFactory.build());
+        this.disruptorExec = Executors.newSingleThreadExecutor(threadFactory.build());
 
-        Disruptor<ReplyProcessorImpl.ReplyBatchEvent> disruptor = new Disruptor<>(ReplyBatchEvent.EVENT_FACTORY, 1 << 12, requestExec, MULTI, new BusySpinWaitStrategy());
+        this.disruptor = new Disruptor<>(EVENT_FACTORY, 1 << 12, disruptorExec, MULTI, new BusySpinWaitStrategy());
         disruptor.handleExceptionsWith(new FatalExceptionHandler(panicker));
         disruptor.handleEventsWith(this);
         this.replyRing = disruptor.start();
@@ -93,6 +98,8 @@ class ReplyProcessorImpl implements EventHandler<ReplyProcessorImpl.ReplyBatchEv
         this.commitMeter = metrics.meter(name("tso", "commits"));
         this.timestampMeter = metrics.meter(name("tso", "timestampAllocation"));
 
+        LOG.info("ReplyProcessor initialized");
+
     }
 
     @VisibleForTesting
@@ -207,6 +214,25 @@ class ReplyProcessorImpl implements EventHandler<ReplyProcessorImpl.ReplyBatchEv
 
     }
 
+    @Override
+    public void close() {
+
+        LOG.info("Terminating Reply Processor...");
+        disruptor.halt();
+        disruptor.shutdown();
+        LOG.info("\tReply Processor Disruptor shutdown");
+        disruptorExec.shutdownNow();
+        try {
+            disruptorExec.awaitTermination(3, SECONDS);
+            LOG.info("\tReply Processor Disruptor executor shutdown");
+        } catch (InterruptedException e) {
+            LOG.error("Interrupted whilst finishing Reply Processor Disruptor executor");
+            Thread.currentThread().interrupt();
+        }
+        LOG.info("Reply Processor terminated");
+
+    }
+
     final static class ReplyBatchEvent {
 
         private Batch batch;

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/08163880/tso-server/src/main/java/org/apache/omid/tso/RequestProcessor.java
----------------------------------------------------------------------
diff --git a/tso-server/src/main/java/org/apache/omid/tso/RequestProcessor.java b/tso-server/src/main/java/org/apache/omid/tso/RequestProcessor.java
index 98edec5..8ab6c9f 100644
--- a/tso-server/src/main/java/org/apache/omid/tso/RequestProcessor.java
+++ b/tso-server/src/main/java/org/apache/omid/tso/RequestProcessor.java
@@ -19,10 +19,11 @@ package org.apache.omid.tso;
 
 import org.jboss.netty.channel.Channel;
 
+import java.io.Closeable;
 import java.util.Collection;
 
 // NOTE: public is required explicitly in the interface definition for Guice injection
-public interface RequestProcessor extends TSOStateManager.StateObserver {
+public interface RequestProcessor extends TSOStateManager.StateObserver, Closeable {
 
     void timestampRequest(Channel c, MonitoringContext monCtx);
 

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/08163880/tso-server/src/main/java/org/apache/omid/tso/RequestProcessorImpl.java
----------------------------------------------------------------------
diff --git a/tso-server/src/main/java/org/apache/omid/tso/RequestProcessorImpl.java b/tso-server/src/main/java/org/apache/omid/tso/RequestProcessorImpl.java
index b945eaf..808d43d 100644
--- a/tso-server/src/main/java/org/apache/omid/tso/RequestProcessorImpl.java
+++ b/tso-server/src/main/java/org/apache/omid/tso/RequestProcessorImpl.java
@@ -18,11 +18,9 @@
 package org.apache.omid.tso;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.lmax.disruptor.BatchEventProcessor;
 import com.lmax.disruptor.EventFactory;
 import com.lmax.disruptor.EventHandler;
 import com.lmax.disruptor.RingBuffer;
-import com.lmax.disruptor.SequenceBarrier;
 import com.lmax.disruptor.TimeoutBlockingWaitStrategy;
 import com.lmax.disruptor.TimeoutHandler;
 import com.lmax.disruptor.dsl.Disruptor;
@@ -40,21 +38,26 @@ import java.util.NoSuchElementException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
 
 import static com.lmax.disruptor.dsl.ProducerType.MULTI;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.omid.tso.RequestProcessorImpl.RequestEvent.EVENT_FACTORY;
 
 class RequestProcessorImpl implements EventHandler<RequestProcessorImpl.RequestEvent>, RequestProcessor, TimeoutHandler {
 
     private static final Logger LOG = LoggerFactory.getLogger(RequestProcessorImpl.class);
 
+    // Disruptor-related attributes
+    private final ExecutorService disruptorExec;
+    private final Disruptor<RequestEvent> disruptor;
+    private final RingBuffer<RequestEvent> requestRing;
+
     private final TimestampOracle timestampOracle;
     private final CommitHashMap hashmap;
     private final MetricsRegistry metrics;
     private final PersistenceProcessor persistProc;
-    private final RingBuffer<RequestEvent> requestRing;
+
     private long lowWatermark = -1L;
 
     @Inject
@@ -72,10 +75,10 @@ class RequestProcessorImpl implements EventHandler<RequestProcessorImpl.RequestE
         TimeoutBlockingWaitStrategy timeoutStrategy = new TimeoutBlockingWaitStrategy(config.getBatchPersistTimeoutInMs(), MILLISECONDS);
 
         ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat("request-%d").build();
-        ExecutorService requestExec = Executors.newSingleThreadExecutor(threadFactory);
+        this.disruptorExec = Executors.newSingleThreadExecutor(threadFactory);
 
-        Disruptor<RequestEvent> disruptor = new Disruptor<>(EVENT_FACTORY, 1 << 12, requestExec, MULTI, timeoutStrategy);
-        disruptor.handleExceptionsWith(new FatalExceptionHandler(panicker)); // This should be before the event handler
+        this.disruptor = new Disruptor<>(EVENT_FACTORY, 1 << 12, disruptorExec, MULTI, timeoutStrategy);
+        disruptor.handleExceptionsWith(new FatalExceptionHandler(panicker)); // This must be before handleEventsWith()
         disruptor.handleEventsWith(this);
         this.requestRing = disruptor.start();
 
@@ -88,6 +91,8 @@ class RequestProcessorImpl implements EventHandler<RequestProcessorImpl.RequestE
         this.timestampOracle = timestampOracle;
         this.hashmap = new CommitHashMap(config.getMaxItems());
 
+        LOG.info("RequestProcessor initialized");
+
     }
 
     /**
@@ -95,10 +100,10 @@ class RequestProcessorImpl implements EventHandler<RequestProcessorImpl.RequestE
      */
     @Override
     public void update(TSOState state) throws Exception {
-        LOG.info("Initializing RequestProcessor...");
+        LOG.info("Initializing RequestProcessor state...");
         this.lowWatermark = state.getLowWatermark();
         persistProc.persistLowWatermark(lowWatermark).get(); // Sync persist
-        LOG.info("RequestProcessor initialized with LWMs {} and Epoch {}", lowWatermark, state.getEpoch());
+        LOG.info("RequestProcessor state initialized with LWMs {} and Epoch {}", lowWatermark, state.getEpoch());
     }
 
     @Override
@@ -222,6 +227,25 @@ class RequestProcessorImpl implements EventHandler<RequestProcessorImpl.RequestE
 
     }
 
+    @Override
+    public void close() throws IOException {
+
+        LOG.info("Terminating Request Processor...");
+        disruptor.halt();
+        disruptor.shutdown();
+        LOG.info("\tRequest Processor Disruptor shutdown");
+        disruptorExec.shutdownNow();
+        try {
+            disruptorExec.awaitTermination(3, SECONDS);
+            LOG.info("\tRequest Processor Disruptor executor shutdown");
+        } catch (InterruptedException e) {
+            LOG.error("Interrupted whilst finishing Request Processor Disruptor executor");
+            Thread.currentThread().interrupt();
+        }
+        LOG.info("Request Processor terminated");
+
+    }
+
     final static class RequestEvent implements Iterable<Long> {
 
         enum Type {

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/08163880/tso-server/src/main/java/org/apache/omid/tso/RetryProcessor.java
----------------------------------------------------------------------
diff --git a/tso-server/src/main/java/org/apache/omid/tso/RetryProcessor.java b/tso-server/src/main/java/org/apache/omid/tso/RetryProcessor.java
index 1e2951f..c17b29a 100644
--- a/tso-server/src/main/java/org/apache/omid/tso/RetryProcessor.java
+++ b/tso-server/src/main/java/org/apache/omid/tso/RetryProcessor.java
@@ -19,6 +19,8 @@ package org.apache.omid.tso;
 
 import org.jboss.netty.channel.Channel;
 
-interface RetryProcessor {
+import java.io.Closeable;
+
+interface RetryProcessor extends Closeable {
     void disambiguateRetryRequestHeuristically(long startTimestamp, Channel c, MonitoringContext monCtx);
 }

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/08163880/tso-server/src/main/java/org/apache/omid/tso/RetryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/tso-server/src/main/java/org/apache/omid/tso/RetryProcessorImpl.java b/tso-server/src/main/java/org/apache/omid/tso/RetryProcessorImpl.java
index 776f0ed..872daa0 100644
--- a/tso-server/src/main/java/org/apache/omid/tso/RetryProcessorImpl.java
+++ b/tso-server/src/main/java/org/apache/omid/tso/RetryProcessorImpl.java
@@ -42,6 +42,7 @@ import java.util.concurrent.ThreadFactory;
 
 import static com.codahale.metrics.MetricRegistry.name;
 import static com.lmax.disruptor.dsl.ProducerType.SINGLE;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.omid.tso.RetryProcessorImpl.RetryEvent.EVENT_FACTORY;
 
 /**
@@ -52,9 +53,12 @@ class RetryProcessorImpl implements EventHandler<RetryProcessorImpl.RetryEvent>,
 
     private static final Logger LOG = LoggerFactory.getLogger(RetryProcessor.class);
 
-    // Disruptor chain stuff
+    // Disruptor-related attributes
+    private final ExecutorService disruptorExec;
+    private final Disruptor<RetryEvent> disruptor;
+    private final RingBuffer<RetryEvent> retryRing;
+
     final ReplyProcessor replyProc;
-    final RingBuffer<RetryEvent> retryRing;
 
     final CommitTable.Client commitTableClient;
     final ObjectPool<Batch> batchPool;
@@ -77,10 +81,10 @@ class RetryProcessorImpl implements EventHandler<RetryProcessorImpl.RetryEvent>,
         // ------------------------------------------------------------------------------------------------------------
 
         ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat("retry-%d").build();
-        ExecutorService requestExec = Executors.newSingleThreadExecutor(threadFactory);
+        this.disruptorExec = Executors.newSingleThreadExecutor(threadFactory);
 
-        Disruptor<RetryProcessorImpl.RetryEvent> disruptor = new Disruptor<>(EVENT_FACTORY, 1 << 12, requestExec, SINGLE, new YieldingWaitStrategy());
-        disruptor.handleExceptionsWith(new FatalExceptionHandler(panicker)); // This should be before the event handler
+        this.disruptor = new Disruptor<>(EVENT_FACTORY, 1 << 12, disruptorExec, SINGLE, new YieldingWaitStrategy());
+        disruptor.handleExceptionsWith(new FatalExceptionHandler(panicker)); // This must be before handleEventsWith()
         disruptor.handleEventsWith(this);
         this.retryRing = disruptor.start();
 
@@ -97,6 +101,8 @@ class RetryProcessorImpl implements EventHandler<RetryProcessorImpl.RetryEvent>,
         this.invalidTxMeter = metrics.meter(name("tso", "retries", "aborts", "tx-invalid"));
         this.noCTFoundMeter = metrics.meter(name("tso", "retries", "aborts", "tx-without-commit-timestamp"));
 
+        LOG.info("RetryProcessor initialized");
+
     }
 
     @Override
@@ -153,6 +159,25 @@ class RetryProcessorImpl implements EventHandler<RetryProcessorImpl.RetryEvent>,
         retryRing.publish(seq);
     }
 
+    @Override
+    public void close() throws IOException {
+
+        LOG.info("Terminating Retry Processor...");
+        disruptor.halt();
+        disruptor.shutdown();
+        LOG.info("\tRetry Processor Disruptor shutdown");
+        disruptorExec.shutdownNow();
+        try {
+            disruptorExec.awaitTermination(3, SECONDS);
+            LOG.info("\tRetry Processor Disruptor executor shutdown");
+        } catch (InterruptedException e) {
+            LOG.error("Interrupted whilst finishing Retry Processor Disruptor executor");
+            Thread.currentThread().interrupt();
+        }
+        LOG.info("Retry Processor terminated");
+
+    }
+
     public final static class RetryEvent {
 
         enum Type {

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/08163880/tso-server/src/main/java/org/apache/omid/tso/SystemExitPanicker.java
----------------------------------------------------------------------
diff --git a/tso-server/src/main/java/org/apache/omid/tso/SystemExitPanicker.java b/tso-server/src/main/java/org/apache/omid/tso/SystemExitPanicker.java
new file mode 100644
index 0000000..908d86a
--- /dev/null
+++ b/tso-server/src/main/java/org/apache/omid/tso/SystemExitPanicker.java
@@ -0,0 +1,38 @@
+/*
+ * 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.omid.tso;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SystemExitPanicker implements Panicker {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SystemExitPanicker.class);
+
+    @Override
+    public void panic(String reason) {
+        panic(reason, new Throwable("TSO Error"));
+    }
+
+    @Override
+    public void panic(String reason, Throwable cause) {
+        LOG.error(reason, cause);
+        System.exit(-1);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/08163880/tso-server/src/main/java/org/apache/omid/tso/TSOModule.java
----------------------------------------------------------------------
diff --git a/tso-server/src/main/java/org/apache/omid/tso/TSOModule.java b/tso-server/src/main/java/org/apache/omid/tso/TSOModule.java
index 4b0d61e..9b67fa2 100644
--- a/tso-server/src/main/java/org/apache/omid/tso/TSOModule.java
+++ b/tso-server/src/main/java/org/apache/omid/tso/TSOModule.java
@@ -44,7 +44,7 @@ class TSOModule extends AbstractModule {
         bind(TSOChannelHandler.class).in(Singleton.class);
         bind(TSOStateManager.class).to(TSOStateManagerImpl.class).in(Singleton.class);
         bind(TimestampOracle.class).to(TimestampOracleImpl.class).in(Singleton.class);
-        bind(Panicker.class).to(RuntimeExceptionPanicker.class).in(Singleton.class);
+        bind(Panicker.class).to(SystemExitPanicker.class).in(Singleton.class);
 
         install(new BatchPoolModule(config));
         // Disruptor setup

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/08163880/tso-server/src/main/java/org/apache/omid/tso/TSOServer.java
----------------------------------------------------------------------
diff --git a/tso-server/src/main/java/org/apache/omid/tso/TSOServer.java b/tso-server/src/main/java/org/apache/omid/tso/TSOServer.java
index c93a0b6..19d9f01 100644
--- a/tso-server/src/main/java/org/apache/omid/tso/TSOServer.java
+++ b/tso-server/src/main/java/org/apache/omid/tso/TSOServer.java
@@ -37,12 +37,21 @@ public class TSOServer extends AbstractIdleService {
 
     private static final Logger LOG = LoggerFactory.getLogger(TSOServer.class);
 
+    public static final String DASH_SEPARATOR_80_CHARS =
+            "--------------------------------------------------------------------------------";
+
     public static final String TSO_HOST_AND_PORT_KEY = "tso.hostandport";
 
     @Inject
     private TSOStateManager tsoStateManager;
     @Inject
     private RequestProcessor requestProcessor;
+    @Inject
+    private PersistenceProcessor persistenceProcessor;
+    @Inject
+    private RetryProcessor retryProcessor;
+    @Inject
+    private ReplyProcessor replyProcessor;
 
     // ----------------------------------------------------------------------------------------------------------------
     // High availability related variables
@@ -84,16 +93,36 @@ public class TSOServer extends AbstractIdleService {
 
     @Override
     protected void startUp() throws Exception {
+        LOG.info("{}", DASH_SEPARATOR_80_CHARS);
+        LOG.info("Starting TSO Server");
+        LOG.info("{}", DASH_SEPARATOR_80_CHARS);
         tsoStateManager.register(requestProcessor);
         leaseManagement.startService();
-        LOG.info("********** TSO Server running **********");
+        LOG.info("{}", DASH_SEPARATOR_80_CHARS);
+        if (leaseManagement instanceof VoidLeaseManager) {
+            LOG.info("TSO Server running and accepting connections");
+        } else if (leaseManagement instanceof LeaseManager) {
+            LOG.info("TSO Server running on HA mode. Waiting to be signaled as the Master replica...");
+        } else {
+            throw new RuntimeException("Wrong TSO mode");
+        }
+        LOG.info("{}", DASH_SEPARATOR_80_CHARS);
     }
 
     @Override
     protected void shutDown() throws Exception {
+        LOG.info("{}", DASH_SEPARATOR_80_CHARS);
+        LOG.info("Shutting Down TSO Server");
+        LOG.info("{}", DASH_SEPARATOR_80_CHARS);
         leaseManagement.stopService();
         tsoStateManager.unregister(requestProcessor);
-        LOG.info("********** TSO Server stopped successfully **********");
+        requestProcessor.close();
+        persistenceProcessor.close();
+        retryProcessor.close();
+        replyProcessor.close();
+        LOG.info("{}", DASH_SEPARATOR_80_CHARS);
+        LOG.info("TSO Server stopped");
+        LOG.info("{}", DASH_SEPARATOR_80_CHARS);
     }
 
     // ----------------------------------------------------------------------------------------------------------------


[10/14] incubator-omid git commit: Merge pull request #118 from yahoo/omid-43

Posted by fp...@apache.org.
Merge pull request #118 from yahoo/omid-43

Add DISCLAIMER file

Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/67fc7a4c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/67fc7a4c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/67fc7a4c

Branch: refs/heads/master
Commit: 67fc7a4c94f8b0e191cb672796ba16ab511dfed1
Parents: 4d3883a d9b29da
Author: ikatkov <ik...@gmail.com>
Authored: Mon Jun 6 13:05:28 2016 -0700
Committer: ikatkov <ik...@gmail.com>
Committed: Mon Jun 6 13:05:28 2016 -0700

----------------------------------------------------------------------
 DISCLAIMER | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------



[08/14] incubator-omid git commit: Merge pull request #114 from yahoo/omid-39

Posted by fp...@apache.org.
Merge pull request #114 from yahoo/omid-39

[OMID-39] Implement coprocessor metrics in Hadoop

Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/d72a4e0d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/d72a4e0d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/d72a4e0d

Branch: refs/heads/master
Commit: d72a4e0d952e4c8d9478548c5bd10ff8eaaf2226
Parents: 19063a7 2b2ed44
Author: ikatkov <ik...@gmail.com>
Authored: Mon Jun 6 13:04:35 2016 -0700
Committer: ikatkov <ik...@gmail.com>
Committed: Mon Jun 6 13:04:35 2016 -0700

----------------------------------------------------------------------
 hbase-coprocessor/pom.xml                       |   5 +
 .../hbase/regionserver/CompactorScanner.java    |  32 +++-
 .../metrics/CompactorCoprocessorMetrics.java    | 178 +++++++++++++++++++
 .../CompactorCoprocessorMetricsSource.java      |  90 ++++++++++
 .../apache/omid/transaction/OmidCompactor.java  |  41 ++++-
 .../TestCompactorCoprocessorMetrics.java        | 102 +++++++++++
 .../omid/transaction/TestCompactorScanner.java  |  13 +-
 7 files changed, 445 insertions(+), 16 deletions(-)
----------------------------------------------------------------------



[06/14] incubator-omid git commit: Merge pull request #112 from yahoo/omid-36

Posted by fp...@apache.org.
Merge pull request #112 from yahoo/omid-36

[OMID-36] Use SystemExitPanicker for killing TSO process

Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/6c6b27ca
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/6c6b27ca
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/6c6b27ca

Branch: refs/heads/master
Commit: 6c6b27ca9216f72d1bc53072fc9512bae1be2f62
Parents: 90cea32 0816388
Author: ikatkov <ik...@gmail.com>
Authored: Mon Jun 6 13:03:44 2016 -0700
Committer: ikatkov <ik...@gmail.com>
Committed: Mon Jun 6 13:03:44 2016 -0700

----------------------------------------------------------------------
 .../apache/omid/tso/PersistenceProcessor.java   |  3 +-
 .../omid/tso/PersistenceProcessorImpl.java      | 32 +++++++++++++--
 .../org/apache/omid/tso/ReplyProcessor.java     |  4 +-
 .../org/apache/omid/tso/ReplyProcessorImpl.java | 34 ++++++++++++++--
 .../org/apache/omid/tso/RequestProcessor.java   |  3 +-
 .../apache/omid/tso/RequestProcessorImpl.java   | 42 +++++++++++++++-----
 .../org/apache/omid/tso/RetryProcessor.java     |  4 +-
 .../org/apache/omid/tso/RetryProcessorImpl.java | 35 +++++++++++++---
 .../org/apache/omid/tso/SystemExitPanicker.java | 38 ++++++++++++++++++
 .../java/org/apache/omid/tso/TSOModule.java     |  2 +-
 .../java/org/apache/omid/tso/TSOServer.java     | 33 ++++++++++++++-
 11 files changed, 201 insertions(+), 29 deletions(-)
----------------------------------------------------------------------



[04/14] incubator-omid git commit: [OMID-41] Add GPG key to KEYS file for Francisco

Posted by fp...@apache.org.
[OMID-41] Add GPG key to KEYS file for Francisco

Change-Id: Ia5c22d4e30f3c193fc25857a174f5aba12db21fb


Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/8b2bfb5d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/8b2bfb5d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/8b2bfb5d

Branch: refs/heads/master
Commit: 8b2bfb5d8d0ef7199a56cc97123ebc3ce4e0c29e
Parents: 90cea32
Author: Francisco Perez-Sorrosal <fp...@yahoo-inc.com>
Authored: Fri Jun 3 15:35:14 2016 -0700
Committer: Francisco Perez-Sorrosal <fp...@yahoo-inc.com>
Committed: Fri Jun 3 16:48:01 2016 -0700

----------------------------------------------------------------------
 KEYS | 58 ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 58 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/8b2bfb5d/KEYS
----------------------------------------------------------------------
diff --git a/KEYS b/KEYS
index 9142a46..5048c56 100644
--- a/KEYS
+++ b/KEYS
@@ -111,3 +111,61 @@ Io2Cs9jOV2iU4yYf3dJgUiBR/Vq4Dt0AqNzPTNojBAuAUOP+d63+ojF9Qp/VqUKj
 GfBNQDqZTcwRAan7+TJD21r9uaJAIXYr/SQX8z0bbw==
 =oZEC
 -----END PGP PUBLIC KEY BLOCK-----
+
+pub   4096R/49E5B1D0 2016-06-03
+      Key fingerprint = 47C2 273F C9B2 5A3D F6C1  AB03 A1C2 32C0 49E5 B1D0
+uid       [ultimate] Francisco Perez-Sorrosal <fp...@gmail.com>
+sub   4096R/EEE9FA1B 2016-06-03
+
+-----BEGIN PGP PUBLIC KEY BLOCK-----
+Version: GnuPG v2
+
+mQINBFdRvwgBEADyVkfmQuC5MD5KKG/F27c+x7pEhLHLiHNRLwDg6iqdxq3zDJuG
+uRAsHyP8g4MKSgWfUfAL2HpH0/yFWczDnijWk/YtVx5pIe8gUiZAxaui5T5a5Tv7
+4pNF5JH7yB5/3KhYYpMbpkcKqRczS1jzdlz4JVE34l47GhAjWbzd6W3S+7YZ9gNt
+DPYnbU5ZKYZeOUYDCcGsEAeRHi+Nlh7w3zw4+qzQNapryMDiXeczhhzK7PfXtyAT
+qXExgkdqIhirQCsJDrFNHV6/ZX2U/ZviCvflrar0zBCFwXnwkB56OUv2zxRTijfR
+Rykn4nteAYMhqMdIMuy4eaeuU1iSOC7Wn/jMWgXAeYjojmSa8z88jDb9HeU8xfE8
+Jv37OsJNIKWazo1oUkvJobffBz7R1QT+D4J2rZcWVxtONvER+a9V3BoHJ9ZaOo39
+ZHVSYuQBjKmeQW0hrVRbiGTfHQu7GSC57anztabENuZplHBsMG93UIrgQG+RFs/4
+PIEQ2T8ugstBjwvEpAI29He14QhEFBnhisd5M6wOqc4rdc5873lk1jQlAN3jPr1R
+XPQrlRJJYOH5FnJCmpqbogBINy9l+FtlPqqA7yyR5+PewJQscrnuLl2h2zn77WcP
+WDbaONMdmtYHSo7seGIHnl2/NGBXrq1P+ccQqY8UmpFIPCl4fRt6ectdYQARAQAB
+tDNGcmFuY2lzY28gUGVyZXotU29ycm9zYWwgPGZwZXJlenNvcnJvc2FsQGdtYWls
+LmNvbT6JAjkEEwEIACMFAldRvwgCGwMHCwkIBwMCAQYVCAIJCgsEFgIDAQIeAQIX
+gAAKCRChwjLASeWx0AqbEADqRZeEaVXVnF2uidwJ4tYvvjyOZTL2/HMXhmIjo76Q
+AYJJeQZ0JcrFE7iA6N/N/n42XIAY6YZoA1knTSyH7/r2nWZFgwikkPzETE5tJ0SB
+Qh2pWYqrJrbbxJp2qRUJGqhrIV+hJd0LpyBTx0R6JN1nZd3BDC7TBOhXwmtBCNVT
+4Iidfgl8SQmWfyKVyS7pJbqeRBREdn7zqJ4a1gitYIhVj4+eKSNrw0+BvjQ0xj2Z
+LAG9GhH5+Ston6jmaT0CDcmeC9m/6Fj8RRX5T+HmyffMnFMmxQwTdyPkit/BgccP
+z2hS43QkQB4HNd4RsA5c/hHqoWOXm05t4YRHLBOlwhY2W6ZlgORAu7h4U0w3b0iy
+bQVu2tYE67OAU3R8epWaziDSpZJIIVGuUG4BXqfiklIv+uQeoDcXu6Xbj2oS1bla
+w+8AZK2/n5NidDVOVeqHEvo2NOTrx6jG9I0y2ETHdPE5a165xJOKjoBCyuAO4f0B
+MD7+JVY4dQBOy5fnZbvJTr7O8hgFxFmHRCdNHYM7re1KvLMcAOdh9D6bJO4Ae8ER
+dT60AwaO9enQ46BzedN0luV1Kee5UAY8vp9KCSJLwToNcl/o4bJqzTge7CEfaRze
+eWF5qnIjoCb2cpzdMhk/WjvD3Vy/NHzoedgrb6QKGBgrznmBLhHepZm+PwOwRJLp
+7rkCDQRXUb8IARAAzdpcLgQy+zHPP9UcDK1HuDMOLU6yVoR4vFZtFL2PeihUHG1O
+0cmmWIwlOxCpwrQutdcNbasDSM9Mfjnv4LCxqEndlMq0H6i6akMVEB3qB8MjjRvI
+esOZHwlHb+Qm1N9Ca5mqgT4j5T9uxuQI+QBepiHXE4eoSIELNLdcqYGX2gxglP7K
+CLcNs6e3BbDRTP1ln2r0fu7J54B+TGK92tvxtnr7bBO1RMu+KKW3FoB2EBMWjkGn
+fzc0i/FToFnLQfog2tigs5XFKsGLnIetZEeVkdU5UKcWHqB9lLwtCP2ZZMUdwQQ7
+2c8BP2Jjt3QSDL6vSoPZlsBKKeCia5T/PEMY58cjz3zdRFO8rwIFaVmml9pfvppm
+xWcED1d/FfFxbe8m4HmLUjx2ONyVcsfmLZMk3OdtO6+k2dAKH14XiayGI63aavoZ
+jrOM2jiEnRHCrmJuH3Dk5xlg2zSY0/w3A3oRFYEmnGdpE7rojao0aN15Dfyhosz+
+a0HWM4qfdfdy+9MVHwO9fK2xSwJy9yQB8xM9+RPnpTZ9nLTqJtxW7oBOAjajV9j1
+YWsdGnfVOmUs68ri3ys2CXrKvATX4NqjhABz4ljBBTXonE74+ephQ034dcWEho0e
+U8GrCWB75Dng1mT24yrwQX7oBdhac/7ajyVf6ZFVe9gtTApLodaXyoT9BG8AEQEA
+AYkCHwQYAQgACQUCV1G/CAIbDAAKCRChwjLASeWx0NnVEACr+ysUBdiM9QpSApJh
+RcDcUU/hvVHhfqYeE61VZn3E959L21ZqeR+6eGREkGjC46Q86wzzQEtw3jTnLhZ0
+Yfl6LiobPYrQ17wYKaDoSsJPPGWS6ID5hqBJLVY1UgmjVhDBoBrXVIqje/Frobjh
+EIhq5bMx7SAtoclnnpTIz835zozkeXdfGSAE+tjGhamQ2Mt61In9NSBO4R3OmMes
+dfv/QM2+iEYSKKSQPHglDKngRh/tqQbfP7IPx74ub0CPckDLT86wI+eF3jT1eYB0
+xl2Z73Bs3cWat5jgKLP7D7oaOCvDkahXWwB9EI6MeL9lD2PB3oZKUEw2nqFHviZP
+F/LyG8zLZl8g5DhQCd8JygIIkXdC4Tysdo+4hF2HRJLu3UCUCgS7gaowxIpn3F+w
+SuWytx6z46AQMht2MgtzrTS9oWXxK9a8v3xhuCuqZkxG2hPlTCb9ClnOV2bPPYeW
++mMqK/R5nT67sWLFQLcHQJlVMYm3qr3zaF4MgLebgcFu0VDe5i0sXvI6zbP72FlE
+99tatlJ6/bBkGa7dWGXfcmIQx49Ku8tViG7s6XLOS+GgJoHQkfRaK+wtvibi+4Sq
+CmWKfWjoe+hEbAEQ67oJk6WbWw6CsgzbGcZJynRnH+IxhFXX2DCLqy0fPIohnq8+
+tfrKNmj6HQPRul1+OimBoavVwA==
+=4ayV
+-----END PGP PUBLIC KEY BLOCK-----


[05/14] incubator-omid git commit: Add DISCLAIMER file

Posted by fp...@apache.org.
Add DISCLAIMER file

Required in incubator projects

Change-Id: I079cf7fc21cb9794417a0f046f7c88f9058719a4


Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/d9b29da4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/d9b29da4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/d9b29da4

Branch: refs/heads/master
Commit: d9b29da41fe13e0a5d368f312644c1ee5806076b
Parents: 90cea32
Author: Francisco Perez-Sorrosal <fp...@yahoo-inc.com>
Authored: Mon Jun 6 12:03:50 2016 -0700
Committer: Francisco Perez-Sorrosal <fp...@yahoo-inc.com>
Committed: Mon Jun 6 12:03:50 2016 -0700

----------------------------------------------------------------------
 DISCLAIMER | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d9b29da4/DISCLAIMER
----------------------------------------------------------------------
diff --git a/DISCLAIMER b/DISCLAIMER
new file mode 100644
index 0000000..0f19dce
--- /dev/null
+++ b/DISCLAIMER
@@ -0,0 +1,10 @@
+Apache Omid is an effort undergoing incubation at the Apache Software
+Foundation (ASF), sponsored by the Apache Incubator PMC.
+
+Incubation is required of all newly accepted projects until a further review
+indicates that the infrastructure, communications, and decision making process
+have stabilized in a manner consistent with other successful ASF projects.
+
+While incubation status is not necessarily a reflection of the completeness
+or stability of the code, it does indicate that the project has yet to be
+fully endorsed by the ASF.


[14/14] incubator-omid git commit: [ci skip]prepare for next development iteration

Posted by fp...@apache.org.
[ci skip]prepare for next development iteration


Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/d0ac6890
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/d0ac6890
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/d0ac6890

Branch: refs/heads/master
Commit: d0ac6890a2cd8c4a5152bc17b50911e941d5c13e
Parents: 6c8dfeb
Author: Omid CI <om...@yahoo-inc.com>
Authored: Mon Jun 6 22:26:50 2016 +0000
Committer: Omid CI <om...@yahoo-inc.com>
Committed: Mon Jun 6 22:26:50 2016 +0000

----------------------------------------------------------------------
 benchmarks/pom.xml          | 2 +-
 codahale-metrics/pom.xml    | 2 +-
 commit-table/pom.xml        | 2 +-
 common/pom.xml              | 2 +-
 examples/pom.xml            | 2 +-
 hbase-client/pom.xml        | 2 +-
 hbase-commit-table/pom.xml  | 2 +-
 hbase-common/pom.xml        | 2 +-
 hbase-coprocessor/pom.xml   | 2 +-
 hbase-shims/hbase-0/pom.xml | 2 +-
 hbase-shims/hbase-1/pom.xml | 2 +-
 hbase-shims/pom.xml         | 2 +-
 hbase-tools/pom.xml         | 2 +-
 metrics/pom.xml             | 2 +-
 pom.xml                     | 4 ++--
 statemachine/pom.xml        | 2 +-
 timestamp-storage/pom.xml   | 2 +-
 transaction-client/pom.xml  | 2 +-
 tso-server/pom.xml          | 2 +-
 19 files changed, 20 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/benchmarks/pom.xml
----------------------------------------------------------------------
diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml
index 5560f3e..02fc336 100644
--- a/benchmarks/pom.xml
+++ b/benchmarks/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-benchmarks</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/codahale-metrics/pom.xml
----------------------------------------------------------------------
diff --git a/codahale-metrics/pom.xml b/codahale-metrics/pom.xml
index 972a5be..b38ae20 100644
--- a/codahale-metrics/pom.xml
+++ b/codahale-metrics/pom.xml
@@ -4,7 +4,7 @@
     <parent>
         <artifactId>omid</artifactId>
         <groupId>org.apache.omid</groupId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/commit-table/pom.xml
----------------------------------------------------------------------
diff --git a/commit-table/pom.xml b/commit-table/pom.xml
index bcd5dd3..cb1ed23 100644
--- a/commit-table/pom.xml
+++ b/commit-table/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-commit-table</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index ff1d2bb..e541c1e 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index bb25df1..4bc35f1 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -6,7 +6,7 @@
     <parent>
         <artifactId>omid</artifactId>
         <groupId>org.apache.omid</groupId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/hbase-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index 4a5ff03..cd2e85c 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-hbase-client</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/hbase-commit-table/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-commit-table/pom.xml b/hbase-commit-table/pom.xml
index b78a935..cb8efc2 100644
--- a/hbase-commit-table/pom.xml
+++ b/hbase-commit-table/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-hbase-commit-table</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/hbase-common/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index 073a44c..3dd1ff8 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-hbase-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/hbase-coprocessor/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-coprocessor/pom.xml b/hbase-coprocessor/pom.xml
index cb9672f..faaba76 100644
--- a/hbase-coprocessor/pom.xml
+++ b/hbase-coprocessor/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-hbase-coprocessor</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/hbase-shims/hbase-0/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shims/hbase-0/pom.xml b/hbase-shims/hbase-0/pom.xml
index d9937b2..86fac21 100644
--- a/hbase-shims/hbase-0/pom.xml
+++ b/hbase-shims/hbase-0/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid-shims-aggregator</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-hbase0-shims</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/hbase-shims/hbase-1/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shims/hbase-1/pom.xml b/hbase-shims/hbase-1/pom.xml
index c406863..f0aa93d 100644
--- a/hbase-shims/hbase-1/pom.xml
+++ b/hbase-shims/hbase-1/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid-shims-aggregator</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-hbase1-shims</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/hbase-shims/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shims/pom.xml b/hbase-shims/pom.xml
index c2817e0..3d4f49c 100644
--- a/hbase-shims/pom.xml
+++ b/hbase-shims/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-shims-aggregator</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/hbase-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-tools/pom.xml b/hbase-tools/pom.xml
index b9a599e..09d68df 100644
--- a/hbase-tools/pom.xml
+++ b/hbase-tools/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-hbase-tools</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/metrics/pom.xml
----------------------------------------------------------------------
diff --git a/metrics/pom.xml b/metrics/pom.xml
index be7b2bd..338e6a6 100644
--- a/metrics/pom.xml
+++ b/metrics/pom.xml
@@ -6,7 +6,7 @@
     <parent>
         <artifactId>omid</artifactId>
         <groupId>org.apache.omid</groupId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-metrics</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 312e5d2..b8adcd5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -13,7 +13,7 @@
     <artifactId>omid</artifactId>
     <packaging>pom</packaging>
     <!-- WARNING: do not update version manually, use mvn versions:set -->
-    <version>0.8.2.6</version>
+    <version>0.8.2.7-SNAPSHOT</version>
 
     <organization>
         <name>Apache Software Foundation</name>
@@ -105,7 +105,7 @@
         <connection>scm:git:git://github.com/yahoo/omid.git</connection>
         <developerConnection>scm:git:https://${GH_TOKEN}@github.com/yahoo/omid.git</developerConnection>
         <url>https://github.com/yahoo/omid</url>
-        <tag>omid-0.8.2.6</tag>
+        <tag>master</tag>
     </scm>
 
     <distributionManagement>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/statemachine/pom.xml
----------------------------------------------------------------------
diff --git a/statemachine/pom.xml b/statemachine/pom.xml
index 6fb26ac..29987a8 100644
--- a/statemachine/pom.xml
+++ b/statemachine/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <name>State Machine</name>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/timestamp-storage/pom.xml
----------------------------------------------------------------------
diff --git a/timestamp-storage/pom.xml b/timestamp-storage/pom.xml
index 5ef1843..3866833 100644
--- a/timestamp-storage/pom.xml
+++ b/timestamp-storage/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-timestamp-storage</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/transaction-client/pom.xml
----------------------------------------------------------------------
diff --git a/transaction-client/pom.xml b/transaction-client/pom.xml
index b6807e0..5f0d28b 100644
--- a/transaction-client/pom.xml
+++ b/transaction-client/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-transaction-client</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/d0ac6890/tso-server/pom.xml
----------------------------------------------------------------------
diff --git a/tso-server/pom.xml b/tso-server/pom.xml
index 0748095..560c181 100644
--- a/tso-server/pom.xml
+++ b/tso-server/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6</version>
+        <version>0.8.2.7-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-tso-server</artifactId>


[07/14] incubator-omid git commit: Merge pull request #113 from yahoo/omid-38

Posted by fp...@apache.org.
Merge pull request #113 from yahoo/omid-38

[OMID-38] Align and improve client/server configurations

Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/19063a70
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/19063a70
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/19063a70

Branch: refs/heads/master
Commit: 19063a70b0fe9e76a6fdfacac4e675b2869983a7
Parents: 6c6b27c 5c1c0a4
Author: ikatkov <ik...@gmail.com>
Authored: Mon Jun 6 13:04:04 2016 -0700
Committer: ikatkov <ik...@gmail.com>
Committed: Mon Jun 6 13:04:04 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/omid/NetworkUtils.java |  53 +++++++
 .../apache/omid/transaction/OmidTestBase.java   |   2 +-
 .../TestEndToEndScenariosWithHA.java            |   4 +-
 .../apache/omid/transaction/TestCompaction.java |   2 +-
 tso-server/bin/omid-env.sh                      |  27 ++++
 tso-server/bin/omid.sh                          |  14 +-
 tso-server/conf/bench.log4j.xml                 |  15 --
 tso-server/conf/omid-env.sh                     |  27 ----
 tso-server/conf/omid-server-configuration.yml   |   8 +
 tso-server/conf/omid.yml                        |   1 -
 .../apache/omid/tso/RequestProcessorImpl.java   |   2 +-
 .../org/apache/omid/tso/TSOServerConfig.java    |  47 ++----
 .../default-omid-server-configuration.yml       | 148 ++++++++++++++++++
 tso-server/src/main/resources/default-omid.yml  | 149 -------------------
 .../apache/omid/tso/TestRequestProcessor.java   |   3 +-
 ...tionOfTSOClientServerBasicFunctionality.java |   2 +-
 .../client/TestTSOClientConnectionToTSO.java    |   6 +-
 ...stTSOClientRequestAndResponseBehaviours.java |   2 +-
 tso-server/src/test/resources/test-omid.yml     |   5 +-
 19 files changed, 265 insertions(+), 252 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/19063a70/tso-server/src/main/java/org/apache/omid/tso/RequestProcessorImpl.java
----------------------------------------------------------------------
diff --cc tso-server/src/main/java/org/apache/omid/tso/RequestProcessorImpl.java
index 808d43d,9b994cf..65416bc
--- a/tso-server/src/main/java/org/apache/omid/tso/RequestProcessorImpl.java
+++ b/tso-server/src/main/java/org/apache/omid/tso/RequestProcessorImpl.java
@@@ -89,10 -83,8 +89,10 @@@ class RequestProcessorImpl implements E
          this.metrics = metrics;
          this.persistProc = persistProc;
          this.timestampOracle = timestampOracle;
-         this.hashmap = new CommitHashMap(config.getMaxItems());
+         this.hashmap = new CommitHashMap(config.getConflictMapSize());
  
 +        LOG.info("RequestProcessor initialized");
 +
      }
  
      /**


[13/14] incubator-omid git commit: [ci skip]prepare release omid-0.8.2.6

Posted by fp...@apache.org.
[ci skip]prepare release omid-0.8.2.6


Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/6c8dfebc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/6c8dfebc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/6c8dfebc

Branch: refs/heads/master
Commit: 6c8dfebc11027a4367210429cae42f24de0bfdc0
Parents: 9b52d11
Author: Omid CI <om...@yahoo-inc.com>
Authored: Mon Jun 6 22:26:47 2016 +0000
Committer: Omid CI <om...@yahoo-inc.com>
Committed: Mon Jun 6 22:26:47 2016 +0000

----------------------------------------------------------------------
 benchmarks/pom.xml          | 2 +-
 codahale-metrics/pom.xml    | 2 +-
 commit-table/pom.xml        | 2 +-
 common/pom.xml              | 2 +-
 examples/pom.xml            | 2 +-
 hbase-client/pom.xml        | 2 +-
 hbase-commit-table/pom.xml  | 2 +-
 hbase-common/pom.xml        | 2 +-
 hbase-coprocessor/pom.xml   | 2 +-
 hbase-shims/hbase-0/pom.xml | 2 +-
 hbase-shims/hbase-1/pom.xml | 2 +-
 hbase-shims/pom.xml         | 2 +-
 hbase-tools/pom.xml         | 2 +-
 metrics/pom.xml             | 2 +-
 pom.xml                     | 4 ++--
 statemachine/pom.xml        | 2 +-
 timestamp-storage/pom.xml   | 2 +-
 transaction-client/pom.xml  | 2 +-
 tso-server/pom.xml          | 2 +-
 19 files changed, 20 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/benchmarks/pom.xml
----------------------------------------------------------------------
diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml
index 3bd6e4c..5560f3e 100644
--- a/benchmarks/pom.xml
+++ b/benchmarks/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-benchmarks</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/codahale-metrics/pom.xml
----------------------------------------------------------------------
diff --git a/codahale-metrics/pom.xml b/codahale-metrics/pom.xml
index 595933b..972a5be 100644
--- a/codahale-metrics/pom.xml
+++ b/codahale-metrics/pom.xml
@@ -4,7 +4,7 @@
     <parent>
         <artifactId>omid</artifactId>
         <groupId>org.apache.omid</groupId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/commit-table/pom.xml
----------------------------------------------------------------------
diff --git a/commit-table/pom.xml b/commit-table/pom.xml
index 423400a..bcd5dd3 100644
--- a/commit-table/pom.xml
+++ b/commit-table/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-commit-table</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 9bb118f..ff1d2bb 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 115d296..bb25df1 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -6,7 +6,7 @@
     <parent>
         <artifactId>omid</artifactId>
         <groupId>org.apache.omid</groupId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/hbase-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index fb9b026..4a5ff03 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-hbase-client</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/hbase-commit-table/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-commit-table/pom.xml b/hbase-commit-table/pom.xml
index 905fa87..b78a935 100644
--- a/hbase-commit-table/pom.xml
+++ b/hbase-commit-table/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-hbase-commit-table</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/hbase-common/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index b2a4910..073a44c 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-hbase-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/hbase-coprocessor/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-coprocessor/pom.xml b/hbase-coprocessor/pom.xml
index c2191b3..cb9672f 100644
--- a/hbase-coprocessor/pom.xml
+++ b/hbase-coprocessor/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-hbase-coprocessor</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/hbase-shims/hbase-0/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shims/hbase-0/pom.xml b/hbase-shims/hbase-0/pom.xml
index e15ef34..d9937b2 100644
--- a/hbase-shims/hbase-0/pom.xml
+++ b/hbase-shims/hbase-0/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid-shims-aggregator</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-hbase0-shims</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/hbase-shims/hbase-1/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shims/hbase-1/pom.xml b/hbase-shims/hbase-1/pom.xml
index 564942d..c406863 100644
--- a/hbase-shims/hbase-1/pom.xml
+++ b/hbase-shims/hbase-1/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid-shims-aggregator</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-hbase1-shims</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/hbase-shims/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shims/pom.xml b/hbase-shims/pom.xml
index 3a2e1cb..c2817e0 100644
--- a/hbase-shims/pom.xml
+++ b/hbase-shims/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-shims-aggregator</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/hbase-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-tools/pom.xml b/hbase-tools/pom.xml
index c37d4d0..b9a599e 100644
--- a/hbase-tools/pom.xml
+++ b/hbase-tools/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-hbase-tools</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/metrics/pom.xml
----------------------------------------------------------------------
diff --git a/metrics/pom.xml b/metrics/pom.xml
index 03cc4c8..be7b2bd 100644
--- a/metrics/pom.xml
+++ b/metrics/pom.xml
@@ -6,7 +6,7 @@
     <parent>
         <artifactId>omid</artifactId>
         <groupId>org.apache.omid</groupId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-metrics</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d8b3747..312e5d2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -13,7 +13,7 @@
     <artifactId>omid</artifactId>
     <packaging>pom</packaging>
     <!-- WARNING: do not update version manually, use mvn versions:set -->
-    <version>0.8.2.6-SNAPSHOT</version>
+    <version>0.8.2.6</version>
 
     <organization>
         <name>Apache Software Foundation</name>
@@ -105,7 +105,7 @@
         <connection>scm:git:git://github.com/yahoo/omid.git</connection>
         <developerConnection>scm:git:https://${GH_TOKEN}@github.com/yahoo/omid.git</developerConnection>
         <url>https://github.com/yahoo/omid</url>
-        <tag>master</tag>
+        <tag>omid-0.8.2.6</tag>
     </scm>
 
     <distributionManagement>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/statemachine/pom.xml
----------------------------------------------------------------------
diff --git a/statemachine/pom.xml b/statemachine/pom.xml
index 21aafc6..6fb26ac 100644
--- a/statemachine/pom.xml
+++ b/statemachine/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <name>State Machine</name>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/timestamp-storage/pom.xml
----------------------------------------------------------------------
diff --git a/timestamp-storage/pom.xml b/timestamp-storage/pom.xml
index af87d92..5ef1843 100644
--- a/timestamp-storage/pom.xml
+++ b/timestamp-storage/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-timestamp-storage</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/transaction-client/pom.xml
----------------------------------------------------------------------
diff --git a/transaction-client/pom.xml b/transaction-client/pom.xml
index df0c1b2..b6807e0 100644
--- a/transaction-client/pom.xml
+++ b/transaction-client/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-transaction-client</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/6c8dfebc/tso-server/pom.xml
----------------------------------------------------------------------
diff --git a/tso-server/pom.xml b/tso-server/pom.xml
index 2ebaf38..0748095 100644
--- a/tso-server/pom.xml
+++ b/tso-server/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.6-SNAPSHOT</version>
+        <version>0.8.2.6</version>
     </parent>
 
     <artifactId>omid-tso-server</artifactId>


[12/14] incubator-omid git commit: [ci skip]prepare for next development iteration

Posted by fp...@apache.org.
[ci skip]prepare for next development iteration


Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/9b52d11c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/9b52d11c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/9b52d11c

Branch: refs/heads/master
Commit: 9b52d11c31393e14ecbe2c1f0fa77bd8850d5ffb
Parents: 9b4aba4
Author: Omid CI <om...@yahoo-inc.com>
Authored: Mon Jun 6 20:21:44 2016 +0000
Committer: Omid CI <om...@yahoo-inc.com>
Committed: Mon Jun 6 20:21:44 2016 +0000

----------------------------------------------------------------------
 benchmarks/pom.xml          | 2 +-
 codahale-metrics/pom.xml    | 2 +-
 commit-table/pom.xml        | 2 +-
 common/pom.xml              | 2 +-
 examples/pom.xml            | 2 +-
 hbase-client/pom.xml        | 2 +-
 hbase-commit-table/pom.xml  | 2 +-
 hbase-common/pom.xml        | 2 +-
 hbase-coprocessor/pom.xml   | 2 +-
 hbase-shims/hbase-0/pom.xml | 2 +-
 hbase-shims/hbase-1/pom.xml | 2 +-
 hbase-shims/pom.xml         | 2 +-
 hbase-tools/pom.xml         | 2 +-
 metrics/pom.xml             | 2 +-
 pom.xml                     | 4 ++--
 statemachine/pom.xml        | 2 +-
 timestamp-storage/pom.xml   | 2 +-
 transaction-client/pom.xml  | 2 +-
 tso-server/pom.xml          | 2 +-
 19 files changed, 20 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/benchmarks/pom.xml
----------------------------------------------------------------------
diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml
index 32838b6..3bd6e4c 100644
--- a/benchmarks/pom.xml
+++ b/benchmarks/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-benchmarks</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/codahale-metrics/pom.xml
----------------------------------------------------------------------
diff --git a/codahale-metrics/pom.xml b/codahale-metrics/pom.xml
index 46d4a53..595933b 100644
--- a/codahale-metrics/pom.xml
+++ b/codahale-metrics/pom.xml
@@ -4,7 +4,7 @@
     <parent>
         <artifactId>omid</artifactId>
         <groupId>org.apache.omid</groupId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/commit-table/pom.xml
----------------------------------------------------------------------
diff --git a/commit-table/pom.xml b/commit-table/pom.xml
index 3198366..423400a 100644
--- a/commit-table/pom.xml
+++ b/commit-table/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-commit-table</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 043f150..9bb118f 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index b822c78..115d296 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -6,7 +6,7 @@
     <parent>
         <artifactId>omid</artifactId>
         <groupId>org.apache.omid</groupId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/hbase-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index acf4d83..fb9b026 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-hbase-client</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/hbase-commit-table/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-commit-table/pom.xml b/hbase-commit-table/pom.xml
index fa4a606..905fa87 100644
--- a/hbase-commit-table/pom.xml
+++ b/hbase-commit-table/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-hbase-commit-table</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/hbase-common/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index f612524..b2a4910 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-hbase-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/hbase-coprocessor/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-coprocessor/pom.xml b/hbase-coprocessor/pom.xml
index 5691912..c2191b3 100644
--- a/hbase-coprocessor/pom.xml
+++ b/hbase-coprocessor/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-hbase-coprocessor</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/hbase-shims/hbase-0/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shims/hbase-0/pom.xml b/hbase-shims/hbase-0/pom.xml
index 46042d3..e15ef34 100644
--- a/hbase-shims/hbase-0/pom.xml
+++ b/hbase-shims/hbase-0/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid-shims-aggregator</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-hbase0-shims</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/hbase-shims/hbase-1/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shims/hbase-1/pom.xml b/hbase-shims/hbase-1/pom.xml
index 8e73574..564942d 100644
--- a/hbase-shims/hbase-1/pom.xml
+++ b/hbase-shims/hbase-1/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid-shims-aggregator</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-hbase1-shims</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/hbase-shims/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shims/pom.xml b/hbase-shims/pom.xml
index 3f5e088..3a2e1cb 100644
--- a/hbase-shims/pom.xml
+++ b/hbase-shims/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-shims-aggregator</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/hbase-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-tools/pom.xml b/hbase-tools/pom.xml
index 6a8d9d4..c37d4d0 100644
--- a/hbase-tools/pom.xml
+++ b/hbase-tools/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-hbase-tools</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/metrics/pom.xml
----------------------------------------------------------------------
diff --git a/metrics/pom.xml b/metrics/pom.xml
index 4832150..03cc4c8 100644
--- a/metrics/pom.xml
+++ b/metrics/pom.xml
@@ -6,7 +6,7 @@
     <parent>
         <artifactId>omid</artifactId>
         <groupId>org.apache.omid</groupId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-metrics</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 89b8d2d..d8b3747 100644
--- a/pom.xml
+++ b/pom.xml
@@ -13,7 +13,7 @@
     <artifactId>omid</artifactId>
     <packaging>pom</packaging>
     <!-- WARNING: do not update version manually, use mvn versions:set -->
-    <version>0.8.2.5</version>
+    <version>0.8.2.6-SNAPSHOT</version>
 
     <organization>
         <name>Apache Software Foundation</name>
@@ -105,7 +105,7 @@
         <connection>scm:git:git://github.com/yahoo/omid.git</connection>
         <developerConnection>scm:git:https://${GH_TOKEN}@github.com/yahoo/omid.git</developerConnection>
         <url>https://github.com/yahoo/omid</url>
-        <tag>omid-0.8.2.5</tag>
+        <tag>master</tag>
     </scm>
 
     <distributionManagement>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/statemachine/pom.xml
----------------------------------------------------------------------
diff --git a/statemachine/pom.xml b/statemachine/pom.xml
index 9e74c9f..21aafc6 100644
--- a/statemachine/pom.xml
+++ b/statemachine/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <name>State Machine</name>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/timestamp-storage/pom.xml
----------------------------------------------------------------------
diff --git a/timestamp-storage/pom.xml b/timestamp-storage/pom.xml
index 35c7fd4..af87d92 100644
--- a/timestamp-storage/pom.xml
+++ b/timestamp-storage/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-timestamp-storage</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/transaction-client/pom.xml
----------------------------------------------------------------------
diff --git a/transaction-client/pom.xml b/transaction-client/pom.xml
index 0e40e19..df0c1b2 100644
--- a/transaction-client/pom.xml
+++ b/transaction-client/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-transaction-client</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b52d11c/tso-server/pom.xml
----------------------------------------------------------------------
diff --git a/tso-server/pom.xml b/tso-server/pom.xml
index 4abb2b0..2ebaf38 100644
--- a/tso-server/pom.xml
+++ b/tso-server/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5</version>
+        <version>0.8.2.6-SNAPSHOT</version>
     </parent>
 
     <artifactId>omid-tso-server</artifactId>


[09/14] incubator-omid git commit: Merge pull request #116 from yahoo/omid-41

Posted by fp...@apache.org.
Merge pull request #116 from yahoo/omid-41

[OMID-41] Add GPG key to KEYS file for Francisco

Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/4d3883a8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/4d3883a8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/4d3883a8

Branch: refs/heads/master
Commit: 4d3883a8b26449b1dcfcffb804e76d57f2aaabea
Parents: d72a4e0 8b2bfb5
Author: ikatkov <ik...@gmail.com>
Authored: Mon Jun 6 13:04:54 2016 -0700
Committer: ikatkov <ik...@gmail.com>
Committed: Mon Jun 6 13:04:54 2016 -0700

----------------------------------------------------------------------
 KEYS | 58 ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 58 insertions(+)
----------------------------------------------------------------------



[03/14] incubator-omid git commit: [OMID-39] Implement coprocessor metrics in Hadoop

Posted by fp...@apache.org.
[OMID-39] Implement coprocessor metrics in Hadoop

Histograms added:
-time on compactions
-time on processing a cell
-time on contacting commit table for retrieve a commit TS (RPC)

Counters added:
-major & minor compactions
-scanned rows (in all compactions)
-total cells explored (in all compactions)
-retained cells (in all compactions)
-shadow cells healed (in all compactions)
-discarded cells (in all compactions)
-tombstone cells found (in all compactions)

Change-Id: I4a3607f1992505896e674ddb073a51ebf2582550


Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/2b2ed449
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/2b2ed449
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/2b2ed449

Branch: refs/heads/master
Commit: 2b2ed44936d7aef438757ba945185be4ba1c212d
Parents: 90cea32
Author: Francisco Perez-Sorrosal <fp...@yahoo-inc.com>
Authored: Wed Jul 8 13:58:29 2015 -0700
Committer: Francisco Perez-Sorrosal <fp...@yahoo-inc.com>
Committed: Fri Jun 3 16:47:46 2016 -0700

----------------------------------------------------------------------
 hbase-coprocessor/pom.xml                       |   5 +
 .../hbase/regionserver/CompactorScanner.java    |  32 +++-
 .../metrics/CompactorCoprocessorMetrics.java    | 178 +++++++++++++++++++
 .../CompactorCoprocessorMetricsSource.java      |  90 ++++++++++
 .../apache/omid/transaction/OmidCompactor.java  |  41 ++++-
 .../TestCompactorCoprocessorMetrics.java        | 102 +++++++++++
 .../omid/transaction/TestCompactorScanner.java  |  13 +-
 7 files changed, 445 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/2b2ed449/hbase-coprocessor/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-coprocessor/pom.xml b/hbase-coprocessor/pom.xml
index c178a25..bb7a154 100644
--- a/hbase-coprocessor/pom.xml
+++ b/hbase-coprocessor/pom.xml
@@ -53,6 +53,11 @@
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-server</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-hadoop-compat</artifactId>
+            <version>${hbase.version}</version>
+        </dependency>
 
         <!-- end storage related -->
 

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/2b2ed449/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/regionserver/CompactorScanner.java
----------------------------------------------------------------------
diff --git a/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/regionserver/CompactorScanner.java b/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/regionserver/CompactorScanner.java
index 12ecbb0..b79b1af 100644
--- a/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/regionserver/CompactorScanner.java
+++ b/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/regionserver/CompactorScanner.java
@@ -25,6 +25,7 @@ import org.apache.omid.HBaseShims;
 import org.apache.omid.committable.CommitTable;
 import org.apache.omid.committable.CommitTable.Client;
 import org.apache.omid.committable.CommitTable.CommitTimestamp;
+import org.apache.omid.hbase.coprocessor.metrics.CompactorCoprocessorMetrics;
 import org.apache.omid.transaction.CellUtils;
 import org.apache.omid.transaction.CellInfo;
 import org.apache.hadoop.hbase.Cell;
@@ -62,14 +63,20 @@ public class CompactorScanner implements InternalScanner {
     private final Region hRegion;
 
     private boolean hasMoreRows = false;
-    private List<Cell> currentRowWorthValues = new ArrayList<Cell>();
+    private List<Cell> currentRowWorthValues = new ArrayList<>();
+
+    // Metrics
+    private CompactorCoprocessorMetrics metrics;
 
     public CompactorScanner(ObserverContext<RegionCoprocessorEnvironment> e,
                             InternalScanner internalScanner,
                             Client commitTableClient,
                             Queue<CommitTable.Client> commitTableClientQueue,
                             boolean isMajorCompaction,
-                            boolean preserveNonTransactionallyDeletedCells) throws IOException {
+                            boolean preserveNonTransactionallyDeletedCells,
+                            CompactorCoprocessorMetrics metrics) throws IOException
+    {
+
         this.internalScanner = internalScanner;
         this.commitTableClient = commitTableClient;
         this.commitTableClientQueue = commitTableClientQueue;
@@ -78,6 +85,7 @@ public class CompactorScanner implements InternalScanner {
         this.lowWatermark = getLowWatermarkFromCommitTable();
         // Obtain the table in which the scanner is going to operate
         this.hRegion = HBaseShims.getRegionCoprocessorRegion(e.getEnvironment());
+        this.metrics = metrics;
         LOG.info("Scanner cleaning up uncommitted txs older than LW [{}] in region [{}]",
                 lowWatermark, hRegion.getRegionInfo());
     }
@@ -101,6 +109,9 @@ public class CompactorScanner implements InternalScanner {
             if (LOG.isTraceEnabled()) {
                 LOG.trace("Row: Result {} limit {} more rows? {}", scanResult, limit, hasMoreRows);
             }
+            if (hasMoreRows) {
+                metrics.incrScannedRows();
+            }
             // 2) Traverse result list separating normal cells from shadow
             // cells and building a map to access easily the shadow cells.
             SortedMap<Cell, Optional<Cell>> cellToSc = CellUtils.mapCellsToShadowCells(scanResult);
@@ -111,17 +122,21 @@ public class CompactorScanner implements InternalScanner {
             PeekingIterator<Map.Entry<Cell, Optional<Cell>>> iter
                     = Iterators.peekingIterator(cellToSc.entrySet().iterator());
             while (iter.hasNext()) {
+                long cellProcessingTimeStartTimeInMs = System.currentTimeMillis();
+                metrics.incrTotalCells();
                 Map.Entry<Cell, Optional<Cell>> entry = iter.next();
                 Cell cell = entry.getKey();
                 Optional<Cell> shadowCellOp = entry.getValue();
 
                 if (cell.getTimestamp() > lowWatermark) {
                     retain(currentRowWorthValues, cell, shadowCellOp);
+                    metrics.updateCellProcessingTime(System.currentTimeMillis() - cellProcessingTimeStartTimeInMs);
                     continue;
                 }
 
                 if (shouldRetainNonTransactionallyDeletedCell(cell)) {
                     retain(currentRowWorthValues, cell, shadowCellOp);
+                    metrics.updateCellProcessingTime(System.currentTimeMillis() - cellProcessingTimeStartTimeInMs);
                     continue;
                 }
 
@@ -142,6 +157,8 @@ public class CompactorScanner implements InternalScanner {
                                 skipToNextColumn(cell, iter);
                             }
                         }
+                        metrics.incrTombstoneCells();
+                        metrics.updateCellProcessingTime(System.currentTimeMillis() - cellProcessingTimeStartTimeInMs);
                         continue;
                     }
                 }
@@ -155,12 +172,16 @@ public class CompactorScanner implements InternalScanner {
                         byte[] shadowCellValue = Bytes.toBytes(commitTimestamp.get().getValue());
                         Cell shadowCell = CellUtils.buildShadowCellFromCell(cell, shadowCellValue);
                         saveLastTimestampedCell(lastTimestampedCellsInRow, cell, shadowCell);
+                        metrics.incrHealedShadowCells();
                     } else {
                         LOG.trace("Discarding cell {}", cell);
+                        metrics.incrDiscardedCells();
                     }
                 }
+                metrics.updateCellProcessingTime(System.currentTimeMillis() - cellProcessingTimeStartTimeInMs);
             }
             retainLastTimestampedCellsSaved(currentRowWorthValues, lastTimestampedCellsInRow);
+            metrics.incrRetainedCells(lastTimestampedCellsInRow.values().size());
 
             // 4) Sort the list
             Collections.sort(currentRowWorthValues, KeyValue.COMPARATOR);
@@ -229,7 +250,9 @@ public class CompactorScanner implements InternalScanner {
 
     private Optional<CommitTimestamp> queryCommitTimestamp(Cell cell) throws IOException {
         try {
+            long queryCommitTableStartTimeInMs = System.currentTimeMillis();
             Optional<CommitTimestamp> ct = commitTableClient.getCommitTimestamp(cell.getTimestamp()).get();
+            metrics.updateCommitTableQueryTime(System.currentTimeMillis() - queryCommitTableStartTimeInMs);
             if (ct.isPresent()) {
                 return Optional.of(ct.get());
             } else {
@@ -276,11 +299,16 @@ public class CompactorScanner implements InternalScanner {
     }
 
     private void skipToNextColumn(Cell cell, PeekingIterator<Map.Entry<Cell, Optional<Cell>>> iter) {
+
+        int skippedCellsCount = 0;
         while (iter.hasNext()
                 && CellUtil.matchingFamily(iter.peek().getKey(), cell)
                 && CellUtil.matchingQualifier(iter.peek().getKey(), cell)) {
             iter.next();
+            skippedCellsCount++;
         }
+        metrics.incrSkippedCells(skippedCellsCount);
+
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/2b2ed449/hbase-coprocessor/src/main/java/org/apache/omid/hbase/coprocessor/metrics/CompactorCoprocessorMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-coprocessor/src/main/java/org/apache/omid/hbase/coprocessor/metrics/CompactorCoprocessorMetrics.java b/hbase-coprocessor/src/main/java/org/apache/omid/hbase/coprocessor/metrics/CompactorCoprocessorMetrics.java
new file mode 100644
index 0000000..f0e4acd
--- /dev/null
+++ b/hbase-coprocessor/src/main/java/org/apache/omid/hbase/coprocessor/metrics/CompactorCoprocessorMetrics.java
@@ -0,0 +1,178 @@
+/*
+ * 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.omid.hbase.coprocessor.metrics;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.MetricHistogram;
+import org.apache.hadoop.metrics2.lib.MetricMutableCounterLong;
+
+public class CompactorCoprocessorMetrics extends BaseSourceImpl implements CompactorCoprocessorMetricsSource {
+
+    private static final String METRICS_NAME = "CompactorCoprocessor";
+    private static final String METRICS_CONTEXT = "omid.coprocessor.compactor";
+    private static final String METRICS_DESCRIPTION = "Omid Compactor Coprocessor Metrics";
+    private static final String METRICS_JMX_CONTEXT = "Omid,sub=" + METRICS_NAME;
+
+    // ----------------------------------------------------------------------------------------------------------------
+    // Metrics
+    // ----------------------------------------------------------------------------------------------------------------
+
+    // Histogram-related keys & descriptions
+    static final String COMPACTIONS_KEY = "compactions";
+    static final String COMPACTIONS_DESC = "Histogram about Compactions";
+    static final String CELL_PROCESSING_KEY = "cellProcessing";
+    static final String CELL_PROCESSING_DESC = "Histogram about Cell Processing";
+    static final String COMMIT_TABLE_QUERY_KEY = "commitTableQuery";
+    static final String COMMIT_TABLE_QUERY_DESC = "Histogram about Commit Table Query";
+
+    // Counter-related keys & descriptions
+    static final String MAJOR_COMPACTION_KEY = "major-compactions";
+    static final String MAJOR_COMPACTION_DESC = "Number of major compactions";
+    static final String MINOR_COMPACTION_KEY = "minor-compactions";
+    static final String MINOR_COMPACTION_DESC = "Number of minor compactions";
+    static final String SCANNED_ROWS_KEY = "scanned-rows";
+    static final String SCANNED_ROWS_DESC = "Number of rows scanned";
+    static final String TOTAL_CELLS_KEY = "total-cells";
+    static final String TOTAL_CELLS_DESC = "Number of cells processed";
+    static final String RETAINED_CELLS_KEY = "retained-cells";
+    static final String RETAINED_CELLS_DESC = "Number of cells retained when compacting";
+    static final String SKIPPED_CELLS_KEY = "skipped-cells";
+    static final String SKIPPED_CELLS_DESC = "Number of cells skipped when compacting";
+    static final String HEALED_SHADOW_CELLS_KEY = "healed-shadow-cells";
+    static final String HEALED_SHADOW_CELLS_DESC = "Number of cells healed when compacting";
+    static final String DISCARDED_CELLS_KEY = "discarded-cells";
+    static final String DISCARDED_CELLS_DESC = "Number of cells discarded when compacting";
+    static final String TOMBSTONE_CELLS_KEY = "tombstone-cells";
+    static final String TOMBSTONE_CELLS_DESC = "Number of tombstone cells found when compacting";
+
+    // *************************** Elements **********************************/
+
+    // Histograms
+    private final MetricHistogram compactionsHistogram;
+    private final MetricHistogram cellProcessingHistogram;
+    private final MetricHistogram commitTableQueryHistogram;
+
+    // Counters
+    private final MetricMutableCounterLong majorCompactionsCounter;
+    private final MetricMutableCounterLong minorCompactionsCounter;
+    private final MetricMutableCounterLong scannedRowsCounter;
+    private final MetricMutableCounterLong totalCellsCounter;
+    private final MetricMutableCounterLong retainedCellsCounter;
+    private final MetricMutableCounterLong skippedCellsCounter;
+    private final MetricMutableCounterLong healedShadowCellsCounter;
+    private final MetricMutableCounterLong discardedCellsCounter;
+    private final MetricMutableCounterLong tombstoneCellsCounter;
+
+    // ----------------------------------------------------------------------------------------------------------------
+    // End of Metrics
+    // ----------------------------------------------------------------------------------------------------------------
+
+    public CompactorCoprocessorMetrics() {
+        this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+    }
+
+    public CompactorCoprocessorMetrics(String metricsName,
+                                       String metricsDescription,
+                                       String metricsContext,
+                                       String metricsJmxContext) {
+
+        super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+
+        // Histograms
+        compactionsHistogram = getMetricsRegistry().newHistogram(COMPACTIONS_KEY, COMPACTIONS_DESC);
+        cellProcessingHistogram = getMetricsRegistry().newHistogram(CELL_PROCESSING_KEY, CELL_PROCESSING_DESC);
+        commitTableQueryHistogram = getMetricsRegistry().newHistogram(COMMIT_TABLE_QUERY_KEY, COMMIT_TABLE_QUERY_DESC);
+
+        // Counters
+        majorCompactionsCounter = getMetricsRegistry().newCounter(MAJOR_COMPACTION_KEY, MAJOR_COMPACTION_DESC, 0L);
+        minorCompactionsCounter = getMetricsRegistry().newCounter(MINOR_COMPACTION_KEY, MINOR_COMPACTION_DESC, 0L);
+        scannedRowsCounter = getMetricsRegistry().newCounter(SCANNED_ROWS_KEY, SCANNED_ROWS_DESC, 0L);
+        totalCellsCounter = getMetricsRegistry().newCounter(TOTAL_CELLS_KEY, TOTAL_CELLS_DESC, 0L);
+        retainedCellsCounter = getMetricsRegistry().newCounter(RETAINED_CELLS_KEY, RETAINED_CELLS_DESC, 0L);
+        skippedCellsCounter = getMetricsRegistry().newCounter(SKIPPED_CELLS_KEY, SKIPPED_CELLS_DESC, 0L);
+        healedShadowCellsCounter = getMetricsRegistry().newCounter(HEALED_SHADOW_CELLS_KEY, HEALED_SHADOW_CELLS_DESC, 0L);
+        discardedCellsCounter = getMetricsRegistry().newCounter(DISCARDED_CELLS_KEY, DISCARDED_CELLS_DESC, 0L);
+        tombstoneCellsCounter = getMetricsRegistry().newCounter(TOMBSTONE_CELLS_KEY, TOMBSTONE_CELLS_DESC, 0L);
+
+    }
+
+    // ----------------------------------------------------------------------------------------------------------------
+    // CompactorCoprocessorMetricsSource Interface Impl
+    // ----------------------------------------------------------------------------------------------------------------
+
+    @Override
+    public void updateCompactionTime(long timeInMs) {
+        compactionsHistogram.add(timeInMs);
+    }
+
+    @Override
+    public void updateCellProcessingTime(long timeInMs) {
+        cellProcessingHistogram.add(timeInMs);
+    }
+
+    @Override
+    public void updateCommitTableQueryTime(long timeInMs) {
+        commitTableQueryHistogram.add(timeInMs);
+    }
+
+    @Override
+    public void incrMajorCompactions() {
+        majorCompactionsCounter.incr();
+    }
+
+    @Override
+    public void incrMinorCompactions() {
+        minorCompactionsCounter.incr();
+    }
+
+    @Override
+    public void incrScannedRows() {
+        scannedRowsCounter.incr();
+    }
+
+    @Override
+    public void incrTotalCells() {
+        totalCellsCounter.incr();
+    }
+
+    @Override
+    public void incrRetainedCells(long delta) {
+        retainedCellsCounter.incr(delta);
+    }
+
+    @Override
+    public void incrSkippedCells(long delta) {
+        skippedCellsCounter.incr(delta);
+    }
+
+    @Override
+    public void incrHealedShadowCells() {
+        healedShadowCellsCounter.incr();
+    }
+
+    @Override
+    public void incrDiscardedCells() {
+        discardedCellsCounter.incr();
+    }
+
+    @Override
+    public void incrTombstoneCells() {
+        tombstoneCellsCounter.incr();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/2b2ed449/hbase-coprocessor/src/main/java/org/apache/omid/hbase/coprocessor/metrics/CompactorCoprocessorMetricsSource.java
----------------------------------------------------------------------
diff --git a/hbase-coprocessor/src/main/java/org/apache/omid/hbase/coprocessor/metrics/CompactorCoprocessorMetricsSource.java b/hbase-coprocessor/src/main/java/org/apache/omid/hbase/coprocessor/metrics/CompactorCoprocessorMetricsSource.java
new file mode 100644
index 0000000..0dd5ff0
--- /dev/null
+++ b/hbase-coprocessor/src/main/java/org/apache/omid/hbase/coprocessor/metrics/CompactorCoprocessorMetricsSource.java
@@ -0,0 +1,90 @@
+/*
+ * 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.omid.hbase.coprocessor.metrics;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+
+public interface CompactorCoprocessorMetricsSource extends BaseSource {
+
+    /**
+     * Update the compaction time histogram
+     * @param timeInMs time it took
+     */
+    void updateCompactionTime(long timeInMs);
+
+    /**
+     * Update the time it took processing a cell
+     * @param timeInMs time it took
+     */
+    void updateCellProcessingTime(long timeInMs);
+
+    /**
+     * Update the time it took to query the commit table for trying to find
+     * the commit timestamp
+     * @param timeInMs time it took
+     */
+    void updateCommitTableQueryTime(long timeInMs);
+
+    /**
+     * Increment the number of major compactions
+     */
+    public void incrMajorCompactions();
+
+    /**
+     * Increment the number of minor compactions
+     */
+    public void incrMinorCompactions();
+
+    /**
+     * Increment the number of scanned rows when compacting
+     */
+    void incrScannedRows();
+
+    /**
+     * Increment the number of total cells processed when compacting
+     */
+    void incrTotalCells();
+
+    /**
+     * Increment the number of retained cells when compacting
+     * @param delta the delta to increment the counter
+     */
+    void incrRetainedCells(long delta);
+
+    /**
+     * Increment the number of skipped cells when compacting
+     * @param delta the delta to increment the counter
+     */
+    void incrSkippedCells(long delta);
+
+    /**
+     * Increment the number of healed shadow cells when compacting
+     */
+    void incrHealedShadowCells();
+
+    /**
+     * Increment the number of discarded cells when compacting
+     */
+    void incrDiscardedCells();
+
+    /**
+     * Increment the number of tombstone cells when compacting
+     */
+    void incrTombstoneCells();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/2b2ed449/hbase-coprocessor/src/main/java/org/apache/omid/transaction/OmidCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-coprocessor/src/main/java/org/apache/omid/transaction/OmidCompactor.java b/hbase-coprocessor/src/main/java/org/apache/omid/transaction/OmidCompactor.java
index 887a2f6..e2e47fa 100644
--- a/hbase-coprocessor/src/main/java/org/apache/omid/transaction/OmidCompactor.java
+++ b/hbase-coprocessor/src/main/java/org/apache/omid/transaction/OmidCompactor.java
@@ -18,6 +18,7 @@
 package org.apache.omid.transaction;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.omid.committable.CommitTable;
 import org.apache.omid.committable.hbase.HBaseCommitTable;
 import org.apache.omid.committable.hbase.HBaseCommitTableConfig;
@@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.omid.hbase.coprocessor.metrics.CompactorCoprocessorMetrics;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -70,6 +72,10 @@ public class OmidCompactor extends BaseRegionObserver {
     // will be deleted anyways after a major one
     private boolean retainNonTransactionallyDeletedCells;
 
+    // Metrics
+    private CompactorCoprocessorMetrics metrics;
+    private long compactionStartTimeInMs;
+
     public OmidCompactor() {
         LOG.info("Compactor coprocessor initialized via empty constructor");
     }
@@ -86,6 +92,8 @@ public class OmidCompactor extends BaseRegionObserver {
         retainNonTransactionallyDeletedCells =
                 conf.getBoolean(HBASE_RETAIN_NON_TRANSACTIONALLY_DELETED_CELLS_KEY,
                         HBASE_RETAIN_NON_TRANSACTIONALLY_DELETED_CELLS_DEFAULT);
+        LOG.info("\tStarting coprocessor metrics...");
+        metrics = new CompactorCoprocessorMetrics();
         LOG.info("Compactor coprocessor started");
     }
 
@@ -106,12 +114,11 @@ public class OmidCompactor extends BaseRegionObserver {
                                       InternalScanner scanner,
                                       ScanType scanType,
                                       CompactionRequest request) throws IOException {
+
         HTableDescriptor desc = e.getEnvironment().getRegion().getTableDesc();
-        HColumnDescriptor famDesc
-                = desc.getFamily(Bytes.toBytes(store.getColumnFamilyName()));
+        HColumnDescriptor famDesc = desc.getFamily(Bytes.toBytes(store.getColumnFamilyName()));
         boolean omidCompactable = Boolean.valueOf(famDesc.getValue(OMID_COMPACTABLE_CF_FLAG));
-        // only column families tagged as compactable are compacted
-        // with omid compactor
+        // only column families tagged as compactable are compacted with omid compactor
         if (!omidCompactable) {
             return scanner;
         } else {
@@ -120,15 +127,31 @@ public class OmidCompactor extends BaseRegionObserver {
                 commitTableClient = initAndGetCommitTableClient();
             }
             boolean isMajorCompaction = request.isMajor();
+            if (isMajorCompaction) {
+                metrics.incrMajorCompactions();
+            } else {
+                metrics.incrMinorCompactions();
+            }
+            compactionStartTimeInMs = System.currentTimeMillis();
             return new CompactorScanner(e,
-                    scanner,
-                    commitTableClient,
-                    commitTableClientQueue,
-                    isMajorCompaction,
-                    retainNonTransactionallyDeletedCells);
+                                        scanner,
+                                        commitTableClient,
+                                        commitTableClientQueue,
+                                        isMajorCompaction,
+                                        retainNonTransactionallyDeletedCells,
+                                        metrics);
         }
     }
 
+    @Override
+    public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
+                            final Store store,
+                            final StoreFile resultFile) throws IOException {
+
+        metrics.updateCompactionTime(System.currentTimeMillis() - compactionStartTimeInMs);
+
+    }
+
     private CommitTable.Client initAndGetCommitTableClient() throws IOException {
         LOG.info("Trying to get the commit table client");
         CommitTable commitTable = new HBaseCommitTable(conf, commitTableConf);

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/2b2ed449/hbase-coprocessor/src/test/java/org/apache/omid/hbase/coprocessor/metrics/TestCompactorCoprocessorMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-coprocessor/src/test/java/org/apache/omid/hbase/coprocessor/metrics/TestCompactorCoprocessorMetrics.java b/hbase-coprocessor/src/test/java/org/apache/omid/hbase/coprocessor/metrics/TestCompactorCoprocessorMetrics.java
new file mode 100644
index 0000000..7f03078
--- /dev/null
+++ b/hbase-coprocessor/src/test/java/org/apache/omid/hbase/coprocessor/metrics/TestCompactorCoprocessorMetrics.java
@@ -0,0 +1,102 @@
+/*
+ * 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.omid.hbase.coprocessor.metrics;
+
+import org.apache.hadoop.hbase.CompatibilityFactory;
+import org.apache.hadoop.hbase.test.MetricsAssertHelper;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import static org.apache.omid.hbase.coprocessor.metrics.CompactorCoprocessorMetrics.DISCARDED_CELLS_KEY;
+import static org.apache.omid.hbase.coprocessor.metrics.CompactorCoprocessorMetrics.HEALED_SHADOW_CELLS_KEY;
+import static org.apache.omid.hbase.coprocessor.metrics.CompactorCoprocessorMetrics.MAJOR_COMPACTION_KEY;
+import static org.apache.omid.hbase.coprocessor.metrics.CompactorCoprocessorMetrics.MINOR_COMPACTION_KEY;
+import static org.apache.omid.hbase.coprocessor.metrics.CompactorCoprocessorMetrics.RETAINED_CELLS_KEY;
+import static org.apache.omid.hbase.coprocessor.metrics.CompactorCoprocessorMetrics.SCANNED_ROWS_KEY;
+import static org.apache.omid.hbase.coprocessor.metrics.CompactorCoprocessorMetrics.SKIPPED_CELLS_KEY;
+import static org.apache.omid.hbase.coprocessor.metrics.CompactorCoprocessorMetrics.TOMBSTONE_CELLS_KEY;
+import static org.apache.omid.hbase.coprocessor.metrics.CompactorCoprocessorMetrics.TOTAL_CELLS_KEY;
+
+public class TestCompactorCoprocessorMetrics {
+
+    public static MetricsAssertHelper HELPER = CompatibilityFactory.getInstance(MetricsAssertHelper.class);
+
+    private CompactorCoprocessorMetrics compactorMetrics;
+
+    @BeforeClass
+    public static void classSetUp() {
+        HELPER.init();
+    }
+
+    @BeforeMethod
+    public void setUp() {
+        compactorMetrics = new CompactorCoprocessorMetrics();
+    }
+
+    @Test
+    public void testCounters() {
+
+        for (int i = 0; i < 10; i++) {
+            compactorMetrics.incrMajorCompactions();
+        }
+        HELPER.assertCounter(MAJOR_COMPACTION_KEY, 10, compactorMetrics);
+
+        for (int i = 0; i < 11; i++) {
+            compactorMetrics.incrMinorCompactions();
+        }
+        HELPER.assertCounter(MINOR_COMPACTION_KEY, 11, compactorMetrics);
+
+        for (int i = 0; i < 12; i++) {
+            compactorMetrics.incrScannedRows();
+        }
+        HELPER.assertCounter(SCANNED_ROWS_KEY, 12, compactorMetrics);
+
+        for (int i = 0; i < 13; i++) {
+            compactorMetrics.incrTotalCells();
+        }
+        HELPER.assertCounter(TOTAL_CELLS_KEY, 13, compactorMetrics);
+
+        for (int i = 0; i < 14; i++) {
+            compactorMetrics.incrRetainedCells(14);
+        }
+        HELPER.assertCounter(RETAINED_CELLS_KEY, 14 * 14, compactorMetrics);
+
+        for (int i = 0; i < 15; i++) {
+            compactorMetrics.incrSkippedCells(15);
+        }
+        HELPER.assertCounter(SKIPPED_CELLS_KEY, 15 * 15, compactorMetrics);
+
+        for (int i = 0; i < 16; i++) {
+            compactorMetrics.incrHealedShadowCells();
+        }
+        HELPER.assertCounter(HEALED_SHADOW_CELLS_KEY, 16, compactorMetrics);
+
+        for (int i = 0; i < 17; i++) {
+            compactorMetrics.incrDiscardedCells();
+        }
+        HELPER.assertCounter(DISCARDED_CELLS_KEY, 17, compactorMetrics);
+
+        for (int i = 0; i < 18; i++) {
+            compactorMetrics.incrTombstoneCells();
+        }
+        HELPER.assertCounter(TOMBSTONE_CELLS_KEY, 18, compactorMetrics);
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/2b2ed449/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompactorScanner.java
----------------------------------------------------------------------
diff --git a/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompactorScanner.java b/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompactorScanner.java
index 8a217b3..b1aa064 100644
--- a/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompactorScanner.java
+++ b/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompactorScanner.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.omid.committable.CommitTable;
 import org.apache.omid.committable.CommitTable.Client;
+import org.apache.omid.hbase.coprocessor.metrics.CompactorCoprocessorMetrics;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.annotations.DataProvider;
@@ -60,6 +61,7 @@ public class TestCompactorScanner {
             throws Exception {
 
         // Create required mocks
+        CompactorCoprocessorMetrics metrics = mock(CompactorCoprocessorMetrics.class);
         @SuppressWarnings("unchecked")
         ObserverContext<RegionCoprocessorEnvironment> ctx = mock(ObserverContext.class);
         InternalScanner internalScanner = mock(InternalScanner.class);
@@ -80,11 +82,12 @@ public class TestCompactorScanner {
 
         LOG.info("Testing when retain is {}", retainOption);
         try (CompactorScanner scanner = spy(new CompactorScanner(ctx,
-                internalScanner,
-                ctClient,
-                queue,
-                false,
-                retainOption))) {
+                                                                 internalScanner,
+                                                                 ctClient,
+                                                                 queue,
+                                                                 false,
+                                                                 retainOption,
+                                                                 metrics))) {
 
             // Different cell types to test
             KeyValue regularKV = new KeyValue(Bytes.toBytes("test-row"), TEST_TS, Type.Put);


[02/14] incubator-omid git commit: [OMID-38] Align and improve client/server configurations

Posted by fp...@apache.org.
[OMID-38] Align and improve client/server configurations

1) Align config file names
2) Move network utility to specific class
3) Rename conflict map-related parameters in tso configuration
4) Cleanup tso start script
5) Remove unused files

Change-Id: I871246ef3349a1939c26fc2309f4959efa1bade2


Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/5c1c0a48
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/5c1c0a48
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/5c1c0a48

Branch: refs/heads/master
Commit: 5c1c0a488886a28fbd54e444b8192b4aad7293d1
Parents: 90cea32
Author: Francisco Perez-Sorrosal <fp...@yahoo-inc.com>
Authored: Fri May 27 12:04:22 2016 -0700
Committer: Francisco Perez-Sorrosal <fp...@yahoo-inc.com>
Committed: Fri Jun 3 16:47:28 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/omid/NetworkUtils.java |  53 +++++++
 .../apache/omid/transaction/OmidTestBase.java   |   2 +-
 .../TestEndToEndScenariosWithHA.java            |   4 +-
 .../apache/omid/transaction/TestCompaction.java |   2 +-
 tso-server/bin/omid-env.sh                      |  27 ++++
 tso-server/bin/omid.sh                          |  14 +-
 tso-server/conf/bench.log4j.xml                 |  15 --
 tso-server/conf/omid-env.sh                     |  27 ----
 tso-server/conf/omid-server-configuration.yml   |   8 +
 tso-server/conf/omid.yml                        |   1 -
 .../apache/omid/tso/RequestProcessorImpl.java   |   5 +-
 .../org/apache/omid/tso/TSOServerConfig.java    |  47 ++----
 .../default-omid-server-configuration.yml       | 148 ++++++++++++++++++
 tso-server/src/main/resources/default-omid.yml  | 149 -------------------
 .../apache/omid/tso/TestRequestProcessor.java   |   3 +-
 ...tionOfTSOClientServerBasicFunctionality.java |   2 +-
 .../client/TestTSOClientConnectionToTSO.java    |   6 +-
 ...stTSOClientRequestAndResponseBehaviours.java |   2 +-
 tso-server/src/test/resources/test-omid.yml     |   5 +-
 19 files changed, 265 insertions(+), 255 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/common/src/main/java/org/apache/omid/NetworkUtils.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/omid/NetworkUtils.java b/common/src/main/java/org/apache/omid/NetworkUtils.java
new file mode 100644
index 0000000..09bd626
--- /dev/null
+++ b/common/src/main/java/org/apache/omid/NetworkUtils.java
@@ -0,0 +1,53 @@
+/*
+ * 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.omid;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.NetworkInterface;
+import java.net.SocketException;
+import java.util.Enumeration;
+
+public class NetworkUtils {
+
+    private static final Logger LOG = LoggerFactory.getLogger(NetworkUtils.class);
+
+    private static final String LINUX_TSO_NET_IFACE_PREFIX = "eth";
+    private static final String MAC_TSO_NET_IFACE_PREFIX = "en";
+
+    public static String getDefaultNetworkInterface() {
+
+        try {
+            Enumeration<NetworkInterface> networkInterfaces = NetworkInterface.getNetworkInterfaces();
+            while (networkInterfaces.hasMoreElements()) {
+                String name = networkInterfaces.nextElement().getDisplayName();
+                LOG.info("Iterating over network interfaces, found '{}'", name);
+                if (name.startsWith(MAC_TSO_NET_IFACE_PREFIX) || name.startsWith(LINUX_TSO_NET_IFACE_PREFIX)) {
+                    return name;
+                }
+            }
+        } catch (SocketException ignored) {
+            throw new RuntimeException("Failed to find any network interfaces", ignored);
+        }
+        throw new IllegalArgumentException(String.format("No network '%s*'/'%s*' interfaces found",
+                                                         MAC_TSO_NET_IFACE_PREFIX, LINUX_TSO_NET_IFACE_PREFIX));
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/hbase-client/src/test/java/org/apache/omid/transaction/OmidTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/omid/transaction/OmidTestBase.java b/hbase-client/src/test/java/org/apache/omid/transaction/OmidTestBase.java
index 4822c16..226db44 100644
--- a/hbase-client/src/test/java/org/apache/omid/transaction/OmidTestBase.java
+++ b/hbase-client/src/test/java/org/apache/omid/transaction/OmidTestBase.java
@@ -82,7 +82,7 @@ public abstract class OmidTestBase {
         // TSO Setup
         TSOServerConfig tsoConfig = new TSOServerConfig();
         tsoConfig.setPort(1234);
-        tsoConfig.setMaxItems(1000);
+        tsoConfig.setConflictMapSize(1000);
         Injector injector = Guice.createInjector(new TSOMockModule(tsoConfig));
         LOG.info("Starting TSO");
         TSOServer tso = injector.getInstance(TSOServer.class);

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/hbase-client/src/test/java/org/apache/omid/transaction/TestEndToEndScenariosWithHA.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/omid/transaction/TestEndToEndScenariosWithHA.java b/hbase-client/src/test/java/org/apache/omid/transaction/TestEndToEndScenariosWithHA.java
index dba54f5..c15a2c2 100644
--- a/hbase-client/src/test/java/org/apache/omid/transaction/TestEndToEndScenariosWithHA.java
+++ b/hbase-client/src/test/java/org/apache/omid/transaction/TestEndToEndScenariosWithHA.java
@@ -115,7 +115,7 @@ public class TestEndToEndScenariosWithHA extends OmidTestBase {
         // Configure TSO 1
         TSOServerConfig config1 = new TSOServerConfig();
         config1.setPort(TSO1_PORT);
-        config1.setMaxItems(1000);
+        config1.setConflictMapSize(1000);
         config1.setLeaseModule(new TestHALeaseManagementModule(TEST_LEASE_PERIOD_MS, TSO_LEASE_PATH, CURRENT_TSO_PATH, zkConnection, NAMESPACE));
         Injector injector1 = Guice.createInjector(new TestTSOModule(hbaseConf, config1));
         LOG.info("===================== Starting TSO 1 =====================");
@@ -128,7 +128,7 @@ public class TestEndToEndScenariosWithHA extends OmidTestBase {
         // Configure TSO 2
         TSOServerConfig config2 = new TSOServerConfig();
         config2.setPort(TSO2_PORT);
-        config2.setMaxItems(1000);
+        config2.setConflictMapSize(1000);
         config2.setLeaseModule(new TestHALeaseManagementModule(TEST_LEASE_PERIOD_MS, TSO_LEASE_PATH, CURRENT_TSO_PATH, zkConnection, NAMESPACE));
         Injector injector2 = Guice.createInjector(new TestTSOModule(hbaseConf, config2));
         LOG.info("===================== Starting TSO 2 =====================");

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompaction.java b/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompaction.java
index 5dff36e..3dfcb2a 100644
--- a/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompaction.java
+++ b/hbase-coprocessor/src/test/java/org/apache/omid/transaction/TestCompaction.java
@@ -111,7 +111,7 @@ public class TestCompaction {
     public void setupTestCompation() throws Exception {
         TSOServerConfig tsoConfig = new TSOServerConfig();
         tsoConfig.setPort(1234);
-        tsoConfig.setMaxItems(1);
+        tsoConfig.setConflictMapSize(1);
         injector = Guice.createInjector(new TSOForHBaseCompactorTestModule(tsoConfig));
         hbaseConf = injector.getInstance(Configuration.class);
         HBaseCommitTableConfig hBaseCommitTableConfig = injector.getInstance(HBaseCommitTableConfig.class);

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/bin/omid-env.sh
----------------------------------------------------------------------
diff --git a/tso-server/bin/omid-env.sh b/tso-server/bin/omid-env.sh
new file mode 100644
index 0000000..578382c
--- /dev/null
+++ b/tso-server/bin/omid-env.sh
@@ -0,0 +1,27 @@
+#
+# 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.
+#
+
+# Set the flags to pass to the jvm when running omid
+# export JVM_FLAGS=-Xmx8096m
+# ---------------------------------------------------------------------------------------------------------------------
+# Check if HADOOP_CONF_DIR and HBASE_CONF_DIR are set
+# ---------------------------------------------------------------------------------------------------------------------
+
+if [ -z ${HADOOP_CONF_DIR+x} ]; then echo "WARNING: HADOOP_CONF_DIR is unset"; else echo "HADOOP_CONF_DIR is set to '$HADOOP_CONF_DIR'"; fi
+if [ -z ${HBASE_CONF_DIR+x} ]; then echo "WARNING: HBASE_CONF_DIR is unset"; else echo "HBASE_CONF_DIR is set to '$HBASE_CONF_DIR'"; fi
+

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/bin/omid.sh
----------------------------------------------------------------------
diff --git a/tso-server/bin/omid.sh b/tso-server/bin/omid.sh
index 61f20ca..6afde68 100755
--- a/tso-server/bin/omid.sh
+++ b/tso-server/bin/omid.sh
@@ -17,19 +17,21 @@
 # limitations under the License.
 #
 
-
 SCRIPTDIR=`dirname $0`
 cd $SCRIPTDIR;
-CLASSPATH=../conf:$(HBASE_CONF_DIR):$(HADOOP_CONF_DIR)
 
-. ../conf/omid-env.sh
+# Load Omid environment variables
+source omid-env.sh
+
+# Configure classpath...
+CLASSPATH=../conf:$(HBASE_CONF_DIR):$(HADOOP_CONF_DIR)
 
-# for source release
+# ...for source release and...
 for j in ../target/omid-tso*.jar; do
     CLASSPATH=$CLASSPATH:$j
 done
 
-# for binary release
+# and for binary release
 for j in ../omid-tso*.jar; do
     CLASSPATH=$CLASSPATH:$j
 done
@@ -38,7 +40,7 @@ for j in ../lib/*.jar; do
 done
 
 tso() {
-    exec java $JVM_FLAGS -cp $CLASSPATH org.apache.omid.tso.TSOServer @../conf/omid.conf $@
+    exec java $JVM_FLAGS -cp $CLASSPATH org.apache.omid.tso.TSOServer $@
 }
 
 tsoRelauncher() {

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/conf/bench.log4j.xml
----------------------------------------------------------------------
diff --git a/tso-server/conf/bench.log4j.xml b/tso-server/conf/bench.log4j.xml
deleted file mode 100644
index dba0c65..0000000
--- a/tso-server/conf/bench.log4j.xml
+++ /dev/null
@@ -1,15 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" ?>
-<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
-<log4j:configuration xmlns:log4j='http://jakarta.apache.org/log4j/'>
-    <appender name="console" class="org.apache.log4j.ConsoleAppender">
-        <param name="Threshold" value="INFO"/>
-        <layout class="org.apache.log4j.PatternLayout">
-            <param name="ConversionPattern" value="%d [%t] %-5p %c{1} %x %m%n"/>
-        </layout>
-    </appender>
-
-    <root>
-        <level value="INFO"/>
-        <appender-ref ref="console"/>
-    </root>
-</log4j:configuration>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/conf/omid-env.sh
----------------------------------------------------------------------
diff --git a/tso-server/conf/omid-env.sh b/tso-server/conf/omid-env.sh
deleted file mode 100644
index 578382c..0000000
--- a/tso-server/conf/omid-env.sh
+++ /dev/null
@@ -1,27 +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.
-#
-
-# Set the flags to pass to the jvm when running omid
-# export JVM_FLAGS=-Xmx8096m
-# ---------------------------------------------------------------------------------------------------------------------
-# Check if HADOOP_CONF_DIR and HBASE_CONF_DIR are set
-# ---------------------------------------------------------------------------------------------------------------------
-
-if [ -z ${HADOOP_CONF_DIR+x} ]; then echo "WARNING: HADOOP_CONF_DIR is unset"; else echo "HADOOP_CONF_DIR is set to '$HADOOP_CONF_DIR'"; fi
-if [ -z ${HBASE_CONF_DIR+x} ]; then echo "WARNING: HBASE_CONF_DIR is unset"; else echo "HBASE_CONF_DIR is set to '$HBASE_CONF_DIR'"; fi
-

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/conf/omid-server-configuration.yml
----------------------------------------------------------------------
diff --git a/tso-server/conf/omid-server-configuration.yml b/tso-server/conf/omid-server-configuration.yml
new file mode 100644
index 0000000..018dd31
--- /dev/null
+++ b/tso-server/conf/omid-server-configuration.yml
@@ -0,0 +1,8 @@
+# =====================================================================================================================
+#
+# Omid TSO Server Configuration
+# ---------------------------------------------------------------------------------------------------------------------
+#
+# Tune here the default values for TSO server config parameters found in 'default-omid-server-configuration.yml' file
+#
+# =====================================================================================================================

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/conf/omid.yml
----------------------------------------------------------------------
diff --git a/tso-server/conf/omid.yml b/tso-server/conf/omid.yml
deleted file mode 100644
index ddf916b..0000000
--- a/tso-server/conf/omid.yml
+++ /dev/null
@@ -1 +0,0 @@
-# Put here any overwrites to the default configuration in 'default-omid.yml'
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/src/main/java/org/apache/omid/tso/RequestProcessorImpl.java
----------------------------------------------------------------------
diff --git a/tso-server/src/main/java/org/apache/omid/tso/RequestProcessorImpl.java b/tso-server/src/main/java/org/apache/omid/tso/RequestProcessorImpl.java
index b945eaf..9b994cf 100644
--- a/tso-server/src/main/java/org/apache/omid/tso/RequestProcessorImpl.java
+++ b/tso-server/src/main/java/org/apache/omid/tso/RequestProcessorImpl.java
@@ -18,11 +18,9 @@
 package org.apache.omid.tso;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.lmax.disruptor.BatchEventProcessor;
 import com.lmax.disruptor.EventFactory;
 import com.lmax.disruptor.EventHandler;
 import com.lmax.disruptor.RingBuffer;
-import com.lmax.disruptor.SequenceBarrier;
 import com.lmax.disruptor.TimeoutBlockingWaitStrategy;
 import com.lmax.disruptor.TimeoutHandler;
 import com.lmax.disruptor.dsl.Disruptor;
@@ -40,7 +38,6 @@ import java.util.NoSuchElementException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
 
 import static com.lmax.disruptor.dsl.ProducerType.MULTI;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
@@ -86,7 +83,7 @@ class RequestProcessorImpl implements EventHandler<RequestProcessorImpl.RequestE
         this.metrics = metrics;
         this.persistProc = persistProc;
         this.timestampOracle = timestampOracle;
-        this.hashmap = new CommitHashMap(config.getMaxItems());
+        this.hashmap = new CommitHashMap(config.getConflictMapSize());
 
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/src/main/java/org/apache/omid/tso/TSOServerConfig.java
----------------------------------------------------------------------
diff --git a/tso-server/src/main/java/org/apache/omid/tso/TSOServerConfig.java b/tso-server/src/main/java/org/apache/omid/tso/TSOServerConfig.java
index 05cb3f4..489b806 100644
--- a/tso-server/src/main/java/org/apache/omid/tso/TSOServerConfig.java
+++ b/tso-server/src/main/java/org/apache/omid/tso/TSOServerConfig.java
@@ -19,16 +19,13 @@ package org.apache.omid.tso;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.inject.Module;
+import org.apache.omid.NetworkUtils;
 import org.apache.omid.YAMLUtils;
 import org.apache.omid.metrics.MetricsRegistry;
 import org.apache.omid.tools.hbase.SecureHBaseConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.net.NetworkInterface;
-import java.net.SocketException;
-import java.util.Enumeration;
-
 /**
  * Reads the configuration parameters of a TSO server instance from CONFIG_FILE_NAME.
  * If file CONFIG_FILE_NAME is missing defaults to DEFAULT_CONFIG_FILE_NAME
@@ -38,11 +35,8 @@ public class TSOServerConfig extends SecureHBaseConfig {
 
     private static final Logger LOG = LoggerFactory.getLogger(TSOServerConfig.class);
 
-    private static final String DEFAULT_CONFIG_FILE_NAME = "default-omid.yml";
-    private static final String CONFIG_FILE_NAME = "omid.yml";
-
-    private static final String LINUX_TSO_NET_IFACE_PREFIX = "eth";
-    private static final String MAC_TSO_NET_IFACE_PREFIX = "en";
+    private static final String CONFIG_FILE_NAME = "omid-server-configuration.yml";
+    private static final String DEFAULT_CONFIG_FILE_NAME = "default-omid-server-configuration.yml";
 
     // ----------------------------------------------------------------------------------------------------------------
     // Instantiation
@@ -70,7 +64,7 @@ public class TSOServerConfig extends SecureHBaseConfig {
 
     private MetricsRegistry metrics;
 
-    private int maxItems;
+    private int conflictMapSize;
 
     private int numConcurrentCTWriters;
 
@@ -78,7 +72,7 @@ public class TSOServerConfig extends SecureHBaseConfig {
 
     private int batchPersistTimeoutInMs;
 
-    private String networkIfaceName = getDefaultNetworkInterface();
+    private String networkIfaceName = NetworkUtils.getDefaultNetworkInterface();
 
     public int getPort() {
         return port;
@@ -88,12 +82,12 @@ public class TSOServerConfig extends SecureHBaseConfig {
         this.port = port;
     }
 
-    public int getMaxItems() {
-        return maxItems;
+    public int getConflictMapSize() {
+        return conflictMapSize;
     }
 
-    public void setMaxItems(int maxItems) {
-        this.maxItems = maxItems;
+    public void setConflictMapSize(int conflictMapSize) {
+        this.conflictMapSize = conflictMapSize;
     }
 
     public int getNumConcurrentCTWriters() {
@@ -160,27 +154,4 @@ public class TSOServerConfig extends SecureHBaseConfig {
         this.metrics = metrics;
     }
 
-    // ----------------------------------------------------------------------------------------------------------------
-    // Helper methods
-    // ----------------------------------------------------------------------------------------------------------------
-
-    private String getDefaultNetworkInterface() {
-
-        try {
-            Enumeration<NetworkInterface> networkInterfaces = NetworkInterface.getNetworkInterfaces();
-            while (networkInterfaces.hasMoreElements()) {
-                String name = networkInterfaces.nextElement().getDisplayName();
-                LOG.info("Iterating over network interfaces, found '{}'", name);
-                if (name.startsWith(MAC_TSO_NET_IFACE_PREFIX) || name.startsWith(LINUX_TSO_NET_IFACE_PREFIX)) {
-                    return name;
-                }
-            }
-        } catch (SocketException ignored) {
-            throw new RuntimeException("Failed to find any network interfaces", ignored);
-        }
-        throw new IllegalArgumentException(String.format("No network '%s*'/'%s*' interfaces found",
-                                                         MAC_TSO_NET_IFACE_PREFIX, LINUX_TSO_NET_IFACE_PREFIX));
-
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/src/main/resources/default-omid-server-configuration.yml
----------------------------------------------------------------------
diff --git a/tso-server/src/main/resources/default-omid-server-configuration.yml b/tso-server/src/main/resources/default-omid-server-configuration.yml
new file mode 100644
index 0000000..de3faaf
--- /dev/null
+++ b/tso-server/src/main/resources/default-omid-server-configuration.yml
@@ -0,0 +1,148 @@
+# =====================================================================================================================
+# Omid TSO Server Configuration (Default parameters)
+# =====================================================================================================================
+
+# ---------------------------------------------------------------------------------------------------------------------
+# Basic configuration parameters
+# ---------------------------------------------------------------------------------------------------------------------
+
+# Network interface for TSO server communication. Uncomment the following line to use a specific interface
+# networkIfaceName: eth0
+# If a network interface in the configuration, the TSO will attempt to guess default network interface.
+# See org.apache.omid.tso.TSOServerConfig.getDefaultNetworkInterface for more information.
+
+# Port reserved by the Status Oracle
+port: 54758
+# The number of elements reserved in the conflict map to perform conflict resolution
+conflictMapSize: 100000000
+# The number of Commit Table writers that persist data concurrently to the datastore. It has to be at least 2.
+numConcurrentCTWriters: 2
+# The size of the batch of operations that each Commit Table writes has. The maximum number of operations that can be
+# batched in the system at a certain point in time is: numConcurrentCTWriters * batchSizePerCTWriter
+batchSizePerCTWriter: 25
+# When this timeout expires, the contents of the batch are flushed to the datastore
+batchPersistTimeoutInMs: 10
+
+# Default module configuration (No TSO High Availability & in-memory storage for timestamp and commit tables)
+timestampStoreModule: !!org.apache.omid.tso.InMemoryTimestampStorageModule [ ]
+commitTableStoreModule: !!org.apache.omid.tso.InMemoryCommitTableStorageModule [ ]
+leaseModule: !!org.apache.omid.tso.VoidLeaseManagementModule [ ]
+
+# Default stats/metrics configuration
+metrics: !!org.apache.omid.metrics.NullMetricsProvider [ ]
+
+# ---------------------------------------------------------------------------------------------------------------------
+# Timestamp storage configuration options
+# ---------------------------------------------------------------------------------------------------------------------
+# Could be any guava module that binds org.apache.omid.timestamp.storage.TimestampStorage
+# Current available Timestamp stores:
+#     org.apache.omid.tso.InMemoryTimestampStorageModule
+#     org.apache.omid.timestamp.storage.HBaseTimestampStorageModule
+#     org.apache.omid.timestamp.storage.ZKTimestampStorageModule
+
+# ---------------------------------------------------------------------------------------------------------------------
+# Commit Table storage configuration options
+# ---------------------------------------------------------------------------------------------------------------------
+# Could be any guava module that binds org.apache.omid.committable.CommitTable
+# Available CommitTable stores:
+#     org.apache.omid.committable.hbase.HBaseCommitTableStorageModule
+#     org.apache.omid.tso.InMemoryCommitTableStorageModule
+
+# ---------------------------------------------------------------------------------------------------------------------
+# Metrics configuration options
+# ---------------------------------------------------------------------------------------------------------------------
+# Metrics could be anything that is org.apache.omid.metrics.MetricsRegistry
+# There are 4 built-in reporters: CSV, SLF4J, GRAPHITE, CONSOLE
+# Please see org.apache.omid.metrics.CodahaleMetricsConfig for details.
+
+# Example configuration for reporting statistics to the console every minute:
+#
+# metrics: !!org.apache.omid.metrics.CodahaleMetricsProvider [
+#     !!org.apache.omid.metrics.CodahaleMetricsConfig {
+#         outputFreqInSecs: 60,
+#         reporters: !!set {
+#             !!org.apache.omid.metrics.CodahaleMetricsConfig$Reporter CONSOLE
+#         },
+#     }
+# ]
+
+# Example of multiple reporter configuration (to CSV files and console)
+#
+# metrics: !!org.apache.omid.metrics.CodahaleMetricsProvider [
+#     !!org.apache.omid.metrics.CodahaleMetricsConfig {
+#         outputFreqInSecs: 60,
+#         reporters: !!set {
+#             !!org.apache.omid.metrics.CodahaleMetricsConfig$Reporter CSV,
+#             !!org.apache.omid.metrics.CodahaleMetricsConfig$Reporter CONSOLE
+#         },
+#         csvDir: "csvMetrics",
+#         prefix: "somePrefix",
+#     }
+# ]
+
+# =====================================================================================================================
+# Some example configurations
+# =====================================================================================================================
+
+# ---------------------------------------------------------------------------------------------------------------------
+# Configuration WITHOUT High Availability using HBase for all required storage & reporting metrics to CSV files
+# ---------------------------------------------------------------------------------------------------------------------
+#
+# commitTableStoreModule: !!org.apache.omid.tso.DefaultHBaseCommitTableStorageModule [ ]
+#     See optional params
+#         - tableName
+#         - familyName
+#         - principal
+#         - keytab
+# timestampStoreModule: !!org.apache.omid.tso.DefaultHBaseTimestampStorageModule [ ]
+#     See optional params
+#         - tableName
+#         - familyName
+#         - principal
+#         - keytab
+# leaseModule: !!org.apache.omid.tso.VoidLeaseManagementModule [ ]
+# metrics: !!org.apache.omid.metrics.CodahaleMetricsProvider [
+#     !!org.apache.omid.metrics.CodahaleMetricsConfig {
+#         reporters: !!set {
+#             !!org.apache.omid.metrics.CodahaleMetricsConfig$Reporter CSV
+#         },
+#         csvDir: "myCSVMetricsDir",
+#         prefix: "myAppPrefixForMetrics",
+#     }
+# ]
+
+# ---------------------------------------------------------------------------------------------------------------------
+# Configuration WITHOUT High Availability using ZK to store the timestamps & reporting metrics to console every 30 secs
+# ---------------------------------------------------------------------------------------------------------------------
+#
+# commitTableStoreModule: !!org.apache.omid.tso.DefaultHBaseCommitTableStorageModule [ ]
+# timestampStoreModule: !!org.apache.omid.tso.DefaultZKTimestampStorageModule
+#         zkCluster: "localhost:2181"
+#         namespace: "omid"
+# leaseModule: !!org.apache.omid.tso.VoidLeaseManagementModule [ ]
+# metrics: !!org.apache.omid.metrics.CodahaleMetricsProvider [
+#     !!org.apache.omid.metrics.CodahaleMetricsConfig {
+#         outputFreqInSecs: 30,
+#         reporters: !!set {
+#             !!org.apache.omid.metrics.CodahaleMetricsConfig$Reporter CONSOLE
+#         },
+#     }
+# ]
+
+
+# ---------------------------------------------------------------------------------------------------------------------
+# Configuration WITH High Availability using HBase for all required storage and no metrics reports
+# ---------------------------------------------------------------------------------------------------------------------
+#
+# commitTableStoreModule: !!org.apache.omid.tso.DefaultHBaseCommitTableStorageModule [ ]
+# timestampStoreModule: !!org.apache.omid.tso.DefaultHBaseTimestampStorageModule [ ]
+# leaseModule: !!org.apache.omid.tso.HALeaseManagementModule
+#     leasePeriodInMs: 10000
+#     tsoLeasePath: "/tso-lease"
+#     currentTsoPath: "/current-tso"
+#     zkCluster: "localhost:2181"
+#     zkNamespace: "omid"
+# metrics: !!org.apache.omid.metrics.NullMetricsProvider [ ]
+
+
+

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/src/main/resources/default-omid.yml
----------------------------------------------------------------------
diff --git a/tso-server/src/main/resources/default-omid.yml b/tso-server/src/main/resources/default-omid.yml
deleted file mode 100644
index 552a77a..0000000
--- a/tso-server/src/main/resources/default-omid.yml
+++ /dev/null
@@ -1,149 +0,0 @@
-# =====================================================================================================================
-# Omid TSO Server Configuration
-# =====================================================================================================================
-
-# ---------------------------------------------------------------------------------------------------------------------
-# Basic configuration parameters
-# ---------------------------------------------------------------------------------------------------------------------
-
-# Network interface for TSO server communication. Uncomment the following line to use a specific interface
-# networkIfaceName: eth0
-# If a network interface in the configuration, the TSO will attempt to guess default network interface.
-# See org.apache.omid.tso.TSOServerConfig.getDefaultNetworkInterface for more information.
-
-# Port reserved by the Status Oracle
-port: 54758
-# The number of elements reserved in the conflict map to perform conflict resolution
-# TODO Rename to conflictMapSize
-maxItems: 100000000
-# The number of Commit Table writers that persist data concurrently to the datastore. It has to be at least 2.
-numConcurrentCTWriters: 2
-# The size of the batch of operations that each Commit Table writes has. The maximum number of operations that can be
-# batched in the system at a certain point in time is: numConcurrentCTWriters * batchSizePerCTWriter
-batchSizePerCTWriter: 25
-# When this timeout expires, the contents of the batch are flushed to the datastore
-batchPersistTimeoutInMs: 10
-
-# Default module configuration (No TSO High Availability & in-memory storage for timestamp and commit tables)
-timestampStoreModule: !!org.apache.omid.tso.InMemoryTimestampStorageModule [ ]
-commitTableStoreModule: !!org.apache.omid.tso.InMemoryCommitTableStorageModule [ ]
-leaseModule: !!org.apache.omid.tso.VoidLeaseManagementModule [ ]
-
-# Default stats/metrics configuration
-metrics: !!org.apache.omid.metrics.NullMetricsProvider [ ]
-
-# ---------------------------------------------------------------------------------------------------------------------
-# Timestamp storage configuration options
-# ---------------------------------------------------------------------------------------------------------------------
-# Could be any guava module that binds org.apache.omid.timestamp.storage.TimestampStorage
-# Current available Timestamp stores:
-#     org.apache.omid.tso.InMemoryTimestampStorageModule
-#     org.apache.omid.timestamp.storage.HBaseTimestampStorageModule
-#     org.apache.omid.timestamp.storage.ZKTimestampStorageModule
-
-# ---------------------------------------------------------------------------------------------------------------------
-# Commit Table storage configuration options
-# ---------------------------------------------------------------------------------------------------------------------
-# Could be any guava module that binds org.apache.omid.committable.CommitTable
-# Available CommitTable stores:
-#     org.apache.omid.committable.hbase.HBaseCommitTableStorageModule
-#     org.apache.omid.tso.InMemoryCommitTableStorageModule
-
-# ---------------------------------------------------------------------------------------------------------------------
-# Metrics configuration options
-# ---------------------------------------------------------------------------------------------------------------------
-# Metrics could be anything that is org.apache.omid.metrics.MetricsRegistry
-# There are 4 built-in reporters: CSV, SLF4J, GRAPHITE, CONSOLE
-# Please see org.apache.omid.metrics.CodahaleMetricsConfig for details.
-
-# Example configuration for reporting statistics to the console every minute:
-#
-# metrics: !!org.apache.omid.metrics.CodahaleMetricsProvider [
-#     !!org.apache.omid.metrics.CodahaleMetricsConfig {
-#         outputFreqInSecs: 60,
-#         reporters: !!set {
-#             !!org.apache.omid.metrics.CodahaleMetricsConfig$Reporter CONSOLE
-#         },
-#     }
-# ]
-
-# Example of multiple reporter configuration (to CSV files and console)
-#
-# metrics: !!org.apache.omid.metrics.CodahaleMetricsProvider [
-#     !!org.apache.omid.metrics.CodahaleMetricsConfig {
-#         outputFreqInSecs: 60,
-#         reporters: !!set {
-#             !!org.apache.omid.metrics.CodahaleMetricsConfig$Reporter CSV,
-#             !!org.apache.omid.metrics.CodahaleMetricsConfig$Reporter CONSOLE
-#         },
-#         csvDir: "csvMetrics",
-#         prefix: "somePrefix",
-#     }
-# ]
-
-# =====================================================================================================================
-# Some example configurations
-# =====================================================================================================================
-
-# ---------------------------------------------------------------------------------------------------------------------
-# Configuration WITHOUT High Availability using HBase for all required storage & reporting metrics to CSV files
-# ---------------------------------------------------------------------------------------------------------------------
-#
-# commitTableStoreModule: !!org.apache.omid.tso.DefaultHBaseCommitTableStorageModule [ ]
-#     See optional params
-#         - tableName
-#         - familyName
-#         - principal
-#         - keytab
-# timestampStoreModule: !!org.apache.omid.tso.DefaultHBaseTimestampStorageModule [ ]
-#     See optional params
-#         - tableName
-#         - familyName
-#         - principal
-#         - keytab
-# leaseModule: !!org.apache.omid.tso.VoidLeaseManagementModule [ ]
-# metrics: !!org.apache.omid.metrics.CodahaleMetricsProvider [
-#     !!org.apache.omid.metrics.CodahaleMetricsConfig {
-#         reporters: !!set {
-#             !!org.apache.omid.metrics.CodahaleMetricsConfig$Reporter CSV
-#         },
-#         csvDir: "myCSVMetricsDir",
-#         prefix: "myAppPrefixForMetrics",
-#     }
-# ]
-
-# ---------------------------------------------------------------------------------------------------------------------
-# Configuration WITHOUT High Availability using ZK to store the timestamps & reporting metrics to console every 30 secs
-# ---------------------------------------------------------------------------------------------------------------------
-#
-# commitTableStoreModule: !!org.apache.omid.tso.DefaultHBaseCommitTableStorageModule [ ]
-# timestampStoreModule: !!org.apache.omid.tso.DefaultZKTimestampStorageModule
-#         zkCluster: "localhost:2181"
-#         namespace: "omid"
-# leaseModule: !!org.apache.omid.tso.VoidLeaseManagementModule [ ]
-# metrics: !!org.apache.omid.metrics.CodahaleMetricsProvider [
-#     !!org.apache.omid.metrics.CodahaleMetricsConfig {
-#         outputFreqInSecs: 30,
-#         reporters: !!set {
-#             !!org.apache.omid.metrics.CodahaleMetricsConfig$Reporter CONSOLE
-#         },
-#     }
-# ]
-
-
-# ---------------------------------------------------------------------------------------------------------------------
-# Configuration WITH High Availability using HBase for all required storage and no metrics reports
-# ---------------------------------------------------------------------------------------------------------------------
-#
-# commitTableStoreModule: !!org.apache.omid.tso.DefaultHBaseCommitTableStorageModule [ ]
-# timestampStoreModule: !!org.apache.omid.tso.DefaultHBaseTimestampStorageModule [ ]
-# leaseModule: !!org.apache.omid.tso.HALeaseManagementModule
-#     leasePeriodInMs: 10000
-#     tsoLeasePath: "/tso-lease"
-#     currentTsoPath: "/current-tso"
-#     zkCluster: "localhost:2181"
-#     zkNamespace: "omid"
-# metrics: !!org.apache.omid.metrics.NullMetricsProvider [ ]
-
-
-

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/src/test/java/org/apache/omid/tso/TestRequestProcessor.java
----------------------------------------------------------------------
diff --git a/tso-server/src/test/java/org/apache/omid/tso/TestRequestProcessor.java b/tso-server/src/test/java/org/apache/omid/tso/TestRequestProcessor.java
index 8f2b06e..405102a 100644
--- a/tso-server/src/test/java/org/apache/omid/tso/TestRequestProcessor.java
+++ b/tso-server/src/test/java/org/apache/omid/tso/TestRequestProcessor.java
@@ -32,7 +32,6 @@ import java.util.Collections;
 import java.util.List;
 
 import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.doReturn;
@@ -74,7 +73,7 @@ public class TestRequestProcessor {
         doReturn(f).when(persist).persistLowWatermark(any(Long.class));
 
         TSOServerConfig config = new TSOServerConfig();
-        config.setMaxItems(CONFLICT_MAP_SIZE);
+        config.setConflictMapSize(CONFLICT_MAP_SIZE);
 
         requestProc = new RequestProcessorImpl(metrics, timestampOracle, persist, new MockPanicker(), config);
 

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/src/test/java/org/apache/omid/tso/client/TestIntegrationOfTSOClientServerBasicFunctionality.java
----------------------------------------------------------------------
diff --git a/tso-server/src/test/java/org/apache/omid/tso/client/TestIntegrationOfTSOClientServerBasicFunctionality.java b/tso-server/src/test/java/org/apache/omid/tso/client/TestIntegrationOfTSOClientServerBasicFunctionality.java
index e997685..535d1db 100644
--- a/tso-server/src/test/java/org/apache/omid/tso/client/TestIntegrationOfTSOClientServerBasicFunctionality.java
+++ b/tso-server/src/test/java/org/apache/omid/tso/client/TestIntegrationOfTSOClientServerBasicFunctionality.java
@@ -66,7 +66,7 @@ public class TestIntegrationOfTSOClientServerBasicFunctionality {
         tsoServerPortForTest = TestUtils.getFreeLocalPort();
 
         TSOServerConfig tsoConfig = new TSOServerConfig();
-        tsoConfig.setMaxItems(1000);
+        tsoConfig.setConflictMapSize(1000);
         tsoConfig.setPort(tsoServerPortForTest);
         Module tsoServerMockModule = new TSOMockModule(tsoConfig);
         Injector injector = Guice.createInjector(tsoServerMockModule);

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/src/test/java/org/apache/omid/tso/client/TestTSOClientConnectionToTSO.java
----------------------------------------------------------------------
diff --git a/tso-server/src/test/java/org/apache/omid/tso/client/TestTSOClientConnectionToTSO.java b/tso-server/src/test/java/org/apache/omid/tso/client/TestTSOClientConnectionToTSO.java
index 5e6aca9..2650e0e 100644
--- a/tso-server/src/test/java/org/apache/omid/tso/client/TestTSOClientConnectionToTSO.java
+++ b/tso-server/src/test/java/org/apache/omid/tso/client/TestTSOClientConnectionToTSO.java
@@ -117,7 +117,7 @@ public class TestTSOClientConnectionToTSO {
 
         // Launch a TSO WITHOUT publishing the address in HA...
         TSOServerConfig tsoConfig = new TSOServerConfig();
-        tsoConfig.setMaxItems(1000);
+        tsoConfig.setConflictMapSize(1000);
         tsoConfig.setPort(tsoPortForTest);
         tsoConfig.setLeaseModule(new VoidLeaseManagementModule());
         injector = Guice.createInjector(new TSOMockModule(tsoConfig));
@@ -153,7 +153,7 @@ public class TestTSOClientConnectionToTSO {
 
         // Launch a TSO publishing the address in HA...
         TSOServerConfig config = new TSOServerConfig();
-        config.setMaxItems(1000);
+        config.setConflictMapSize(1000);
         config.setPort(tsoPortForTest);
         config.setLeaseModule(new HALeaseManagementModule(1000, TSO_LEASE_PATH, CURRENT_TSO_PATH, zkClusterForTest, "omid"));
         injector = Guice.createInjector(new TSOMockModule(config));
@@ -191,7 +191,7 @@ public class TestTSOClientConnectionToTSO {
 
         // Start a TSO with HA...
         TSOServerConfig config = new TSOServerConfig();
-        config.setMaxItems(1000);
+        config.setConflictMapSize(1000);
         config.setPort(tsoPortForTest);
         config.setLeaseModule(new HALeaseManagementModule(1000, TSO_LEASE_PATH, CURRENT_TSO_PATH, zkClusterForTest, "omid"));
         injector = Guice.createInjector(new TSOMockModule(config));

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/src/test/java/org/apache/omid/tso/client/TestTSOClientRequestAndResponseBehaviours.java
----------------------------------------------------------------------
diff --git a/tso-server/src/test/java/org/apache/omid/tso/client/TestTSOClientRequestAndResponseBehaviours.java b/tso-server/src/test/java/org/apache/omid/tso/client/TestTSOClientRequestAndResponseBehaviours.java
index 7317847..1b5dce8 100644
--- a/tso-server/src/test/java/org/apache/omid/tso/client/TestTSOClientRequestAndResponseBehaviours.java
+++ b/tso-server/src/test/java/org/apache/omid/tso/client/TestTSOClientRequestAndResponseBehaviours.java
@@ -71,7 +71,7 @@ public class TestTSOClientRequestAndResponseBehaviours {
     public void beforeMethod() throws Exception {
 
         TSOServerConfig tsoConfig = new TSOServerConfig();
-        tsoConfig.setMaxItems(1000);
+        tsoConfig.setConflictMapSize(1000);
         tsoConfig.setPort(TSO_SERVER_PORT);
         tsoConfig.setNumConcurrentCTWriters(2);
         Module tsoServerMockModule = new TSOMockModule(tsoConfig);

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/5c1c0a48/tso-server/src/test/resources/test-omid.yml
----------------------------------------------------------------------
diff --git a/tso-server/src/test/resources/test-omid.yml b/tso-server/src/test/resources/test-omid.yml
index 2d4fee2..bfc0be4 100644
--- a/tso-server/src/test/resources/test-omid.yml
+++ b/tso-server/src/test/resources/test-omid.yml
@@ -1,10 +1,7 @@
 # Non High Availability configuration with HBase for everything
 
-principal: sieve_omid0@YGRID.YAHOO.COM
-keytab: /home/sieve_omid0/sieve_omid0.prod.headless.keytab
-
 port: 54758
-maxItems: 100000000
+conflictMapSize: 100000000
 numConcurrentCTWriters: 2
 batchSizePerCTWriter: 500
 batchPersistTimeoutInMs: 100



[11/14] incubator-omid git commit: [ci skip]prepare release omid-0.8.2.5

Posted by fp...@apache.org.
[ci skip]prepare release omid-0.8.2.5


Project: http://git-wip-us.apache.org/repos/asf/incubator-omid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-omid/commit/9b4aba45
Tree: http://git-wip-us.apache.org/repos/asf/incubator-omid/tree/9b4aba45
Diff: http://git-wip-us.apache.org/repos/asf/incubator-omid/diff/9b4aba45

Branch: refs/heads/master
Commit: 9b4aba452adbe1962b33aae0f6754355d20f68f1
Parents: 67fc7a4
Author: Omid CI <om...@yahoo-inc.com>
Authored: Mon Jun 6 20:21:41 2016 +0000
Committer: Omid CI <om...@yahoo-inc.com>
Committed: Mon Jun 6 20:21:41 2016 +0000

----------------------------------------------------------------------
 benchmarks/pom.xml          | 2 +-
 codahale-metrics/pom.xml    | 2 +-
 commit-table/pom.xml        | 2 +-
 common/pom.xml              | 2 +-
 examples/pom.xml            | 2 +-
 hbase-client/pom.xml        | 2 +-
 hbase-commit-table/pom.xml  | 2 +-
 hbase-common/pom.xml        | 2 +-
 hbase-coprocessor/pom.xml   | 2 +-
 hbase-shims/hbase-0/pom.xml | 2 +-
 hbase-shims/hbase-1/pom.xml | 2 +-
 hbase-shims/pom.xml         | 2 +-
 hbase-tools/pom.xml         | 2 +-
 metrics/pom.xml             | 2 +-
 pom.xml                     | 4 ++--
 statemachine/pom.xml        | 2 +-
 timestamp-storage/pom.xml   | 2 +-
 transaction-client/pom.xml  | 2 +-
 tso-server/pom.xml          | 2 +-
 19 files changed, 20 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/benchmarks/pom.xml
----------------------------------------------------------------------
diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml
index aa528d7..32838b6 100644
--- a/benchmarks/pom.xml
+++ b/benchmarks/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-benchmarks</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/codahale-metrics/pom.xml
----------------------------------------------------------------------
diff --git a/codahale-metrics/pom.xml b/codahale-metrics/pom.xml
index 6e9e5a0..46d4a53 100644
--- a/codahale-metrics/pom.xml
+++ b/codahale-metrics/pom.xml
@@ -4,7 +4,7 @@
     <parent>
         <artifactId>omid</artifactId>
         <groupId>org.apache.omid</groupId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/commit-table/pom.xml
----------------------------------------------------------------------
diff --git a/commit-table/pom.xml b/commit-table/pom.xml
index e2f82c8..3198366 100644
--- a/commit-table/pom.xml
+++ b/commit-table/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-commit-table</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 94d1d8a..043f150 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index a51409d..b822c78 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -6,7 +6,7 @@
     <parent>
         <artifactId>omid</artifactId>
         <groupId>org.apache.omid</groupId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/hbase-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index a77e626..acf4d83 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-hbase-client</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/hbase-commit-table/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-commit-table/pom.xml b/hbase-commit-table/pom.xml
index c642682..fa4a606 100644
--- a/hbase-commit-table/pom.xml
+++ b/hbase-commit-table/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-hbase-commit-table</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/hbase-common/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index 43269d9..f612524 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-hbase-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/hbase-coprocessor/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-coprocessor/pom.xml b/hbase-coprocessor/pom.xml
index bb7a154..5691912 100644
--- a/hbase-coprocessor/pom.xml
+++ b/hbase-coprocessor/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-hbase-coprocessor</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/hbase-shims/hbase-0/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shims/hbase-0/pom.xml b/hbase-shims/hbase-0/pom.xml
index 1f8462b..46042d3 100644
--- a/hbase-shims/hbase-0/pom.xml
+++ b/hbase-shims/hbase-0/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid-shims-aggregator</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-hbase0-shims</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/hbase-shims/hbase-1/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shims/hbase-1/pom.xml b/hbase-shims/hbase-1/pom.xml
index 33c96c8..8e73574 100644
--- a/hbase-shims/hbase-1/pom.xml
+++ b/hbase-shims/hbase-1/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid-shims-aggregator</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-hbase1-shims</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/hbase-shims/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shims/pom.xml b/hbase-shims/pom.xml
index 24a8173..3f5e088 100644
--- a/hbase-shims/pom.xml
+++ b/hbase-shims/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-shims-aggregator</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/hbase-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-tools/pom.xml b/hbase-tools/pom.xml
index fc1cbc0..6a8d9d4 100644
--- a/hbase-tools/pom.xml
+++ b/hbase-tools/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-hbase-tools</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/metrics/pom.xml
----------------------------------------------------------------------
diff --git a/metrics/pom.xml b/metrics/pom.xml
index 4527608..4832150 100644
--- a/metrics/pom.xml
+++ b/metrics/pom.xml
@@ -6,7 +6,7 @@
     <parent>
         <artifactId>omid</artifactId>
         <groupId>org.apache.omid</groupId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-metrics</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e4288a2..89b8d2d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -13,7 +13,7 @@
     <artifactId>omid</artifactId>
     <packaging>pom</packaging>
     <!-- WARNING: do not update version manually, use mvn versions:set -->
-    <version>0.8.2.5-SNAPSHOT</version>
+    <version>0.8.2.5</version>
 
     <organization>
         <name>Apache Software Foundation</name>
@@ -105,7 +105,7 @@
         <connection>scm:git:git://github.com/yahoo/omid.git</connection>
         <developerConnection>scm:git:https://${GH_TOKEN}@github.com/yahoo/omid.git</developerConnection>
         <url>https://github.com/yahoo/omid</url>
-        <tag>master</tag>
+        <tag>omid-0.8.2.5</tag>
     </scm>
 
     <distributionManagement>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/statemachine/pom.xml
----------------------------------------------------------------------
diff --git a/statemachine/pom.xml b/statemachine/pom.xml
index f7f6091..9e74c9f 100644
--- a/statemachine/pom.xml
+++ b/statemachine/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <name>State Machine</name>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/timestamp-storage/pom.xml
----------------------------------------------------------------------
diff --git a/timestamp-storage/pom.xml b/timestamp-storage/pom.xml
index 0b23efc..35c7fd4 100644
--- a/timestamp-storage/pom.xml
+++ b/timestamp-storage/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-timestamp-storage</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/transaction-client/pom.xml
----------------------------------------------------------------------
diff --git a/transaction-client/pom.xml b/transaction-client/pom.xml
index 3bfdcee..0e40e19 100644
--- a/transaction-client/pom.xml
+++ b/transaction-client/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-transaction-client</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/9b4aba45/tso-server/pom.xml
----------------------------------------------------------------------
diff --git a/tso-server/pom.xml b/tso-server/pom.xml
index 2216d83..4abb2b0 100644
--- a/tso-server/pom.xml
+++ b/tso-server/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <groupId>org.apache.omid</groupId>
         <artifactId>omid</artifactId>
-        <version>0.8.2.5-SNAPSHOT</version>
+        <version>0.8.2.5</version>
     </parent>
 
     <artifactId>omid-tso-server</artifactId>