Repository: phoenix Updated Branches: refs/heads/4.x-HBase-1.2 13bb73031 -> 78666ee40
PHOENIX-4204 Modify SequenceBulkAllocationIT.java to not use currentSCN Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/78666ee4 Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/78666ee4 Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/78666ee4 Branch: refs/heads/4.x-HBase-1.2 Commit: 78666ee40964af5054a1eba7dc53997e82946b2a Parents: 13bb730 Author: Mujtaba <mujt...@apache.org> Authored: Fri Sep 15 10:54:51 2017 -0700 Committer: Mujtaba <mujt...@apache.org> Committed: Fri Sep 15 10:54:51 2017 -0700 ---------------------------------------------------------------------- .../end2end/SequenceBulkAllocationIT.java | 538 +++++++++---------- 1 file changed, 258 insertions(+), 280 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/78666ee4/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceBulkAllocationIT.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceBulkAllocationIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceBulkAllocationIT.java index c057f49..2c2df73 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceBulkAllocationIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceBulkAllocationIT.java @@ -39,7 +39,9 @@ import org.apache.phoenix.util.PhoenixRuntime; import org.apache.phoenix.util.PropertiesUtil; import org.apache.phoenix.util.QueryUtil; import org.apache.phoenix.util.ReadOnlyProps; +import org.apache.phoenix.util.SchemaUtil; import org.junit.After; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; @@ -47,6 +49,7 @@ import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; /** * Suite of integration tests that validate that Bulk Allocation of Sequence values @@ -57,19 +60,19 @@ import com.google.common.collect.Lists; * */ @RunWith(Parameterized.class) -public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { +public class SequenceBulkAllocationIT extends ParallelStatsDisabledIT { private static final long BATCH_SIZE = 3; private static final String SELECT_NEXT_VALUE_SQL = - "SELECT NEXT VALUE FOR %s FROM \"SYSTEM\".\"SEQUENCE\" LIMIT 1"; + "SELECT NEXT VALUE FOR %s"; private static final String SELECT_CURRENT_VALUE_SQL = - "SELECT CURRENT VALUE FOR %s FROM \"SYSTEM\".\"SEQUENCE\" LIMIT 1"; + "SELECT CURRENT VALUE FOR %s"; private static final String CREATE_SEQUENCE_NO_MIN_MAX_TEMPLATE = - "CREATE SEQUENCE bulkalloc.alpha START WITH %s INCREMENT BY %s CACHE %s"; + "CREATE SEQUENCE %s START WITH %s INCREMENT BY %s CACHE %s"; private static final String CREATE_SEQUENCE_WITH_MIN_MAX_TEMPLATE = - "CREATE SEQUENCE bulkalloc.alpha START WITH %s INCREMENT BY %s MINVALUE %s MAXVALUE %s CACHE %s"; + "CREATE SEQUENCE %s START WITH %s INCREMENT BY %s MINVALUE %s MAXVALUE %s CACHE %s"; private static final String CREATE_SEQUENCE_WITH_MIN_MAX_AND_CYCLE_TEMPLATE = - "CREATE SEQUENCE bulkalloc.alpha START WITH %s INCREMENT BY %s MINVALUE %s MAXVALUE %s CYCLE CACHE %s"; + "CREATE SEQUENCE %s START WITH %s INCREMENT BY %s MINVALUE %s MAXVALUE %s CYCLE CACHE %s"; private Connection conn; @@ -78,16 +81,29 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { public SequenceBulkAllocationIT(String tenantId) { this.tenantId = tenantId; } + + @Parameters(name="SequenceBulkAllocationIT_tenantId={0}") // name is used by failsafe as file name in reports + public static Object[] data() { + return new Object[] {null, "tenant1"}; + } + + private static String generateNameWithSchema() { + return SchemaUtil.getTableName(generateUniqueName(), generateUniqueName()); + } @BeforeClass - @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class) + @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class) public static void doSetup() throws Exception { - Map<String, String> props = getDefaultProps(); - // Must update config before starting server + Map<String, String> props = Maps.newHashMap(); props.put(QueryServices.SEQUENCE_CACHE_SIZE_ATTRIB, Long.toString(BATCH_SIZE)); setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator())); } + @Before + public void init() throws Exception { + createConnection(); + } + @After public void tearDown() throws Exception { // close any open connection between tests, so that connections are not leaked @@ -96,18 +112,12 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { } } - @Parameters(name="SequenceBulkAllocationIT_tenantId={0}") // name is used by failsafe as file name in reports - public static Object[] data() { - return new Object[] {null, "tenant1"}; - } - - @Test public void testSequenceParseNextValuesWithNull() throws Exception { - nextConnection(); + String sequenceName = generateNameWithSchema(); try { conn.createStatement().executeQuery( - "SELECT NEXT NULL VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\" LIMIT 1"); + "SELECT NEXT NULL VALUES FOR " + sequenceName); fail("null is not allowed to be used for <n> in NEXT <n> VALUES FOR <seq>"); } catch (SQLException e) { assertEquals(SQLExceptionCode.NUM_SEQ_TO_ALLOCATE_MUST_BE_CONSTANT.getErrorCode(), @@ -118,10 +128,10 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { @Test public void testSequenceParseNextValuesWithNonNumber() throws Exception { - nextConnection(); + String sequenceName = generateNameWithSchema(); try { conn.createStatement().executeQuery( - "SELECT NEXT '89b' VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\" LIMIT 1"); + "SELECT NEXT '89b' VALUES FOR " + sequenceName); fail("Only integers and longs are allowed to be used for <n> in NEXT <n> VALUES FOR <seq>"); } catch (SQLException e) { assertEquals(SQLExceptionCode.NUM_SEQ_TO_ALLOCATE_MUST_BE_CONSTANT.getErrorCode(), @@ -133,10 +143,10 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { @Test public void testSequenceParseNextValuesWithNegativeNumber() throws Exception { - nextConnection(); + String sequenceName = generateNameWithSchema(); try { conn.createStatement().executeQuery( - "SELECT NEXT '-1' VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\" LIMIT 1"); + "SELECT NEXT '-1' VALUES FOR " + sequenceName); fail("null is not allowed to be used for <n> in NEXT <n> VALUES FOR <seq>"); } catch (SQLException e) { assertEquals(SQLExceptionCode.NUM_SEQ_TO_ALLOCATE_MUST_BE_CONSTANT.getErrorCode(), @@ -147,10 +157,10 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { @Test public void testParseNextValuesSequenceWithZeroAllocated() throws Exception { - nextConnection(); + String sequenceName = generateNameWithSchema(); try { conn.createStatement().executeQuery( - "SELECT NEXT 0 VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\" LIMIT 1"); + "SELECT NEXT 0 VALUES FOR " + sequenceName); fail("Only integers and longs are allowed to be used for <n> in NEXT <n> VALUES FOR <seq>"); } catch (SQLException e) { assertEquals(SQLExceptionCode.NUM_SEQ_TO_ALLOCATE_MUST_BE_CONSTANT.getErrorCode(), @@ -162,24 +172,22 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { @Test public void testNextValuesForSequenceWithNoAllocatedValues() throws Exception { - // Create Sequence + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(1).startsWith(1).cacheSize(1) + new SequenceProperties.Builder().name(sequenceName).incrementBy(1).startsWith(1).cacheSize(1) .numAllocated(100).build(); - nextConnection(); createSequenceWithNoMinMax(props); - nextConnection(); // Bulk Allocate Sequence Slots final int currentValueAfterAllocation = 100; - reserveSlotsInBulkAndAssertValue(1, props.numAllocated); + reserveSlotsInBulkAndAssertValue(sequenceName, 1, props.numAllocated); assertExpectedStateInSystemSequence(props, 101); assertExpectedNumberOfValuesAllocated(1, currentValueAfterAllocation, props.incrementBy, props.numAllocated); // Assert standard Sequence Operations return expected values - assertExpectedCurrentValueForSequence(currentValueAfterAllocation); - assertExpectedNextValueForSequence(101); + assertExpectedCurrentValueForSequence(sequenceName, currentValueAfterAllocation); + assertExpectedNextValueForSequence(sequenceName, 101); } @Test @@ -187,52 +195,48 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * Validates we can invoke NEXT <n> VALUES FOR using bind vars. */ public void testNextValuesForSequenceUsingBinds() throws Exception { - // Create Sequence + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(1).startsWith(1).cacheSize(1) + new SequenceProperties.Builder().name(sequenceName).incrementBy(1).startsWith(1).cacheSize(1) .numAllocated(100).build(); - nextConnection(); createSequenceWithNoMinMax(props); - nextConnection(); // Allocate 100 slots using SQL with Bind Params and a PreparedStatement final int currentValueAfterAllocation = 100; - reserveSlotsInBulkUsingBindsAndAssertValue(1,props.numAllocated); + reserveSlotsInBulkUsingBindsAndAssertValue(sequenceName, 1,props.numAllocated); assertExpectedStateInSystemSequence(props, 101); assertExpectedNumberOfValuesAllocated(1, currentValueAfterAllocation, props.incrementBy, props.numAllocated); // Assert standard Sequence Operations return expected values - assertExpectedCurrentValueForSequence(currentValueAfterAllocation); - assertExpectedNextValueForSequence(101); + assertExpectedCurrentValueForSequence(sequenceName, currentValueAfterAllocation); + assertExpectedNextValueForSequence(sequenceName, 101); } @Test public void testNextValuesForSequenceWithPreviouslyAllocatedValues() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(1).startsWith(1).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(1).startsWith(1).cacheSize(100) .numAllocated(1000).build(); - nextConnection(); createSequenceWithNoMinMax(props); - nextConnection(); - - assertExpectedNextValueForSequence(1); - assertExpectedCurrentValueForSequence(1); - assertExpectedNextValueForSequence(2); + assertExpectedNextValueForSequence(sequenceName, 1); + assertExpectedCurrentValueForSequence(sequenceName, 1); + assertExpectedNextValueForSequence(sequenceName, 2); // Bulk Allocate Sequence Slots int currentValueAfterAllocation = 1100; int nextValueAfterAllocation = currentValueAfterAllocation + props.incrementBy; int startValueAfterAllocation = 101; - reserveSlotsInBulkAndAssertValue(startValueAfterAllocation, props.numAllocated); + reserveSlotsInBulkAndAssertValue(sequenceName, startValueAfterAllocation, props.numAllocated); assertBulkAllocationSucceeded(props, currentValueAfterAllocation, startValueAfterAllocation); // Assert standard Sequence Operations return expected values - assertExpectedCurrentValueForSequence(currentValueAfterAllocation); - assertExpectedNextValueForSequence(nextValueAfterAllocation); + assertExpectedCurrentValueForSequence(sequenceName, currentValueAfterAllocation); + assertExpectedNextValueForSequence(sequenceName, nextValueAfterAllocation); } @@ -243,31 +247,30 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * the latest batch. */ public void testConnectionCloseReturnsSequenceValuesCorrectly() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(2).startsWith(1).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(2).startsWith(1).cacheSize(100) .numAllocated(100).build(); - nextConnection(); createSequenceWithNoMinMax(props); - nextConnection(); - assertExpectedNextValueForSequence(1); - assertExpectedCurrentValueForSequence(1); - assertExpectedNextValueForSequence(3); + assertExpectedNextValueForSequence(sequenceName, 1); + assertExpectedCurrentValueForSequence(sequenceName, 1); + assertExpectedNextValueForSequence(sequenceName, 3); // Bulk Allocate Sequence Slots int currentValueAfterAllocation = 399; int nextValueAfterAllocation = currentValueAfterAllocation + props.incrementBy; int startValueAfterAllocation = 201; - reserveSlotsInBulkAndAssertValue(startValueAfterAllocation, props.numAllocated); - assertExpectedCurrentValueForSequence(currentValueAfterAllocation); + reserveSlotsInBulkAndAssertValue(sequenceName, startValueAfterAllocation, props.numAllocated); + assertExpectedCurrentValueForSequence(sequenceName, currentValueAfterAllocation); // Close the Connection conn.close(); // Test that sequence, doesn't have gaps after closing the connection - nextConnection(); - assertExpectedNextValueForSequence(nextValueAfterAllocation); - assertExpectedCurrentValueForSequence(nextValueAfterAllocation); + createConnection(); + assertExpectedNextValueForSequence(sequenceName, nextValueAfterAllocation); + assertExpectedCurrentValueForSequence(sequenceName, nextValueAfterAllocation); } @@ -276,34 +279,33 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * Validates that calling NEXT <n> VALUES FOR <seq> works correctly with UPSERT. */ public void testNextValuesForSequenceWithUpsert() throws Exception { - + String sequenceName = generateNameWithSchema(); + String tableName = generateUniqueName(); + // Create Sequence final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(1).startsWith(1).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(1).startsWith(1).cacheSize(100) .numAllocated(1000).build(); - nextConnection(); createSequenceWithNoMinMax(props); // Create TABLE - nextGenericConnection(); - conn.createStatement().execute("CREATE TABLE bulkalloc.test ( id INTEGER NOT NULL PRIMARY KEY)"); - nextConnection(); + Connection genericConn = createGenericConnection(); + genericConn.createStatement().execute("CREATE TABLE " + tableName + " ( id INTEGER NOT NULL PRIMARY KEY)"); + genericConn.close(); // Grab batch from Sequence - assertExpectedNextValueForSequence(1); - assertExpectedCurrentValueForSequence(1); - assertExpectedNextValueForSequence(2); + assertExpectedNextValueForSequence(sequenceName, 1); + assertExpectedCurrentValueForSequence(sequenceName, 1); + assertExpectedNextValueForSequence(sequenceName, 2); assertExpectedStateInSystemSequence(props, 101); - // Perform UPSERT and validate Sequence was incremented as expected - conn.createStatement().execute("UPSERT INTO bulkalloc.test (id) VALUES (NEXT " + props.numAllocated + " VALUES FOR bulkalloc.alpha)"); + conn.createStatement().execute("UPSERT INTO " + tableName + " (id) VALUES (NEXT " + props.numAllocated + " VALUES FOR " + sequenceName + " )"); conn.commit(); assertExpectedStateInSystemSequence(props, 1101); // SELECT values out and verify - nextConnection(); - String query = "SELECT id, NEXT VALUE FOR bulkalloc.alpha FROM bulkalloc.test"; + String query = "SELECT id, NEXT VALUE FOR " + sequenceName + " FROM " + tableName; ResultSet rs = conn.prepareStatement(query).executeQuery(); assertTrue(rs.next()); assertEquals(101, rs.getInt(1)); // Threw out cache of 100, incremented by 1000 @@ -316,77 +318,74 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { @Test public void testNextValuesForSequenceWithIncrementBy() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(3).startsWith(1).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(3).startsWith(1).cacheSize(100) .numAllocated(1000).build(); - nextConnection(); createSequenceWithNoMinMax(props); - nextConnection(); - assertExpectedNextValueForSequence(1); - assertExpectedCurrentValueForSequence(1); - assertExpectedNextValueForSequence(4); + assertExpectedNextValueForSequence(sequenceName, 1); + assertExpectedCurrentValueForSequence(sequenceName, 1); + assertExpectedNextValueForSequence(sequenceName, 4); // Bulk Allocate Sequence Slots int currentValueAfterAllocation = 3298; int startValueAfterAllocation = 301; - reserveSlotsInBulkAndAssertValue(startValueAfterAllocation, props.numAllocated); + reserveSlotsInBulkAndAssertValue(sequenceName, startValueAfterAllocation, props.numAllocated); assertBulkAllocationSucceeded(props, currentValueAfterAllocation, startValueAfterAllocation); // Assert standard Sequence Operations return expected values - assertExpectedCurrentValueForSequence(3298); - assertExpectedNextValueForSequence(3301); + assertExpectedCurrentValueForSequence(sequenceName, 3298); + assertExpectedNextValueForSequence(sequenceName, 3301); } @Test public void testNextValuesForSequenceWithNegativeIncrementBy() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(-1).startsWith(2000).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(-1).startsWith(2000).cacheSize(100) .numAllocated(1000).build(); - nextConnection(); createSequenceWithNoMinMax(props); - nextConnection(); - assertExpectedNextValueForSequence(2000); - assertExpectedCurrentValueForSequence(2000); - assertExpectedNextValueForSequence(1999); + assertExpectedNextValueForSequence(sequenceName, 2000); + assertExpectedCurrentValueForSequence(sequenceName, 2000); + assertExpectedNextValueForSequence(sequenceName, 1999); // Bulk Allocate Sequence Slots int currentValueAfterAllocation = 901; int startValueAfterAllocation = 1900; - reserveSlotsInBulkAndAssertValue(startValueAfterAllocation, props.numAllocated); + reserveSlotsInBulkAndAssertValue(sequenceName, startValueAfterAllocation, props.numAllocated); assertBulkAllocationSucceeded(props, currentValueAfterAllocation, startValueAfterAllocation); // Assert standard Sequence Operations return expected values - assertExpectedCurrentValueForSequence(901); - assertExpectedNextValueForSequence(900); + assertExpectedCurrentValueForSequence(sequenceName, 901); + assertExpectedNextValueForSequence(sequenceName, 900); } @Test public void testNextValuesForSequenceWithNegativeIncrementByGreaterThanOne() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(-5).startsWith(2000).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(-5).startsWith(2000).cacheSize(100) .numAllocated(100).build(); - nextConnection(); createSequenceWithNoMinMax(props); - nextConnection(); // Pull first batch from Sequence - assertExpectedNextValueForSequence(2000); - assertExpectedCurrentValueForSequence(2000); - assertExpectedNextValueForSequence(1995); + assertExpectedNextValueForSequence(sequenceName, 2000); + assertExpectedCurrentValueForSequence(sequenceName, 2000); + assertExpectedNextValueForSequence(sequenceName, 1995); // Bulk Allocate Sequence Slots int currentValueAfterAllocation = 1005; int startValueAfterAllocation = 1500; - reserveSlotsInBulkAndAssertValue(startValueAfterAllocation, props.numAllocated); + reserveSlotsInBulkAndAssertValue(sequenceName, startValueAfterAllocation, props.numAllocated); assertBulkAllocationSucceeded(props, currentValueAfterAllocation, startValueAfterAllocation); // Assert standard Sequence Operations return expected values - assertExpectedCurrentValueForSequence(1005); - assertExpectedNextValueForSequence(1000); + assertExpectedCurrentValueForSequence(sequenceName, 1005); + assertExpectedNextValueForSequence(sequenceName, 1000); } @@ -398,24 +397,23 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * have access to all slots requested. */ public void testNextValuesForSequenceExceedsMaxValue() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties sequenceProps = - new SequenceProperties.Builder().incrementBy(1).startsWith(100).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(1).startsWith(100).cacheSize(100) .numAllocated(1000).minValue(100).maxValue(900).build(); - nextConnection(); createSequenceWithMinMax(sequenceProps); - nextConnection(); // Pull first batch from the sequence - assertExpectedNextValueForSequence(100); - assertExpectedCurrentValueForSequence(100); - assertExpectedNextValueForSequence(101); + assertExpectedNextValueForSequence(sequenceName, 100); + assertExpectedCurrentValueForSequence(sequenceName, 100); + assertExpectedNextValueForSequence(sequenceName, 101); // Attempt to bulk Allocate more slots than available try { conn.createStatement().executeQuery( "SELECT NEXT " + sequenceProps.numAllocated - + " VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\" LIMIT 1"); + + " VALUES FOR " + sequenceName + " LIMIT 1"); fail("Invoking SELECT NEXT VALUES should have thrown Reached Max Value Exception"); } catch (SQLException e) { assertEquals(SQLExceptionCode.SEQUENCE_VAL_REACHED_MAX_VALUE.getErrorCode(), @@ -424,8 +422,8 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { } // Assert sequence didn't advance - assertExpectedCurrentValueForSequence(101); - assertExpectedNextValueForSequence(102); + assertExpectedCurrentValueForSequence(sequenceName, 101); + assertExpectedNextValueForSequence(sequenceName, 102); } @Test @@ -436,24 +434,23 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * have access to all slots requested. */ public void testNextValuesForSequenceExceedsMinValue() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties sequenceProps = - new SequenceProperties.Builder().incrementBy(-5).startsWith(900).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(-5).startsWith(900).cacheSize(100) .numAllocated(160).minValue(100).maxValue(900).build(); - nextConnection(); createSequenceWithMinMax(sequenceProps); - nextConnection(); // Pull first batch from the sequence - assertExpectedNextValueForSequence(900); - assertExpectedCurrentValueForSequence(900); - assertExpectedNextValueForSequence(895); + assertExpectedNextValueForSequence(sequenceName, 900); + assertExpectedCurrentValueForSequence(sequenceName, 900); + assertExpectedNextValueForSequence(sequenceName, 895); // Attempt to bulk Allocate more slots than available try { conn.createStatement().executeQuery( "SELECT NEXT " + sequenceProps.numAllocated - + " VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\" LIMIT 1"); + + " VALUES FOR " + sequenceName + " LIMIT 1"); fail("Invoking SELECT NEXT VALUES should have thrown Reached Max Value Exception"); } catch (SQLException e) { assertEquals(SQLExceptionCode.SEQUENCE_VAL_REACHED_MIN_VALUE.getErrorCode(), @@ -462,8 +459,8 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { } // Assert sequence didn't advance (we still throw out the cached values) - assertExpectedCurrentValueForSequence(895); - assertExpectedNextValueForSequence(890); + assertExpectedCurrentValueForSequence(sequenceName, 895); + assertExpectedNextValueForSequence(sequenceName, 890); } @@ -473,49 +470,46 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * min and max defined. */ public void testNextValuesForSequenceWithMinMaxDefined() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(5).startsWith(100).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(5).startsWith(100).cacheSize(100) .numAllocated(1000).minValue(100).maxValue(6000).build(); - nextConnection(); createSequenceWithMinMax(props); - nextConnection(); - - assertExpectedNextValueForSequence(100); - assertExpectedCurrentValueForSequence(100); - assertExpectedNextValueForSequence(105); + assertExpectedNextValueForSequence(sequenceName, 100); + assertExpectedCurrentValueForSequence(sequenceName, 100); + assertExpectedNextValueForSequence(sequenceName, 105); // Bulk Allocate Sequence Slots int currentValueAfterAllocation = 5595; int startValueAfterAllocation = 600; - reserveSlotsInBulkAndAssertValue(startValueAfterAllocation, props.numAllocated); + reserveSlotsInBulkAndAssertValue(sequenceName, startValueAfterAllocation, props.numAllocated); assertBulkAllocationSucceeded(props, currentValueAfterAllocation, startValueAfterAllocation); // Assert standard Sequence Operations return expected values - assertExpectedCurrentValueForSequence(5595); - assertExpectedNextValueForSequence(5600); + assertExpectedCurrentValueForSequence(sequenceName, 5595); + assertExpectedNextValueForSequence(sequenceName, 5600); } @Test public void testNextValuesForSequenceWithDefaultMax() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(1).startsWith(100).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(1).startsWith(100).cacheSize(100) .numAllocated(Long.MAX_VALUE - 100).build(); // Create Sequence - nextConnection(); createSequenceWithMinMax(props); - nextConnection(); // Bulk Allocate Sequence Slots long currentValueAfterAllocation = 100; long startValueAfterAllocation = Long.MAX_VALUE; - reserveSlotsInBulkAndAssertValue(currentValueAfterAllocation, props.numAllocated); + reserveSlotsInBulkAndAssertValue(sequenceName, currentValueAfterAllocation, props.numAllocated); assertExpectedStateInSystemSequence(props, startValueAfterAllocation); // Try and get next value try { - conn.createStatement().executeQuery(String.format(SELECT_NEXT_VALUE_SQL, "bulkalloc.alpha")); + conn.createStatement().executeQuery(String.format(SELECT_NEXT_VALUE_SQL, sequenceName)); } catch (SQLException e) { assertEquals(SQLExceptionCode.SEQUENCE_VAL_REACHED_MAX_VALUE.getErrorCode(), e.getErrorCode()); @@ -530,20 +524,19 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * the expression is evaluated. */ public void testNextValuesForSequenceOverflowAllocation() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(1).startsWith(100).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(1).startsWith(100).cacheSize(100) .numAllocated(Long.MAX_VALUE).build(); // Create Sequence - nextConnection(); createSequenceWithMinMax(props); - nextConnection(); // Bulk Allocate Sequence Slots try { conn.createStatement().executeQuery( "SELECT NEXT " + Long.MAX_VALUE - + " VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\""); + + " VALUES FOR " + sequenceName + " "); } catch (SQLException e) { assertEquals(SQLExceptionCode.SEQUENCE_VAL_REACHED_MAX_VALUE.getErrorCode(), e.getErrorCode()); @@ -558,31 +551,30 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * as expected. */ public void testNextValuesForSequenceAllocationLessThanCacheSize() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(5).startsWith(100).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(5).startsWith(100).cacheSize(100) .numAllocated(50).minValue(100).maxValue(6000).build(); - nextConnection(); createSequenceWithMinMax(props); - nextConnection(); - assertExpectedNextValueForSequence(100); - assertExpectedCurrentValueForSequence(100); - assertExpectedNextValueForSequence(105); + assertExpectedNextValueForSequence(sequenceName, 100); + assertExpectedCurrentValueForSequence(sequenceName, 100); + assertExpectedNextValueForSequence(sequenceName, 105); // Bulk Allocate Sequence Slots int currentValueAfterAllocation = 355; int startValueAfterAllocation = 110; - reserveSlotsInBulkAndAssertValue(startValueAfterAllocation, props.numAllocated); + reserveSlotsInBulkAndAssertValue(sequenceName, startValueAfterAllocation, props.numAllocated); assertExpectedStateInSystemSequence(props, 600); assertExpectedNumberOfValuesAllocated(startValueAfterAllocation, currentValueAfterAllocation, props.incrementBy, props.numAllocated); // Assert standard Sequence Operations return expected values // 105 + (50 * 5) = 355 - assertExpectedCurrentValueForSequence(355); - assertExpectedNextValueForSequence(360); - assertExpectedNextValueForSequence(365); - assertExpectedNextValueForSequence(370); + assertExpectedCurrentValueForSequence(sequenceName, 355); + assertExpectedNextValueForSequence(sequenceName, 360); + assertExpectedNextValueForSequence(sequenceName, 365); + assertExpectedNextValueForSequence(sequenceName, 370); } @Test @@ -591,34 +583,33 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * as expected if we don't have enough values in the cache to support the allocation. */ public void testNextValuesForInsufficentCacheValuesAllocationLessThanCacheSize() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(5).startsWith(100).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(5).startsWith(100).cacheSize(100) .numAllocated(50).minValue(100).maxValue(6000).build(); - nextConnection(); createSequenceWithMinMax(props); - nextConnection(); // Allocate 51 slots, only 49 will be left int currentValueAfter51Allocations = 355; // 100 + 51 * 5 for (int i = 100; i <= currentValueAfter51Allocations; i = i + 5) { - assertExpectedNextValueForSequence(i); + assertExpectedNextValueForSequence(sequenceName, i); } - assertExpectedCurrentValueForSequence(currentValueAfter51Allocations); + assertExpectedCurrentValueForSequence(sequenceName, currentValueAfter51Allocations); // Bulk Allocate 50 Sequence Slots which greater than amount left in cache // This should throw away rest of the cache, and allocate the request slot // from the next start value int currentValueAfterAllocation = 845; int startValueAfterAllocation = 600; - reserveSlotsInBulkAndAssertValue(startValueAfterAllocation, props.numAllocated); + reserveSlotsInBulkAndAssertValue(sequenceName, startValueAfterAllocation, props.numAllocated); assertBulkAllocationSucceeded(props, currentValueAfterAllocation, startValueAfterAllocation); // Assert standard Sequence Operations return expected values - assertExpectedCurrentValueForSequence(845); - assertExpectedNextValueForSequence(850); - assertExpectedNextValueForSequence(855); - assertExpectedNextValueForSequence(860); + assertExpectedCurrentValueForSequence(sequenceName, 845); + assertExpectedNextValueForSequence(sequenceName, 850); + assertExpectedNextValueForSequence(sequenceName, 855); + assertExpectedNextValueForSequence(sequenceName, 860); } @Test @@ -627,24 +618,23 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * CYCLE flag set to true. */ public void testNextValuesForSequenceWithCycles() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties sequenceProps = - new SequenceProperties.Builder().incrementBy(5).startsWith(100).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(5).startsWith(100).cacheSize(100) .numAllocated(1000).minValue(100).maxValue(900).build(); - nextConnection(); createSequenceWithMinMaxAndCycle(sequenceProps); - nextConnection(); // Full first batch from the sequence - assertExpectedNextValueForSequence(100); - assertExpectedCurrentValueForSequence(100); - assertExpectedNextValueForSequence(105); + assertExpectedNextValueForSequence(sequenceName, 100); + assertExpectedCurrentValueForSequence(sequenceName, 100); + assertExpectedNextValueForSequence(sequenceName, 105); // Attempt to bulk Allocate more slots than available try { conn.createStatement().executeQuery( "SELECT NEXT " + sequenceProps.numAllocated - + " VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\" LIMIT 1"); + + " VALUES FOR " + sequenceName + " LIMIT 1"); fail("Invoking SELECT NEXT VALUES should have failed as operation is not supported for sequences with Cycles."); } catch (SQLException e) { assertEquals(SQLExceptionCode.NUM_SEQ_TO_ALLOCATE_NOT_SUPPORTED.getErrorCode(), @@ -653,9 +643,9 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { } // Assert sequence didn't advance - assertExpectedCurrentValueForSequence(105); - assertExpectedNextValueForSequence(110); - assertExpectedNextValueForSequence(115); + assertExpectedCurrentValueForSequence(sequenceName, 105); + assertExpectedNextValueForSequence(sequenceName, 110); + assertExpectedNextValueForSequence(sequenceName, 115); } @Test @@ -664,17 +654,16 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * CURRENT VALUE FOR expression work correctly when used in the same statement. */ public void testCurrentValueForAndNextValuesForExpressionsForSameSequence() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(1).startsWith(1).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(1).startsWith(1).cacheSize(100) .numAllocated(1000).build(); - nextConnection(); createSequenceWithNoMinMax(props); - nextConnection(); - assertExpectedNextValueForSequence(1); - assertExpectedCurrentValueForSequence(1); - assertExpectedNextValueForSequence(2); + assertExpectedNextValueForSequence(sequenceName, 1); + assertExpectedCurrentValueForSequence(sequenceName, 1); + assertExpectedNextValueForSequence(sequenceName, 2); // Bulk Allocate Sequence Slots int currentValueAfterAllocation = 1100; @@ -682,7 +671,7 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { int startValueAfterAllocation = 101; ResultSet rs = conn.createStatement().executeQuery( - "SELECT CURRENT VALUE FOR bulkalloc.alpha, NEXT " + props.numAllocated + " VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\""); + "SELECT CURRENT VALUE FOR " + sequenceName + " , NEXT " + props.numAllocated + " VALUES FOR " + sequenceName + " "); assertTrue(rs.next()); assertBulkAllocationSucceeded(props, currentValueAfterAllocation, startValueAfterAllocation); int currentValueFor = rs.getInt(1); @@ -691,8 +680,8 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { assertEquals("Expected current value to be the same as next value", startValueAfterAllocation, currentValueFor); // Assert standard Sequence Operations return expected values - assertExpectedCurrentValueForSequence(currentValueAfterAllocation); - assertExpectedNextValueForSequence(nextValueAfterAllocation); + assertExpectedCurrentValueForSequence(sequenceName, currentValueAfterAllocation); + assertExpectedNextValueForSequence(sequenceName, nextValueAfterAllocation); } @Test @@ -702,17 +691,16 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * value for that for all expressions. */ public void testMultipleNextValuesForExpressionsForSameSequence() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(1).startsWith(1).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(1).startsWith(1).cacheSize(100) .numAllocated(1000).build(); - nextConnection(); createSequenceWithNoMinMax(props); - nextConnection(); - assertExpectedNextValueForSequence(1); - assertExpectedCurrentValueForSequence(1); - assertExpectedNextValueForSequence(2); + assertExpectedNextValueForSequence(sequenceName, 1); + assertExpectedCurrentValueForSequence(sequenceName, 1); + assertExpectedNextValueForSequence(sequenceName, 2); // Bulk Allocate Sequence Slots - One for 5 and one for 1000, 1000 should have precedence int currentValueAfterAllocation = 1100; @@ -720,7 +708,7 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { int startValueAfterAllocation = 101; ResultSet rs = conn.createStatement().executeQuery( - "SELECT NEXT 5 VALUES FOR bulkalloc.alpha, NEXT " + props.numAllocated + " VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\""); + "SELECT NEXT 5 VALUES FOR " + sequenceName + " , NEXT " + props.numAllocated + " VALUES FOR " + sequenceName + " FROM \"SYSTEM\".\"SEQUENCE\""); assertTrue(rs.next()); int firstValue = rs.getInt(1); int secondValue = rs.getInt(2); @@ -729,8 +717,8 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { assertBulkAllocationSucceeded(props, currentValueAfterAllocation, startValueAfterAllocation); // Assert standard Sequence Operations return expected values - assertExpectedCurrentValueForSequence(currentValueAfterAllocation); - assertExpectedNextValueForSequence(nextValueAfterAllocation); + assertExpectedCurrentValueForSequence(sequenceName, currentValueAfterAllocation); + assertExpectedNextValueForSequence(sequenceName, nextValueAfterAllocation); } @Test @@ -740,16 +728,15 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * NEXT VALUE FOR <seq> is assumed to be 1. */ public void testMultipleDifferentExpressionsForSameSequence() throws Exception { + String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(1).startsWith(1).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(1).startsWith(1).cacheSize(100) .numAllocated(1000).build(); - nextConnection(); createSequenceWithNoMinMax(props); - nextConnection(); // Pull First Batch from Sequence - assertExpectedNextValueForSequence(1); + assertExpectedNextValueForSequence(sequenceName, 1); // Bulk Allocate Sequence Slots and Get Next Value in Same Statement int currentValueAfterAllocation = 1100; @@ -757,10 +744,10 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { int startValueAfterAllocation = 101; ResultSet rs = conn.createStatement().executeQuery( - "SELECT NEXT VALUE FOR bulkalloc.alpha, " - + "NEXT " + props.numAllocated + " VALUES FOR bulkalloc.alpha, " - + "CURRENT VALUE FOR bulkalloc.alpha, " - + "NEXT 999 VALUES FOR bulkalloc.alpha " + "SELECT NEXT VALUE FOR " + sequenceName + " , " + + "NEXT " + props.numAllocated + " VALUES FOR " + sequenceName + " , " + + "CURRENT VALUE FOR " + sequenceName + " , " + + "NEXT 999 VALUES FOR " + sequenceName + " " + "FROM \"SYSTEM\".\"SEQUENCE\""); assertTrue(rs.next()); assertBulkAllocationSucceeded(props, currentValueAfterAllocation, startValueAfterAllocation); @@ -779,8 +766,8 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { } // Assert standard Sequence Operations return expected values - assertExpectedCurrentValueForSequence(currentValueAfterAllocation); - assertExpectedNextValueForSequence(nextValueAfterAllocation); + assertExpectedCurrentValueForSequence(sequenceName, currentValueAfterAllocation); + assertExpectedNextValueForSequence(sequenceName, nextValueAfterAllocation); } @@ -792,29 +779,29 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * sync during the sequence management process. */ public void testMultipleNextValuesForExpressionsForDifferentSequences() throws Exception { + String sequenceName = generateNameWithSchema(); + String secondSequenceName = generateNameWithSchema(); - nextConnection(); - conn.createStatement().execute("CREATE SEQUENCE bulkalloc.alpha START WITH 30 INCREMENT BY 3 CACHE 100"); - conn.createStatement().execute("CREATE SEQUENCE bulkalloc.beta START WITH 100 INCREMENT BY 5 CACHE 50"); - nextConnection(); + conn.createStatement().execute("CREATE SEQUENCE " + sequenceName + " START WITH 30 INCREMENT BY 3 CACHE 100"); + conn.createStatement().execute("CREATE SEQUENCE " + secondSequenceName + " START WITH 100 INCREMENT BY 5 CACHE 50"); // Bulk Allocate Sequence Slots for Two Sequences ResultSet rs = conn.createStatement().executeQuery( - "SELECT NEXT 100 VALUES FOR bulkalloc.alpha, NEXT 1000 VALUES FOR bulkalloc.beta FROM \"SYSTEM\".\"SEQUENCE\""); + "SELECT NEXT 100 VALUES FOR " + sequenceName + " , NEXT 1000 VALUES FOR " + secondSequenceName + ""); assertTrue(rs.next()); assertEquals(30, rs.getInt(1)); assertEquals(100, rs.getInt(2)); // Assert standard Sequence Operations return expected values for (int i = 330; i < 330 + (2 * 100); i += 3) { - assertExpectedCurrentValueForSequence(i - 3, "bulkalloc.alpha"); - assertExpectedNextValueForSequence(i, "bulkalloc.alpha"); + assertExpectedCurrentValueForSequence(sequenceName, i - 3); + assertExpectedNextValueForSequence(sequenceName, i); } for (int i = 5100; i < 5100 + (2 * 1000); i += 5) { - assertExpectedCurrentValueForSequence(i - 5, "bulkalloc.beta"); - assertExpectedNextValueForSequence(i, "bulkalloc.beta"); + assertExpectedCurrentValueForSequence(secondSequenceName, i - 5); + assertExpectedNextValueForSequence(secondSequenceName, i); } } @@ -824,25 +811,24 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * allocate any slots. */ public void testExplainPlanValidatesSequences() throws Exception { - + String sequenceName = generateNameWithSchema(); + String tableName = generateUniqueName(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(3).startsWith(30).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(3).startsWith(30).cacheSize(100) .numAllocated(1000).build(); - nextConnection(); createSequenceWithNoMinMax(props); + Connection genericConn = createGenericConnection(); + genericConn.createStatement().execute("CREATE TABLE " + tableName + " (k BIGINT NOT NULL PRIMARY KEY)"); + genericConn.close(); - nextGenericConnection(); - conn.createStatement().execute("CREATE TABLE bulkalloc.simpletbl (k BIGINT NOT NULL PRIMARY KEY)"); - nextConnection(); - // Bulk Allocate Sequence Slots int startValueAfterAllocation = 30; - reserveSlotsInBulkAndAssertValue(startValueAfterAllocation, props.numAllocated); + reserveSlotsInBulkAndAssertValue(sequenceName, startValueAfterAllocation, props.numAllocated); // Execute EXPLAIN PLAN multiple times, which should not change Sequence values for (int i = 0; i < 3; i++) { - conn.createStatement().executeQuery("EXPLAIN SELECT NEXT 1000 VALUES FOR bulkalloc.alpha FROM bulkalloc.simpletbl"); + conn.createStatement().executeQuery("EXPLAIN SELECT NEXT 1000 VALUES FOR " + sequenceName + " FROM " + tableName); } // Validate the current value was not advanced and was the starting value @@ -851,29 +837,29 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { // Assert standard Sequence Operations return expected values int startValue = 3030; for (int i = startValue; i < startValue + (2 * props.cacheSize); i += props.incrementBy) { - assertExpectedCurrentValueForSequence(i - props.incrementBy); - assertExpectedNextValueForSequence(i); + assertExpectedCurrentValueForSequence(sequenceName, i - props.incrementBy); + assertExpectedNextValueForSequence(sequenceName, i); } } @Test public void testExplainPlanForNextValuesFor() throws Exception { - + String sequenceName = generateNameWithSchema(); + String tableName = generateUniqueName(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(3).startsWith(30).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(3).startsWith(30).cacheSize(100) .numAllocated(1000).build(); - nextConnection(); createSequenceWithNoMinMax(props); - nextGenericConnection(); - conn.createStatement().execute("CREATE TABLE bulkalloc.simpletbl (k BIGINT NOT NULL PRIMARY KEY)"); - nextConnection(); + Connection genericConn = createGenericConnection(); + genericConn.createStatement().execute("CREATE TABLE "+ tableName + " (k BIGINT NOT NULL PRIMARY KEY)"); + genericConn.close(); // Execute EXPLAIN PLAN which should not change Sequence values - ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT NEXT 1000 VALUES FOR bulkalloc.alpha FROM bulkalloc.simpletbl"); + ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT NEXT 1000 VALUES FOR " + sequenceName + " FROM " + tableName); // Assert output for Explain Plain result is as expected - assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER BULKALLOC.SIMPLETBL\n" + + assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + tableName + "\n" + " SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT RESERVE VALUES FROM 1 SEQUENCE", QueryUtil.getExplainPlan(rs)); } @@ -890,17 +876,15 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { * 4) Read back value from expression NEXT <n> VALUES FOR <seq> via rs.next() */ public void testNextValuesForMixedWithNextValueForMultiThreaded() throws Exception { + final String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(1).startsWith(1).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(1).startsWith(1).cacheSize(100) .numAllocated(1000).build(); - nextConnection(); createSequenceWithNoMinMax(props); - nextConnection(); - - assertExpectedNextValueForSequence(1); - assertExpectedCurrentValueForSequence(1); - assertExpectedNextValueForSequence(2); + assertExpectedNextValueForSequence(sequenceName, 1); + assertExpectedCurrentValueForSequence(sequenceName, 1); + assertExpectedNextValueForSequence(sequenceName, 2); // Bulk Allocate Sequence Slots final long startValueAfterAllocation1 = 101; @@ -919,7 +903,7 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { public Long call() throws Exception { ResultSet rs = conn.createStatement().executeQuery( - "SELECT NEXT " + numSlotToAllocate + " VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\""); + "SELECT NEXT " + numSlotToAllocate + " VALUES FOR " + sequenceName + " "); latch1.countDown(); // Allows NEXT VALUE FOR thread to proceed latch2.await(); // Waits until NEXT VALUE FOR thread reads and increments currentValue rs.next(); @@ -935,7 +919,7 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { latch1.await(); // Wait for execution of NEXT <n> VALUES FOR expression ResultSet rs = conn.createStatement().executeQuery( - "SELECT NEXT VALUE FOR bulkalloc.alpha FROM SYSTEM.\"SEQUENCE\""); + "SELECT NEXT VALUE FOR " + sequenceName); rs.next(); long retVal = rs.getLong(1); latch2.countDown(); // Allow NEXT <n> VALUES for thread to completed @@ -955,17 +939,15 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { @Test public void testMultipleNextValuesWithDiffAllocsForMultiThreaded() throws Exception { + final String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(1).startsWith(1).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(1).startsWith(1).cacheSize(100) .numAllocated(1000).build(); - nextConnection(); createSequenceWithNoMinMax(props); - nextConnection(); - - assertExpectedNextValueForSequence(1); - assertExpectedCurrentValueForSequence(1); - assertExpectedNextValueForSequence(2); + assertExpectedNextValueForSequence(sequenceName, 1); + assertExpectedCurrentValueForSequence(sequenceName, 1); + assertExpectedNextValueForSequence(sequenceName, 2); // Bulk Allocate Sequence Slots final long startValueAfterAllocation1 = 101; @@ -985,7 +967,7 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { public Long call() throws Exception { ResultSet rs = conn.createStatement().executeQuery( - "SELECT NEXT " + numSlotToAllocate1 + " VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\""); + "SELECT NEXT " + numSlotToAllocate1 + " VALUES FOR " + sequenceName + " "); rs.next(); latch1.countDown(); // Allows other thread to proceed latch2.await(); @@ -1001,7 +983,7 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { latch1.await(); // Wait for other thread to execut of NEXT <n> VALUES FOR expression ResultSet rs = conn.createStatement().executeQuery( - "SELECT NEXT " + numSlotToAllocate2 + " VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\""); + "SELECT NEXT " + numSlotToAllocate2 + " VALUES FOR " + sequenceName + " "); rs.next(); long retVal = rs.getLong(1); latch2.countDown(); // Allow thread to completed @@ -1028,13 +1010,12 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { @Test public void testMultipleNextValuesWithSameAllocsForMultiThreaded() throws Exception { + final String sequenceName = generateNameWithSchema(); final SequenceProperties props = - new SequenceProperties.Builder().incrementBy(1).startsWith(1).cacheSize(100) + new SequenceProperties.Builder().name(sequenceName).incrementBy(1).startsWith(1).cacheSize(100) .numAllocated(1000).build(); - nextConnection(); createSequenceWithNoMinMax(props); - nextConnection(); // Bulk Allocate Sequence Slots final long startValueAfterAllocation1 = 1; @@ -1054,7 +1035,7 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { public Long call() throws Exception { ResultSet rs = conn.createStatement().executeQuery( - "SELECT NEXT " + numSlotToAllocate1 + " VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\""); + "SELECT NEXT " + numSlotToAllocate1 + " VALUES FOR " + sequenceName + " "); latch1.countDown(); // Allows other thread to proceed latch2.await(); rs.next(); @@ -1070,7 +1051,7 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { latch1.await(); // Wait for other thread to execut of NEXT <n> VALUES FOR expression ResultSet rs = conn.createStatement().executeQuery( - "SELECT NEXT " + numSlotToAllocate2 + " VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\""); + "SELECT NEXT " + numSlotToAllocate2 + " VALUES FOR " + sequenceName + " "); rs.next(); long retVal = rs.getLong(1); latch2.countDown(); // Allow thread to completed @@ -1104,34 +1085,35 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { private void createSequenceWithNoMinMax(final SequenceProperties props) throws SQLException { conn.createStatement().execute( - String.format(CREATE_SEQUENCE_NO_MIN_MAX_TEMPLATE, props.startsWith, + String.format(CREATE_SEQUENCE_NO_MIN_MAX_TEMPLATE, props.name, props.startsWith, props.incrementBy, props.cacheSize)); } private void createSequenceWithMinMax(final SequenceProperties props) throws SQLException { conn.createStatement().execute( - String.format(CREATE_SEQUENCE_WITH_MIN_MAX_TEMPLATE, props.startsWith, + String.format(CREATE_SEQUENCE_WITH_MIN_MAX_TEMPLATE, props.name, props.startsWith, props.incrementBy, props.minValue, props.maxValue, props.cacheSize)); } private void createSequenceWithMinMaxAndCycle(final SequenceProperties props) throws SQLException { conn.createStatement().execute( - String.format(CREATE_SEQUENCE_WITH_MIN_MAX_AND_CYCLE_TEMPLATE, props.startsWith, + String.format(CREATE_SEQUENCE_WITH_MIN_MAX_AND_CYCLE_TEMPLATE, props.name, props.startsWith, props.incrementBy, props.minValue, props.maxValue, props.cacheSize)); } - private void reserveSlotsInBulkAndAssertValue(long expectedValue, long numSlotToAllocate) + private void reserveSlotsInBulkAndAssertValue(String sequenceName, long expectedValue, long numSlotToAllocate) throws SQLException { ResultSet rs = conn.createStatement().executeQuery( - "SELECT NEXT " + numSlotToAllocate + " VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\""); + "SELECT NEXT " + numSlotToAllocate + " VALUES FOR " + sequenceName + " "); assertTrue(rs.next()); assertEquals(expectedValue, rs.getInt(1)); } - private void reserveSlotsInBulkUsingBindsAndAssertValue(int expectedValue, long numSlotToAllocate) + private void reserveSlotsInBulkUsingBindsAndAssertValue(String sequenceName, int expectedValue, long numSlotToAllocate) throws SQLException { - PreparedStatement ps = conn.prepareStatement("SELECT NEXT ? VALUES FOR bulkalloc.alpha FROM \"SYSTEM\".\"SEQUENCE\""); + + PreparedStatement ps = conn.prepareStatement("SELECT NEXT ? VALUES FOR " + sequenceName + " "); ps.setLong(1, numSlotToAllocate); ResultSet rs = ps.executeQuery(); assertTrue(rs.next()); @@ -1139,52 +1121,33 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { assertEquals(expectedValue, retValue); } - private void assertExpectedCurrentValueForSequence(int expectedValue) throws SQLException { - assertExpectedCurrentValueForSequence(expectedValue, "bulkalloc.alpha"); - } - - private void assertExpectedCurrentValueForSequence(int expectedValue, String sequenceName) throws SQLException { + private void assertExpectedCurrentValueForSequence(String sequenceName, int expectedValue) throws SQLException { ResultSet rs; rs = conn.createStatement().executeQuery(String.format(SELECT_CURRENT_VALUE_SQL, sequenceName)); assertTrue(rs.next()); assertEquals(expectedValue, rs.getInt(1)); } - - private void assertExpectedNextValueForSequence(int expectedValue) throws SQLException { - assertExpectedNextValueForSequence(expectedValue, "bulkalloc.alpha"); - } - private void assertExpectedNextValueForSequence(int expectedValue, String sequenceName) throws SQLException { + private void assertExpectedNextValueForSequence(String sequenceName, int expectedValue) throws SQLException { ResultSet rs; rs = conn.createStatement().executeQuery(String.format(SELECT_NEXT_VALUE_SQL, sequenceName)); assertTrue(rs.next()); assertEquals(expectedValue, rs.getInt(1)); } - - /** - * Returns a non-tenant specific connection. - */ - private void nextGenericConnection() throws Exception { - if (conn != null) conn.close(); - long ts = nextTimestamp(); + private Connection createGenericConnection() throws Exception { Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); - props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts)); - conn = DriverManager.getConnection(getUrl(), props); + return DriverManager.getConnection(getUrl(), props); } - private void nextConnection() throws Exception { + private void createConnection() throws Exception { if (conn != null) conn.close(); - long ts = nextTimestamp(); if (tenantId != null) { - // Create tenant specific connection Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); - props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(nextTimestamp())); this.conn = DriverManager.getConnection(getUrl() + ';' + TENANT_ID_ATTRIB + '=' + "tenant1", props); } else { Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); - props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts)); conn = DriverManager.getConnection(getUrl(), props); } } @@ -1195,14 +1158,14 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { ResultSet rs = conn.createStatement() .executeQuery( - "SELECT start_with, current_value, increment_by, cache_size, min_value, max_value, cycle_flag, sequence_schema, sequence_name FROM \"SYSTEM\".\"SEQUENCE\""); + "SELECT start_with, current_value, increment_by, cache_size, min_value, max_value, cycle_flag, sequence_schema, sequence_name FROM \"SYSTEM\".\"SEQUENCE\" where sequence_name='" + props.getNameWithoutSchema() + "'"); assertTrue(rs.next()); assertEquals(props.startsWith, rs.getLong("start_with")); assertEquals(props.incrementBy, rs.getLong("increment_by")); assertEquals(props.cacheSize, rs.getLong("cache_size")); assertEquals(false, rs.getBoolean("cycle_flag")); - assertEquals("BULKALLOC", rs.getString("sequence_schema")); - assertEquals("ALPHA", rs.getString("sequence_name")); + assertEquals(props.getSchemaName(), rs.getString("sequence_schema")); + assertEquals(props.getNameWithoutSchema(), rs.getString("sequence_name")); assertEquals(currentValue, rs.getLong("current_value")); assertEquals(props.minValue, rs.getLong("min_value")); assertEquals(props.maxValue, rs.getLong("max_value")); @@ -1226,6 +1189,7 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { private final int cacheSize; private final long minValue; private final long maxValue; + private final String name; public SequenceProperties(Builder builder) { this.numAllocated = builder.numAllocated; @@ -1234,6 +1198,7 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { this.cacheSize = builder.cacheSize; this.minValue = builder.minValue; this.maxValue = builder.maxValue; + this.name = builder.name; } private static class Builder { @@ -1244,6 +1209,7 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { int incrementBy = 1; int startsWith = 1; int cacheSize = 100; + String name = null; public Builder numAllocated(long numAllocated) { this.numAllocated = numAllocated; @@ -1275,11 +1241,23 @@ public class SequenceBulkAllocationIT extends BaseClientManagedTimeIT { return this; } + public Builder name(String name) { + this.name = name; + return this; + } + public SequenceProperties build() { return new SequenceProperties(this); } - } + + private String getSchemaName() { + return name.substring(0, name.indexOf(".")); + } + + private String getNameWithoutSchema() { + return name.substring(name.indexOf(".") + 1, name.length()); + } }