PHOENIX-4720 SequenceIT is flapping

Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/87564a86
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/87564a86
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/87564a86

Branch: refs/heads/5.x-HBase-2.0
Commit: 87564a8647705d85921126ae538ba922a4c69037
Parents: ec5929a
Author: James Taylor <jtay...@salesforce.com>
Authored: Mon Apr 30 19:50:34 2018 -0700
Committer: James Taylor <jtay...@salesforce.com>
Committed: Tue May 1 11:46:13 2018 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/SequenceIT.java  | 42 +++++++++++++++++++-
 .../coprocessor/SequenceRegionObserver.java     |  3 +-
 2 files changed, 43 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/87564a86/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
index 9b870e1..4cc9628 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
@@ -41,6 +41,7 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.SchemaNotFoundException;
 import org.apache.phoenix.schema.SequenceAlreadyExistsException;
 import org.apache.phoenix.schema.SequenceNotFoundException;
+import org.apache.phoenix.util.EnvironmentEdge;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -90,6 +91,19 @@ public class SequenceIT extends ParallelStatsDisabledIT {
                assertTrue(rs.next());
        }
 
+    private static class MyClock extends EnvironmentEdge {
+        public volatile long time;
+
+        public MyClock (long time) {
+            this.time = time;
+        }
+
+        @Override
+        public long currentTime() {
+            return time;
+        }
+    }
+
        @Test
        public void testDuplicateSequences() throws Exception {
         String sequenceName = generateSequenceNameWithSchema();
@@ -105,7 +119,28 @@ public class SequenceIT extends ParallelStatsDisabledIT {
                }
        }
 
-       @Test
+    @Test
+    public void testDuplicateSequencesAtSameTimestamp() throws Exception {
+        final MyClock clock = new MyClock(1000);
+        EnvironmentEdgeManager.injectEdge(clock);
+        try {
+            String sequenceName = generateSequenceNameWithSchema();
+            
+            
+            conn.createStatement().execute("CREATE SEQUENCE " + sequenceName + 
" START WITH 2 INCREMENT BY 4\n");
+    
+            try {
+                conn.createStatement().execute("CREATE SEQUENCE " + 
sequenceName + " START WITH 2 INCREMENT BY 4\n");
+                Assert.fail("Duplicate sequences");
+            } catch (SequenceAlreadyExistsException e){
+    
+            }
+        } finally {
+            EnvironmentEdgeManager.reset();
+        }
+    }
+
+    @Test
        public void testSequenceNotFound() throws Exception {
         String sequenceName = generateSequenceNameWithSchema();
                
@@ -753,26 +788,31 @@ public class SequenceIT extends ParallelStatsDisabledIT {
         assertSequenceValuesForSingleRow(sequenceName, 1, 2, 3);
         conn.createStatement().execute("DROP SEQUENCE " + sequenceName);
         
+        sequenceName = generateSequenceNameWithSchema();
         conn.createStatement().execute("CREATE SEQUENCE " + sequenceName + " 
INCREMENT BY -1");
         
         assertSequenceValuesForSingleRow(sequenceName, 1, 0, -1);
         conn.createStatement().execute("DROP SEQUENCE " + sequenceName);
         
+        sequenceName = generateSequenceNameWithSchema();
         conn.createStatement().execute("CREATE SEQUENCE " + sequenceName + " 
MINVALUE 10");
         
         assertSequenceValuesForSingleRow(sequenceName, 10, 11, 12);
         conn.createStatement().execute("DROP SEQUENCE " + sequenceName);
         
+        sequenceName = generateSequenceNameWithSchema();
         conn.createStatement().execute("CREATE SEQUENCE " + sequenceName + " 
INCREMENT BY -1 MINVALUE 10 ");
         
         assertSequenceValuesForSingleRow(sequenceName, Long.MAX_VALUE, 
Long.MAX_VALUE - 1, Long.MAX_VALUE - 2);
         conn.createStatement().execute("DROP SEQUENCE " + sequenceName);
         
+        sequenceName = generateSequenceNameWithSchema();
         conn.createStatement().execute("CREATE SEQUENCE " + sequenceName + " 
MAXVALUE 0");
         
         assertSequenceValuesForSingleRow(sequenceName, Long.MIN_VALUE, 
Long.MIN_VALUE + 1, Long.MIN_VALUE + 2);
         conn.createStatement().execute("DROP SEQUENCE " + sequenceName);
         
+        sequenceName = generateSequenceNameWithSchema();
         conn.createStatement().execute("CREATE SEQUENCE " + sequenceName + " 
INCREMENT BY -1 MAXVALUE 0");
         
         assertSequenceValuesForSingleRow(sequenceName, 0, -1, -2);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/87564a86/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
index 4c2ec1a..d6c9cbc 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
@@ -364,7 +364,8 @@ public class SequenceRegionObserver implements 
RegionObserver, RegionCoprocessor
                     maxGetTimestamp = clientTimestamp + 1;
                 }            
             } else {
-                clientTimestamp = maxGetTimestamp = 
EnvironmentEdgeManager.currentTimeMillis();
+                clientTimestamp = EnvironmentEdgeManager.currentTimeMillis();
+                maxGetTimestamp = clientTimestamp + 1;
                 clientTimestampBuf = Bytes.toBytes(clientTimestamp);
             }
         }

Reply via email to