You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@omid.apache.org by oh...@apache.org on 2017/07/10 11:43:52 UTC

incubator-omid git commit: [OMID-70] Adding world base timestamp generator. The generator maintains monotonicity and persists the maximal timestamp in each interval. The motivation for this commit is the integration with Apache Phoenix. Phoenix's cus

Repository: incubator-omid
Updated Branches:
  refs/heads/master 0005b3603 -> 60c9e714f


[OMID-70]  Adding world base timestamp generator. The
 generator maintains monotonicity and persists the maximal timestamp in each
 interval. The motivation for this commit is the integration with Apache
 Phoenix. Phoenix's customers requires mapping from timestamp to world time in
 order to debug their logs. This commit also contains augmentation of the
 proto in order to partially serialize transaction. This is also for Phoenix
 usage.


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

Branch: refs/heads/master
Commit: 60c9e714fd862f17407202565ac1eb7011910e1e
Parents: 0005b36
Author: Ohad Shacham <oh...@yahoo-inc.com>
Authored: Mon Jul 10 14:22:44 2017 +0300
Committer: Ohad Shacham <oh...@yahoo-inc.com>
Committed: Mon Jul 10 14:22:44 2017 +0300

----------------------------------------------------------------------
 common/src/main/proto/TSOProto.proto            |   1 +
 .../omid/transaction/HBaseTransaction.java      |   2 +-
 .../java/org/apache/omid/tso/TSOModule.java     |  11 +-
 .../org/apache/omid/tso/TSOServerConfig.java    |  19 ++
 .../apache/omid/tso/WorldClockOracleImpl.java   | 182 +++++++++++++++++++
 .../default-omid-server-configuration.yml       |   4 +
 .../apache/omid/tso/TestWorldTimeOracle.java    | 117 ++++++++++++
 7 files changed, 334 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/60c9e714/common/src/main/proto/TSOProto.proto
----------------------------------------------------------------------
diff --git a/common/src/main/proto/TSOProto.proto b/common/src/main/proto/TSOProto.proto
index 749beaa..e4e3116 100644
--- a/common/src/main/proto/TSOProto.proto
+++ b/common/src/main/proto/TSOProto.proto
@@ -64,3 +64,4 @@ message HandshakeResponse {
     optional bool clientCompatible = 1;
     optional Capabilities serverCapabilities = 2;
 }
+

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/60c9e714/hbase-client/src/main/java/org/apache/omid/transaction/HBaseTransaction.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/omid/transaction/HBaseTransaction.java b/hbase-client/src/main/java/org/apache/omid/transaction/HBaseTransaction.java
index 2bae5f5..3cdf657 100644
--- a/hbase-client/src/main/java/org/apache/omid/transaction/HBaseTransaction.java
+++ b/hbase-client/src/main/java/org/apache/omid/transaction/HBaseTransaction.java
@@ -29,7 +29,7 @@ import java.util.Set;
 public class HBaseTransaction extends AbstractTransaction<HBaseCellId> {
     private static final Logger LOG = LoggerFactory.getLogger(HBaseTransaction.class);
 
-    HBaseTransaction(long transactionId, long epoch, Set<HBaseCellId> writeSet, AbstractTransactionManager tm) {
+    public HBaseTransaction(long transactionId, long epoch, Set<HBaseCellId> writeSet, AbstractTransactionManager tm) {
         super(transactionId, epoch, writeSet, tm);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/60c9e714/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 a7aec27..4d0d844 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
@@ -24,6 +24,9 @@ import com.google.inject.Provides;
 
 import javax.inject.Named;
 import javax.inject.Singleton;
+
+import org.apache.omid.tso.TSOServerConfig.TIMESTAMP_TYPE;
+
 import java.net.SocketException;
 import java.net.UnknownHostException;
 
@@ -43,7 +46,13 @@ 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);
+
+        if (config.getTimestampTypeEnum() == TIMESTAMP_TYPE.WORLD_TIME) {
+            bind(TimestampOracle.class).to(WorldClockOracleImpl.class).in(Singleton.class);
+        } else {
+            bind(TimestampOracle.class).to(TimestampOracleImpl.class).in(Singleton.class);
+        }
+
         bind(Panicker.class).to(SystemExitPanicker.class).in(Singleton.class);
 
         install(new BatchPoolModule(config));

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/60c9e714/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 3292211..8f061a1 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
@@ -44,6 +44,11 @@ public class TSOServerConfig extends SecureHBaseConfig {
         LOW_CPU
     };
 
+    public static enum TIMESTAMP_TYPE {
+      INCREMENTAL,
+      WORLD_TIME
+    };
+
     // ----------------------------------------------------------------------------------------------------------------
     // Instantiation
     // ----------------------------------------------------------------------------------------------------------------
@@ -82,6 +87,8 @@ public class TSOServerConfig extends SecureHBaseConfig {
 
     private String networkIfaceName = NetworkUtils.getDefaultNetworkInterface();
 
+    private String timestampType;
+
     public int getPort() {
         return port;
     }
@@ -130,6 +137,18 @@ public class TSOServerConfig extends SecureHBaseConfig {
         this.networkIfaceName = networkIfaceName;
     }
 
+    public String getTimestampType() {
+        return timestampType;
+    }
+
+    public void setTimestampType(String type) {
+        this.timestampType = type;
+    }
+
+    public TIMESTAMP_TYPE getTimestampTypeEnum() {
+        return TSOServerConfig.TIMESTAMP_TYPE.valueOf(timestampType);
+    }
+
     public Module getTimestampStoreModule() {
         return timestampStoreModule;
     }

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/60c9e714/tso-server/src/main/java/org/apache/omid/tso/WorldClockOracleImpl.java
----------------------------------------------------------------------
diff --git a/tso-server/src/main/java/org/apache/omid/tso/WorldClockOracleImpl.java b/tso-server/src/main/java/org/apache/omid/tso/WorldClockOracleImpl.java
new file mode 100644
index 0000000..454da7e
--- /dev/null
+++ b/tso-server/src/main/java/org/apache/omid/tso/WorldClockOracleImpl.java
@@ -0,0 +1,182 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import org.apache.omid.metrics.Gauge;
+import org.apache.omid.metrics.MetricsRegistry;
+import org.apache.omid.timestamp.storage.TimestampStorage;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+
+import java.io.IOException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.omid.metrics.MetricsUtils.name;
+
+/**
+ * The Timestamp Oracle that gives monotonically increasing timestamps based on world time
+ */
+@Singleton
+public class WorldClockOracleImpl implements TimestampOracle {
+
+    private static final Logger LOG = LoggerFactory.getLogger(WorldClockOracleImpl.class);
+
+    static final long MAX_TX_PER_MS = 1_000_000; // 1 million
+    static final long TIMESTAMP_INTERVAL_MS = 10_000; // 10 seconds interval
+    private static final long TIMESTAMP_ALLOCATION_INTERVAL_MS = 7_000; // 7 seconds
+
+    private long lastTimestamp;
+    private long maxTimestamp;
+
+    private TimestampStorage storage;
+    private Panicker panicker;
+
+    private volatile long maxAllocatedTime;
+
+    private final ScheduledExecutorService scheduler =
+            Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder().setNameFormat("ts-persist-%d").build());
+
+    private Runnable allocateTimestampsBatchTask;
+
+    private class AllocateTimestampBatchTask implements Runnable {
+        long previousMaxTime;
+
+        AllocateTimestampBatchTask(long previousMaxTime) {
+            this.previousMaxTime = previousMaxTime;
+        }
+
+        @Override
+        public void run() {
+            long newMaxTime = (System.currentTimeMillis() + TIMESTAMP_INTERVAL_MS) * MAX_TX_PER_MS;
+            try {
+                storage.updateMaxTimestamp(previousMaxTime, newMaxTime);
+                maxAllocatedTime = newMaxTime;
+                previousMaxTime = newMaxTime;
+            } catch (Throwable e) {
+                panicker.panic("Can't store the new max timestamp", e);
+            }
+        }
+    }
+
+    @Inject
+    public WorldClockOracleImpl(MetricsRegistry metrics,
+                               TimestampStorage tsStorage,
+                               Panicker panicker) throws IOException {
+
+        this.storage = tsStorage;
+        this.panicker = panicker;
+
+        metrics.gauge(name("tso", "maxTimestamp"), new Gauge<Long>() {
+            @Override
+            public Long getValue() {
+                return maxTimestamp;
+            }
+        });
+
+    }
+
+    @Override
+    public void initialize() throws IOException {
+
+        this.lastTimestamp = this.maxTimestamp = storage.getMaxTimestamp();
+
+        this.allocateTimestampsBatchTask = new AllocateTimestampBatchTask(lastTimestamp);
+
+        // Trigger first allocation of timestamps
+        scheduler.schedule(allocateTimestampsBatchTask, 0, TimeUnit.MILLISECONDS);
+
+        // Waiting for the current epoch to start. Occurs in case of failover when the previous TSO allocated the current time frame.
+        while ((System.currentTimeMillis() * MAX_TX_PER_MS) < this.lastTimestamp) {
+            try {
+                Thread.sleep(1000);
+            } catch (InterruptedException e) {
+               continue;
+            }
+        }
+
+        // Launch the periodic timestamp interval allocation. In this case, the timestamp interval is extended even though the TSO is idle.
+        // Because we are world time based, this guarantees that the first request after a long time does not need to wait for new interval allocation.
+        scheduler.scheduleAtFixedRate(allocateTimestampsBatchTask, TIMESTAMP_ALLOCATION_INTERVAL_MS, TIMESTAMP_ALLOCATION_INTERVAL_MS, TimeUnit.MILLISECONDS);
+    }
+
+    /**
+     * Returns the next timestamp if available. Otherwise spins till the ts-persist thread allocates a new timestamp.
+     */
+    @Override
+    public long next() {
+
+        long currentMsFirstTimestamp = System.currentTimeMillis() * MAX_TX_PER_MS;
+
+        // Return the next timestamp in case we are still in the same millisecond as the previous timestamp was. 
+        if (++lastTimestamp >= currentMsFirstTimestamp) {
+            return lastTimestamp;
+        }
+
+        if (currentMsFirstTimestamp >= maxTimestamp) { // Intentional race to reduce synchronization overhead in every access to maxTimestamp                                                                                                                       
+            while (maxAllocatedTime <= currentMsFirstTimestamp) { // Waiting for the interval allocation
+                try {
+                    Thread.sleep(1000);
+                } catch (InterruptedException e) {
+                   continue;
+                }
+            }
+            assert (maxAllocatedTime > maxTimestamp);
+            maxTimestamp = maxAllocatedTime;
+        }
+
+        lastTimestamp = currentMsFirstTimestamp;
+
+        return lastTimestamp;
+    }
+
+    @Override
+    public long getLast() {
+        return lastTimestamp;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("TimestampOracle -> LastTimestamp: %d, MaxTimestamp: %d", lastTimestamp, maxTimestamp);
+    }
+
+    @VisibleForTesting
+    static class InMemoryTimestampStorage implements TimestampStorage {
+
+        long maxTime = 0;
+
+        @Override
+        public void updateMaxTimestamp(long previousMaxTime, long nextMaxTime) {
+            maxTime = nextMaxTime;
+            LOG.info("Updating max timestamp: (previous:{}, new:{})", previousMaxTime, nextMaxTime);
+        }
+
+        @Override
+        public long getMaxTimestamp() {
+            return maxTime;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/60c9e714/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
index 017af4f..9c94488 100644
--- a/tso-server/src/main/resources/default-omid-server-configuration.yml
+++ b/tso-server/src/main/resources/default-omid-server-configuration.yml
@@ -26,6 +26,10 @@ numConcurrentCTWriters: 2
 batchSizePerCTWriter: 25
 # When this timeout expires, the contents of the batch are flushed to the datastore
 batchPersistTimeoutInMs: 10
+# Timestamp generation strategy
+# INCREMENTAL - [Default] regular counter
+# WORLD_TIME - world time based counter
+timestampType: INCREMENTAL
 
 # Default module configuration (No TSO High Availability & in-memory storage for timestamp and commit tables)
 timestampStoreModule: !!org.apache.omid.tso.InMemoryTimestampStorageModule [ ]

http://git-wip-us.apache.org/repos/asf/incubator-omid/blob/60c9e714/tso-server/src/test/java/org/apache/omid/tso/TestWorldTimeOracle.java
----------------------------------------------------------------------
diff --git a/tso-server/src/test/java/org/apache/omid/tso/TestWorldTimeOracle.java b/tso-server/src/test/java/org/apache/omid/tso/TestWorldTimeOracle.java
new file mode 100644
index 0000000..825646c
--- /dev/null
+++ b/tso-server/src/test/java/org/apache/omid/tso/TestWorldTimeOracle.java
@@ -0,0 +1,117 @@
+/*
+ * 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.apache.omid.metrics.MetricsRegistry;
+import org.apache.omid.timestamp.storage.TimestampStorage;
+import org.mockito.InjectMocks;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.io.IOException;
+import java.util.concurrent.CountDownLatch;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+public class TestWorldTimeOracle {
+
+    private static final Logger LOG = LoggerFactory.getLogger(TestWorldTimeOracle.class);
+
+    @Mock
+    private MetricsRegistry metrics;
+    @Mock
+    private Panicker panicker;
+    @Mock
+    private TimestampStorage timestampStorage;
+
+    // Component under test
+    @InjectMocks
+    private WorldClockOracleImpl worldClockOracle;
+
+    @BeforeMethod(alwaysRun = true, timeOut = 30_000)
+    public void initMocksAndComponents() {
+        MockitoAnnotations.initMocks(this);
+    }
+
+    @Test(timeOut = 30_000)
+    public void testMonotonicTimestampGrowth() throws Exception {
+
+        // Intialize component under test
+        worldClockOracle.initialize();
+
+        long last = worldClockOracle.next();
+        
+        int timestampIntervalSec = (int) (WorldClockOracleImpl.TIMESTAMP_INTERVAL_MS / 1000) * 2;
+        for (int i = 0; i < timestampIntervalSec; i++) {
+            long current = worldClockOracle.next();
+            assertTrue(current > last+1 , "Timestamp should be based on world time");
+            last = current;
+            Thread.sleep(1000);
+        }
+
+        assertTrue(worldClockOracle.getLast() == last);
+        LOG.info("Last timestamp: {}", last);
+    }
+
+    @Test(timeOut = 10_000)
+    public void testTimestampOraclePanicsWhenTheStorageHasProblems() throws Exception {
+
+        // Intialize component under test
+        worldClockOracle.initialize();
+
+        // Cause an exception when updating the max timestamp
+        final CountDownLatch updateMaxTimestampMethodCalled = new CountDownLatch(1);
+        doAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+                updateMaxTimestampMethodCalled.countDown();
+                throw new RuntimeException("Out of memory or something");
+            }
+        }).when(timestampStorage).updateMaxTimestamp(anyLong(), anyLong());
+
+        // Make the previous exception to be thrown
+        Thread allocThread = new Thread("AllocThread") {
+            @Override
+            public void run() {
+                while (true) {
+                    worldClockOracle.next();
+                }
+            }
+        };
+        allocThread.start();
+
+        updateMaxTimestampMethodCalled.await();
+
+        // Verify that it has blown up
+        verify(panicker, atLeastOnce()).panic(anyString(), any(Throwable.class));
+    }
+
+}