diff --git a/core/src/main/java/com/scalar/db/transaction/consensuscommit/ConsensusCommitUtils.java b/core/src/main/java/com/scalar/db/transaction/consensuscommit/ConsensusCommitUtils.java index 4ca0f509cc..6ad6f9f9fc 100644 --- a/core/src/main/java/com/scalar/db/transaction/consensuscommit/ConsensusCommitUtils.java +++ b/core/src/main/java/com/scalar/db/transaction/consensuscommit/ConsensusCommitUtils.java @@ -1,12 +1,22 @@ package com.scalar.db.transaction.consensuscommit; import com.google.common.collect.ImmutableMap; +import com.scalar.db.api.AndConditionSet; import com.scalar.db.api.ConditionBuilder; +import com.scalar.db.api.ConditionSetBuilder; +import com.scalar.db.api.ConditionalExpression; +import com.scalar.db.api.Consistency; +import com.scalar.db.api.Get; +import com.scalar.db.api.GetBuilder; import com.scalar.db.api.Insert; +import com.scalar.db.api.LikeExpression; import com.scalar.db.api.MutationCondition; import com.scalar.db.api.Operation; import com.scalar.db.api.Put; import com.scalar.db.api.PutBuilder; +import com.scalar.db.api.Scan; +import com.scalar.db.api.ScanBuilder; +import com.scalar.db.api.Selection; import com.scalar.db.api.TableMetadata; import com.scalar.db.api.Update; import com.scalar.db.api.UpdateIf; @@ -352,4 +362,149 @@ static TransactionTableMetadata getTransactionTableMetadata( } return metadata; } + + static Get prepareGetForStorage(Get get, TableMetadata metadata) { + GetBuilder.BuildableGetOrGetWithIndexFromExisting builder = + Get.newBuilder(get).clearProjections().consistency(Consistency.LINEARIZABLE); + + if (!get.getConjunctions().isEmpty()) { + // If there are conjunctions, we need to convert them to include conditions on the before + // image + Set converted = convertConjunctions(get.getConjunctions(), metadata); + return builder.clearConditions().whereOr(converted).build(); + } + + return builder.build(); + } + + static Scan prepareScanForStorage(Scan scan, TableMetadata metadata) { + ScanBuilder.BuildableScanOrScanAllFromExisting builder = + Scan.newBuilder(scan).clearProjections().consistency(Consistency.LINEARIZABLE); + + if (scan.getLimit() > 0) { + // Since the recovery process and the conjunction processing may exclude some records from + // the scan result, it is necessary to perform the scan without a limit. + builder.limit(0); + } + + if (!scan.getConjunctions().isEmpty()) { + // If there are conjunctions, we need to convert them to include conditions on the before + // image + Set converted = convertConjunctions(scan.getConjunctions(), metadata); + return builder.clearConditions().whereOr(converted).build(); + } + + return builder.build(); + } + + /** + * Converts the given conjunctions to include conditions on before images. + * + *

This is necessary because we might miss prepared records whose before images match the + * original conditions when reading from storage. For example, suppose we have the following + * records in storage: + * + *

+   *   | partition_key | clustering_key | column | status    | before_column | before_status  |
+   *   |---------------|----------------|--------|-----------|---------------|----------------|
+   *   | 0             | 0              | 1000   | COMMITTED |               |                |
+   *   | 0             | 1              | 200    | PREPARED  | 1000          | COMMITTED      |
+   * 
+ * + * If we scan records with the condition "column = 1000" without converting the condition + * (conjunction), we only get the first record, not the second one, because the condition does not + * match. However, the second record has not been committed yet, so we should still retrieve it, + * considering the possibility that the record will be rolled back. + * + *

To handle such cases, we convert the conjunctions to include conditions on the before image. + * For example, if the original condition is: + * + *

+   *   column = 1000
+   * 
+ * + * We convert it to: + * + *
+   *   column = 1000 OR before_column = 1000
+   * 
+ * + *

Here are more examples: + * + *

Example 1: + * + *

+   *   {@code column >= 500 AND column < 1000}
+   * 
+ * + * becomes: + * + *
+   *   {@code (column >= 500 AND column < 1000) OR (before_column >= 500 AND before_column < 1000)}
+   * 
+ * + *

Example 2: + * + *

+   *   {@code column1 = 500 OR column2 != 1000}
+   * 
+ * + * becomes: + * + *
+   *   {@code column1 = 500 OR column2 != 1000 OR before_column1 = 500 OR before_column2 != 1000}
+   * 
+ * + * This way, we can ensure that prepared records whose before images satisfy the original scan + * conditions are not missed during the scan. + * + * @param conjunctions the conjunctions to convert + * @param metadata the table metadata of the target table + * @return the converted conjunctions + */ + private static Set convertConjunctions( + Set conjunctions, TableMetadata metadata) { + Set converted = new HashSet<>(conjunctions.size() * 2); + + // Keep the original conjunctions + conjunctions.forEach( + c -> converted.add(ConditionSetBuilder.andConditionSet(c.getConditions()).build())); + + // Add conditions on the before image + for (Selection.Conjunction conjunction : conjunctions) { + Set conditions = new HashSet<>(conjunction.getConditions().size()); + for (ConditionalExpression condition : conjunction.getConditions()) { + String columnName = condition.getColumn().getName(); + + if (metadata.getPartitionKeyNames().contains(columnName) + || metadata.getClusteringKeyNames().contains(columnName)) { + // If the condition is on the primary key, we don't need to convert it + conditions.add(condition); + continue; + } + + // Convert the condition to use the before image column + ConditionalExpression convertedCondition; + if (condition instanceof LikeExpression) { + LikeExpression likeExpression = (LikeExpression) condition; + convertedCondition = + ConditionBuilder.buildLikeExpression( + likeExpression.getColumn().copyWith(Attribute.BEFORE_PREFIX + columnName), + likeExpression.getOperator(), + likeExpression.getEscape()); + } else { + convertedCondition = + ConditionBuilder.buildConditionalExpression( + condition.getColumn().copyWith(Attribute.BEFORE_PREFIX + columnName), + condition.getOperator()); + } + + conditions.add(convertedCondition); + } + + converted.add(ConditionSetBuilder.andConditionSet(conditions).build()); + } + + return converted; + } } diff --git a/core/src/main/java/com/scalar/db/transaction/consensuscommit/CrudHandler.java b/core/src/main/java/com/scalar/db/transaction/consensuscommit/CrudHandler.java index 9f6a21bfad..3ce4abfb99 100644 --- a/core/src/main/java/com/scalar/db/transaction/consensuscommit/CrudHandler.java +++ b/core/src/main/java/com/scalar/db/transaction/consensuscommit/CrudHandler.java @@ -4,16 +4,11 @@ import static com.scalar.db.transaction.consensuscommit.ConsensusCommitOperationAttributes.isImplicitPreReadEnabled; import com.google.common.annotations.VisibleForTesting; -import com.scalar.db.api.AndConditionSet; -import com.scalar.db.api.ConditionBuilder; -import com.scalar.db.api.ConditionSetBuilder; -import com.scalar.db.api.ConditionalExpression; import com.scalar.db.api.Consistency; import com.scalar.db.api.Delete; import com.scalar.db.api.DistributedStorage; import com.scalar.db.api.Get; import com.scalar.db.api.GetBuilder; -import com.scalar.db.api.LikeExpression; import com.scalar.db.api.Operation; import com.scalar.db.api.Put; import com.scalar.db.api.Result; @@ -31,13 +26,11 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.IOException; import java.util.ArrayList; -import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; @@ -88,10 +81,7 @@ public CrudHandler( this.parallelExecutor = checkNotNull(parallelExecutor); } - public Optional get(Get originalGet, TransactionContext context) throws CrudException { - List originalProjections = new ArrayList<>(originalGet.getProjections()); - Get get = (Get) prepareStorageSelection(originalGet); - + public Optional get(Get get, TransactionContext context) throws CrudException { TableMetadata metadata = getTableMetadata(get, context.transactionId); Snapshot.Key key; @@ -103,43 +93,46 @@ public Optional get(Get originalGet, TransactionContext context) throws } if (isSnapshotReadRequired(context)) { - readUnread(key, get, context); + readUnread(key, get, context, metadata); return context .snapshot .getResult(key, get) - .map(r -> new FilteredResult(r, originalProjections, metadata, isIncludeMetadataEnabled)); + .map( + r -> new FilteredResult(r, get.getProjections(), metadata, isIncludeMetadataEnabled)); } else { - Optional result = read(key, get, context); + Optional result = read(key, get, context, metadata); return context .snapshot .mergeResult(key, result, get.getConjunctions()) - .map(r -> new FilteredResult(r, originalProjections, metadata, isIncludeMetadataEnabled)); + .map( + r -> new FilteredResult(r, get.getProjections(), metadata, isIncludeMetadataEnabled)); } } // Only for a Get with index, the argument `key` is null @VisibleForTesting - void readUnread(@Nullable Snapshot.Key key, Get get, TransactionContext context) + void readUnread( + @Nullable Snapshot.Key key, Get get, TransactionContext context, TableMetadata metadata) throws CrudException { if (!context.snapshot.containsKeyInGetSet(get)) { - read(key, get, context); + read(key, get, context, metadata); } } // Although this class is not thread-safe, this method is actually thread-safe, so we call it // concurrently in the implicit pre-read @VisibleForTesting - Optional read(@Nullable Snapshot.Key key, Get get, TransactionContext context) + Optional read( + @Nullable Snapshot.Key key, Get get, TransactionContext context, TableMetadata metadata) throws CrudException { - Optional result = getFromStorage(get, context); + Optional result = getFromStorage(get, metadata, context.transactionId); if (result.isPresent() && !result.get().isCommitted()) { // Lazy recovery if (key == null) { // Only for a Get with index, the argument `key` is null. In that case, create a key from // the result - TableMetadata tableMetadata = getTableMetadata(get, context.transactionId); - key = new Snapshot.Key(get, result.get(), tableMetadata); + key = new Snapshot.Key(get, result.get(), metadata); } result = executeRecovery(key, get, result.get(), context); @@ -168,8 +161,7 @@ Optional read(@Nullable Snapshot.Key key, Get get, Transactio if (result.isPresent()) { // Only when we can get the record with the Get with index, we can put it into the read // set - TableMetadata tableMetadata = getTableMetadata(get, context.transactionId); - key = new Snapshot.Key(get, result.get(), tableMetadata); + key = new Snapshot.Key(get, result.get(), metadata); putIntoReadSetInSnapshot(key, result, context); } } @@ -205,20 +197,17 @@ private Optional executeRecovery( return recoveryResult.recoveredResult; } - public List scan(Scan originalScan, TransactionContext context) throws CrudException { - List originalProjections = new ArrayList<>(originalScan.getProjections()); - Scan scan = (Scan) prepareStorageSelection(originalScan); - LinkedHashMap results = scanInternal(scan, context); - verifyNoOverlap(scan, results, context); - + public List scan(Scan scan, TransactionContext context) throws CrudException { TableMetadata metadata = getTableMetadata(scan, context.transactionId); + LinkedHashMap results = scanInternal(scan, context, metadata); + verifyNoOverlap(scan, results, context); return results.values().stream() - .map(r -> new FilteredResult(r, originalProjections, metadata, isIncludeMetadataEnabled)) + .map(r -> new FilteredResult(r, scan.getProjections(), metadata, isIncludeMetadataEnabled)) .collect(Collectors.toList()); } private LinkedHashMap scanInternal( - Scan scan, TransactionContext context) throws CrudException { + Scan scan, TransactionContext context, TableMetadata metadata) throws CrudException { Optional> resultsInSnapshot = context.snapshot.getResults(scan); if (resultsInSnapshot.isPresent()) { @@ -227,20 +216,10 @@ private LinkedHashMap scanInternal( LinkedHashMap results = new LinkedHashMap<>(); - Scanner scanner = null; - try { - if (scan.getLimit() > 0) { - // Since recovery and conjunctions may delete some records from the scan result, it is - // necessary to perform the scan without a limit. - scanner = scanFromStorage(Scan.newBuilder(scan).limit(0).build(), context); - } else { - scanner = scanFromStorage(scan, context); - } - + try (Scanner scanner = scanFromStorage(scan, metadata, context.transactionId)) { for (Result r : scanner) { TransactionResult result = new TransactionResult(r); - TableMetadata tableMetadata = getTableMetadata(scan, context.transactionId); - Snapshot.Key key = new Snapshot.Key(scan, r, tableMetadata); + Snapshot.Key key = new Snapshot.Key(scan, r, metadata); Optional processedScanResult = processScanResult(key, scan, result, context); processedScanResult.ifPresent(res -> results.put(key, res)); @@ -261,14 +240,8 @@ private LinkedHashMap scanInternal( CoreError.CONSENSUS_COMMIT_SCANNING_RECORDS_FROM_STORAGE_FAILED.buildMessage(), exception, context.transactionId); - } finally { - if (scanner != null) { - try { - scanner.close(); - } catch (IOException e) { - logger.warn("Failed to close the scanner. Transaction ID: {}", context.transactionId, e); - } - } + } catch (IOException e) { + logger.warn("Failed to close the scanner. Transaction ID: {}", context.transactionId, e); } putIntoScanSetInSnapshot(scan, results, context); @@ -304,21 +277,17 @@ private Optional processScanResult( return ret; } - public TransactionCrudOperable.Scanner getScanner(Scan originalScan, TransactionContext context) + public TransactionCrudOperable.Scanner getScanner(Scan scan, TransactionContext context) throws CrudException { - List originalProjections = new ArrayList<>(originalScan.getProjections()); - Scan scan = (Scan) prepareStorageSelection(originalScan); - + TableMetadata metadata = getTableMetadata(scan, context.transactionId); ConsensusCommitScanner scanner; - Optional> resultsInSnapshot = context.snapshot.getResults(scan); if (resultsInSnapshot.isPresent()) { scanner = - new ConsensusCommitSnapshotScanner( - scan, originalProjections, context, resultsInSnapshot.get()); + new ConsensusCommitSnapshotScanner(scan, context, metadata, resultsInSnapshot.get()); } else { - scanner = new ConsensusCommitStorageScanner(scan, originalProjections, context); + scanner = new ConsensusCommitStorageScanner(scan, context, metadata); } context.scanners.add(scanner); @@ -385,6 +354,7 @@ private boolean isOverlapVerificationRequired(TransactionContext context) { } public void put(Put put, TransactionContext context) throws CrudException { + TableMetadata metadata = getTableMetadata(put, context.transactionId); Snapshot.Key key = new Snapshot.Key(put); if (put.getCondition().isPresent() @@ -397,7 +367,7 @@ public void put(Put put, TransactionContext context) throws CrudException { if (put.getCondition().isPresent()) { if (isImplicitPreReadEnabled(put) && !context.snapshot.containsKeyInReadSet(key)) { - read(key, createGet(key), context); + read(key, createGet(key), context, metadata); } mutationConditionsValidator.checkIfConditionIsSatisfied( put, context.snapshot.getResult(key).orElse(null), context); @@ -407,11 +377,12 @@ public void put(Put put, TransactionContext context) throws CrudException { } public void delete(Delete delete, TransactionContext context) throws CrudException { + TableMetadata metadata = getTableMetadata(delete, context.transactionId); Snapshot.Key key = new Snapshot.Key(delete); if (delete.getCondition().isPresent()) { if (!context.snapshot.containsKeyInReadSet(key)) { - read(key, createGet(key), context); + read(key, createGet(key), context, metadata); } mutationConditionsValidator.checkIfConditionIsSatisfied( delete, context.snapshot.getResult(key).orElse(null), context); @@ -430,7 +401,9 @@ public void readIfImplicitPreReadEnabled(TransactionContext context) throws Crud if (isImplicitPreReadEnabled(put)) { Snapshot.Key key = entry.getKey(); if (!context.snapshot.containsKeyInReadSet(key)) { - tasks.add(() -> read(key, createGet(key), context)); + Get get = createGet(key); + TableMetadata metadata = getTableMetadata(get, context.transactionId); + tasks.add(() -> read(key, get, context, metadata)); } } } @@ -439,7 +412,9 @@ public void readIfImplicitPreReadEnabled(TransactionContext context) throws Crud for (Map.Entry entry : context.snapshot.getDeleteSet()) { Snapshot.Key key = entry.getKey(); if (!context.snapshot.containsKeyInReadSet(key)) { - tasks.add(() -> read(key, createGet(key), context)); + Get get = createGet(key); + TableMetadata metadata = getTableMetadata(get, context.transactionId); + tasks.add(() -> read(key, get, context, metadata)); } } @@ -455,7 +430,7 @@ private Get createGet(Snapshot.Key key) { .table(key.getTable()) .partitionKey(key.getPartitionKey()); key.getClusteringKey().ifPresent(buildableGet::clusteringKey); - return (Get) prepareStorageSelection(buildableGet.build()); + return buildableGet.consistency(Consistency.LINEARIZABLE).build(); } /** @@ -543,174 +518,39 @@ void waitForRecoveryCompletion(TransactionContext context) throws CrudException // Although this class is not thread-safe, this method is actually thread-safe because the storage // is thread-safe @VisibleForTesting - Optional getFromStorage(Get get, TransactionContext context) + Optional getFromStorage(Get get, TableMetadata metadata, String transactionId) throws CrudException { try { - if (get.getConjunctions().isEmpty()) { - // If there are no conjunctions, we can read the record directly - return storage.get(get).map(TransactionResult::new); - } else { - // If there are conjunctions, we need to convert them to include conditions on the before - // image - Set converted = convertConjunctions(get, get.getConjunctions(), context); - Get convertedGet = Get.newBuilder(get).clearConditions().whereOr(converted).build(); - return storage.get(convertedGet).map(TransactionResult::new); - } + return storage + .get(ConsensusCommitUtils.prepareGetForStorage(get, metadata)) + .map(TransactionResult::new); } catch (ExecutionException e) { throw new CrudException( CoreError.CONSENSUS_COMMIT_READING_RECORD_FROM_STORAGE_FAILED.buildMessage(), e, - context.transactionId); + transactionId); } } - private Scanner scanFromStorage(Scan scan, TransactionContext context) throws CrudException { + private Scanner scanFromStorage(Scan scan, TableMetadata metadata, String transactionId) + throws CrudException { try { - if (scan.getConjunctions().isEmpty()) { - // If there are no conjunctions, we can read the record directly - return storage.scan(scan); - } else { - // If there are conjunctions, we need to convert them to include conditions on the before - // image - Set converted = convertConjunctions(scan, scan.getConjunctions(), context); - Scan convertedScan = Scan.newBuilder(scan).clearConditions().whereOr(converted).build(); - return storage.scan(convertedScan); - } + return storage.scan(ConsensusCommitUtils.prepareScanForStorage(scan, metadata)); } catch (ExecutionException e) { throw new CrudException( CoreError.CONSENSUS_COMMIT_SCANNING_RECORDS_FROM_STORAGE_FAILED.buildMessage(), e, - context.transactionId); + transactionId); } } - /** - * Converts the given conjunctions to include conditions on before images. - * - *

This is necessary because we might miss prepared records whose before images match the - * original conditions when reading from storage. For example, suppose we have the following - * records in storage: - * - *

-   *   | partition_key | clustering_key | column | status    | before_column | before_status  |
-   *   |---------------|----------------|--------|-----------|---------------|----------------|
-   *   | 0             | 0              | 1000   | COMMITTED |               |                |
-   *   | 0             | 1              | 200    | PREPARED  | 1000          | COMMITTED      |
-   * 
- * - * If we scan records with the condition "column = 1000" without converting the condition - * (conjunction), we only get the first record, not the second one, because the condition does not - * match. However, the second record has not been committed yet, so we should still retrieve it, - * considering the possibility that the record will be rolled back. - * - *

To handle such cases, we convert the conjunctions to include conditions on the before image. - * For example, if the original condition is: - * - *

-   *   column = 1000
-   * 
- * - * We convert it to: - * - *
-   *   column = 1000 OR before_column = 1000
-   * 
- * - *

Here are more examples: - * - *

Example 1: - * - *

-   *   {@code column >= 500 AND column < 1000}
-   * 
- * - * becomes: - * - *
-   *   {@code (column >= 500 AND column < 1000) OR (before_column >= 500 AND before_column < 1000)}
-   * 
- * - *

Example 2: - * - *

-   *   {@code column1 = 500 OR column2 != 1000}
-   * 
- * - * becomes: - * - *
-   *   {@code column1 = 500 OR column2 != 1000 OR before_column1 = 500 OR before_column2 != 1000}
-   * 
- * - * This way, we can ensure that prepared records whose before images satisfy the original scan - * conditions are not missed during the scan. - * - * @param selection the selection to convert - * @param conjunctions the conjunctions to convert - * @param context the transaction context - * @return the converted conjunctions - */ - private Set convertConjunctions( - Selection selection, Set conjunctions, TransactionContext context) + private TableMetadata getTableMetadata(Operation operation, String transactionId) throws CrudException { - TableMetadata metadata = getTableMetadata(selection, context.transactionId); - - Set converted = new HashSet<>(conjunctions.size() * 2); - - // Keep the original conjunctions - conjunctions.forEach( - c -> converted.add(ConditionSetBuilder.andConditionSet(c.getConditions()).build())); - - // Add conditions on the before image - for (Selection.Conjunction conjunction : conjunctions) { - Set conditions = new HashSet<>(conjunction.getConditions().size()); - for (ConditionalExpression condition : conjunction.getConditions()) { - String columnName = condition.getColumn().getName(); - - if (metadata.getPartitionKeyNames().contains(columnName) - || metadata.getClusteringKeyNames().contains(columnName)) { - // If the condition is on the primary key, we don't need to convert it - conditions.add(condition); - continue; - } - - // Convert the condition to use the before image column - ConditionalExpression convertedCondition; - if (condition instanceof LikeExpression) { - LikeExpression likeExpression = (LikeExpression) condition; - convertedCondition = - ConditionBuilder.buildLikeExpression( - likeExpression.getColumn().copyWith(Attribute.BEFORE_PREFIX + columnName), - likeExpression.getOperator(), - likeExpression.getEscape()); - } else { - convertedCondition = - ConditionBuilder.buildConditionalExpression( - condition.getColumn().copyWith(Attribute.BEFORE_PREFIX + columnName), - condition.getOperator()); - } - - conditions.add(convertedCondition); - } - - converted.add(ConditionSetBuilder.andConditionSet(conditions).build()); - } - - return converted; - } - - private Selection prepareStorageSelection(Selection selection) { - selection.clearProjections(); - selection.withConsistency(Consistency.LINEARIZABLE); - return selection; - } - - private TransactionTableMetadata getTransactionTableMetadata( - Operation operation, String transactionId) throws CrudException { assert operation.forFullTableName().isPresent(); try { - return ConsensusCommitUtils.getTransactionTableMetadata(tableMetadataManager, operation); + return ConsensusCommitUtils.getTransactionTableMetadata(tableMetadataManager, operation) + .getTableMetadata(); } catch (ExecutionException e) { throw new CrudException( CoreError.GETTING_TABLE_METADATA_FAILED.buildMessage(operation.forFullTableName().get()), @@ -719,19 +559,13 @@ private TransactionTableMetadata getTransactionTableMetadata( } } - private TableMetadata getTableMetadata(Operation operation, String transactionId) - throws CrudException { - TransactionTableMetadata metadata = getTransactionTableMetadata(operation, transactionId); - return metadata.getTableMetadata(); - } - @NotThreadSafe private class ConsensusCommitStorageScanner extends AbstractTransactionCrudOperableScanner implements ConsensusCommitScanner { private final Scan scan; - private final List originalProjections; private final TransactionContext context; + private final TableMetadata metadata; private final Scanner scanner; @Nullable private final LinkedHashMap results; @@ -740,19 +574,12 @@ private class ConsensusCommitStorageScanner extends AbstractTransactionCrudOpera private final AtomicBoolean closed = new AtomicBoolean(); public ConsensusCommitStorageScanner( - Scan scan, List originalProjections, TransactionContext context) - throws CrudException { + Scan scan, TransactionContext context, TableMetadata metadata) throws CrudException { this.scan = scan; - this.originalProjections = originalProjections; this.context = context; + this.metadata = metadata; - if (scan.getLimit() > 0) { - // Since recovery and conjunctions may delete some records, it is necessary to perform the - // scan without a limit. - scanner = scanFromStorage(Scan.newBuilder(scan).limit(0).build(), context); - } else { - scanner = scanFromStorage(scan, context); - } + scanner = scanFromStorage(scan, metadata, context.transactionId); if (isValidationOrSnapshotReadRequired(context) || isOverlapVerificationRequired(context)) { results = new LinkedHashMap<>(); @@ -778,8 +605,7 @@ public Optional one() throws CrudException { return Optional.empty(); } - TableMetadata tableMetadata = getTableMetadata(scan, context.transactionId); - Snapshot.Key key = new Snapshot.Key(scan, r.get(), tableMetadata); + Snapshot.Key key = new Snapshot.Key(scan, r.get(), metadata); TransactionResult result = new TransactionResult(r.get()); Optional processedScanResult = @@ -798,11 +624,10 @@ public Optional one() throws CrudException { fullyScanned.set(true); } - TableMetadata metadata = getTableMetadata(scan, context.transactionId); return Optional.of( new FilteredResult( processedScanResult.get(), - originalProjections, + scan.getProjections(), metadata, isIncludeMetadataEnabled)); } @@ -873,8 +698,8 @@ private class ConsensusCommitSnapshotScanner extends AbstractTransactionCrudOper implements ConsensusCommitScanner { private final Scan scan; - private final List originalProjections; private final TransactionContext context; + private final TableMetadata metadata; private final Iterator> resultsIterator; private final LinkedHashMap results = new LinkedHashMap<>(); @@ -882,17 +707,17 @@ private class ConsensusCommitSnapshotScanner extends AbstractTransactionCrudOper public ConsensusCommitSnapshotScanner( Scan scan, - List originalProjections, TransactionContext context, + TableMetadata metadata, LinkedHashMap resultsInSnapshot) { this.scan = scan; - this.originalProjections = originalProjections; this.context = context; + this.metadata = metadata; resultsIterator = resultsInSnapshot.entrySet().iterator(); } @Override - public Optional one() throws CrudException { + public Optional one() { if (!resultsIterator.hasNext()) { return Optional.empty(); } @@ -900,14 +725,13 @@ public Optional one() throws CrudException { Map.Entry entry = resultsIterator.next(); results.put(entry.getKey(), entry.getValue()); - TableMetadata metadata = getTableMetadata(scan, context.transactionId); return Optional.of( new FilteredResult( - entry.getValue(), originalProjections, metadata, isIncludeMetadataEnabled)); + entry.getValue(), scan.getProjections(), metadata, isIncludeMetadataEnabled)); } @Override - public List all() throws CrudException { + public List all() { List results = new ArrayList<>(); while (true) { diff --git a/core/src/main/java/com/scalar/db/transaction/consensuscommit/Snapshot.java b/core/src/main/java/com/scalar/db/transaction/consensuscommit/Snapshot.java index e520678d31..e492b338a2 100644 --- a/core/src/main/java/com/scalar/db/transaction/consensuscommit/Snapshot.java +++ b/core/src/main/java/com/scalar/db/transaction/consensuscommit/Snapshot.java @@ -540,10 +540,11 @@ void toSerializable(DistributedStorage storage) // Get set is re-validated to check if there is no anti-dependency for (Map.Entry> entry : getSet.entrySet()) { Get get = entry.getKey(); + TableMetadata metadata = getTableMetadata(get); - if (ScalarDbUtils.isSecondaryIndexSpecified(get, getTableMetadata(get))) { + if (ScalarDbUtils.isSecondaryIndexSpecified(get, metadata)) { // For Get with index - tasks.add(() -> validateGetWithIndexResult(storage, get, entry.getValue())); + tasks.add(() -> validateGetWithIndexResult(storage, get, entry.getValue(), metadata)); } else { // For other Get @@ -552,7 +553,7 @@ void toSerializable(DistributedStorage storage) continue; } - tasks.add(() -> validateGetResult(storage, get, entry.getValue())); + tasks.add(() -> validateGetResult(storage, get, entry.getValue(), metadata)); } } @@ -589,22 +590,12 @@ private void validateScanResults( throws ExecutionException, ValidationConflictException { Scanner scanner = null; try { - TableMetadata tableMetadata = getTableMetadata(scan); + TableMetadata metadata = getTableMetadata(scan); - // Only get tx_id and primary key columns because we use only them to compare - scan.clearProjections(); - scan.withProjection(Attribute.ID); - ScalarDbUtils.addProjectionsForKeys(scan, getTableMetadata(scan)); - - if (scan.getLimit() == 0) { - scanner = storage.scan(scan); - } else { - // Get a scanner without the limit if the scan has a limit - scanner = storage.scan(Scan.newBuilder(scan).limit(0).build()); - } + scanner = storage.scan(ConsensusCommitUtils.prepareScanForStorage(scan, metadata)); // Initialize the iterator for the latest scan results - Optional latestResult = scanner.one(); + Optional latestResult = getNextResult(scanner, scan); // Initialize the iterator for the original scan results Iterator> originalResultIterator = @@ -615,13 +606,13 @@ private void validateScanResults( // Compare the records of the iterators while (latestResult.isPresent() && originalResultEntry != null) { TransactionResult latestTxResult = new TransactionResult(latestResult.get()); - Key key = new Key(scan, latestTxResult, tableMetadata); + Key key = new Key(scan, latestTxResult, metadata); if (latestTxResult.getId() != null && latestTxResult.getId().equals(id)) { // The record is inserted/deleted/updated by this transaction // Skip the record of the latest scan results - latestResult = scanner.one(); + latestResult = getNextResult(scanner, scan); if (originalResultEntry.getKey().equals(key)) { // The record is updated by this transaction @@ -655,7 +646,7 @@ private void validateScanResults( } // Proceed to the next record - latestResult = scanner.one(); + latestResult = getNextResult(scanner, scan); originalResultEntry = Iterators.getNext(originalResultIterator, null); } @@ -694,7 +685,7 @@ private void validateScanResults( // The record is inserted/deleted by this transaction // Skip the record - latestResult = scanner.one(); + latestResult = getNextResult(scanner, scan); } else { // The record is inserted by another transaction throwExceptionDueToAntiDependency(); @@ -711,8 +702,30 @@ private void validateScanResults( } } + private Optional getNextResult(Scanner scanner, Scan scan) throws ExecutionException { + Optional next = scanner.one(); + if (!next.isPresent()) { + return next; + } + + if (!scan.getConjunctions().isEmpty()) { + // Because we also get records whose before images match the conjunctions, we need to check if + // the current status of the records actually match the conjunctions. + next = + next.filter( + r -> + ScalarDbUtils.columnsMatchAnyOfConjunctions( + r.getColumns(), scan.getConjunctions())); + } + + return next.isPresent() ? next : getNextResult(scanner, scan); + } + private void validateGetWithIndexResult( - DistributedStorage storage, Get get, Optional originalResult) + DistributedStorage storage, + Get get, + Optional originalResult, + TableMetadata metadata) throws ExecutionException, ValidationConflictException { assert get.forNamespace().isPresent() && get.forTable().isPresent(); @@ -733,23 +746,34 @@ private void validateGetWithIndexResult( .build(); LinkedHashMap results = new LinkedHashMap<>(1); - TableMetadata tableMetadata = getTableMetadata(scanWithIndex); - originalResult.ifPresent( - r -> results.put(new Snapshot.Key(scanWithIndex, r, tableMetadata), r)); + originalResult.ifPresent(r -> results.put(new Snapshot.Key(scanWithIndex, r, metadata), r)); // Validate the result to check if there is no anti-dependency validateScanResults(storage, scanWithIndex, results, false); } private void validateGetResult( - DistributedStorage storage, Get get, Optional originalResult) + DistributedStorage storage, + Get get, + Optional originalResult, + TableMetadata metadata) throws ExecutionException, ValidationConflictException { - // Only get the tx_id column because we use only them to compare - get.clearProjections(); - get.withProjection(Attribute.ID); - // Check if a read record is not changed - Optional latestResult = storage.get(get).map(TransactionResult::new); + Optional latestResult = + storage + .get(ConsensusCommitUtils.prepareGetForStorage(get, metadata)) + .map(TransactionResult::new); + + if (!get.getConjunctions().isEmpty()) { + // Because we also get records whose before images match the conjunctions, we need to check if + // the current status of the records actually match the conjunctions. + latestResult = + latestResult.filter( + r -> + ScalarDbUtils.columnsMatchAnyOfConjunctions( + r.getColumns(), get.getConjunctions())); + } + if (isChanged(latestResult, originalResult)) { throwExceptionDueToAntiDependency(); } diff --git a/core/src/test/java/com/scalar/db/transaction/consensuscommit/ConsensusCommitUtilsTest.java b/core/src/test/java/com/scalar/db/transaction/consensuscommit/ConsensusCommitUtilsTest.java index 5ccd98ea53..a17a2f21d9 100644 --- a/core/src/test/java/com/scalar/db/transaction/consensuscommit/ConsensusCommitUtilsTest.java +++ b/core/src/test/java/com/scalar/db/transaction/consensuscommit/ConsensusCommitUtilsTest.java @@ -1,14 +1,20 @@ package com.scalar.db.transaction.consensuscommit; +import static com.scalar.db.api.ConditionBuilder.column; +import static com.scalar.db.api.ConditionSetBuilder.*; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.scalar.db.api.Consistency; +import com.scalar.db.api.Get; +import com.scalar.db.api.Scan; import com.scalar.db.api.TableMetadata; import com.scalar.db.io.Column; import com.scalar.db.io.DataType; import com.scalar.db.io.IntColumn; +import com.scalar.db.io.Key; import com.scalar.db.io.TextColumn; import java.util.HashMap; import java.util.HashSet; @@ -700,4 +706,586 @@ public void createAfterImageColumnsFromBeforeImage_versionColumnWithNonZero_shou assertThat(columns.get("tx_version").hasNullValue()).isFalse(); assertThat(columns.get("tx_version").getIntValue()).isEqualTo(5); } + + @Test + public void prepareGetForStorage_GetWithoutConjunctions_shouldReturnPreparedGet() { + // Arrange + Get get = + Get.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofText("pk", "key1")) + .projections("col1", "col2") + .build(); + + TableMetadata metadata = + TableMetadata.newBuilder() + .addColumn("pk", DataType.TEXT) + .addColumn("col1", DataType.INT) + .addColumn("col2", DataType.TEXT) + .addPartitionKey("pk") + .build(); + + // Act + Get actual = ConsensusCommitUtils.prepareGetForStorage(get, metadata); + + // Assert + assertThat(actual) + .isEqualTo( + Get.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofText("pk", "key1")) + .consistency(Consistency.LINEARIZABLE) + .build()); + } + + @Test + public void prepareGetForStorage_GetWithConjunctions_shouldConvertConjunctions() { + // Arrange + TableMetadata metadata = + ConsensusCommitUtils.buildTransactionTableMetadata( + TableMetadata.newBuilder() + .addColumn("pk", DataType.INT) + .addColumn("ck", DataType.INT) + .addColumn("col1", DataType.INT) + .addColumn("col2", DataType.INT) + .addColumn("col3", DataType.INT) + .addColumn("col4", DataType.TEXT) + .addPartitionKey("pk") + .addClusteringKey("ck") + .addSecondaryIndex("col3") + .build()); + + // Act Assert + + // Single condition + assertThat( + ConsensusCommitUtils.prepareGetForStorage( + Get.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .clusteringKey(Key.ofInt("ck", 200)) + .where(column("col1").isEqualToInt(10)) + .build(), + metadata)) + .isEqualTo( + Get.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .clusteringKey(Key.ofInt("ck", 200)) + .where(column("col1").isEqualToInt(10)) + .or(column(Attribute.BEFORE_PREFIX + "col1").isEqualToInt(10)) + .consistency(Consistency.LINEARIZABLE) + .build()); + + // AND condition + assertThat( + ConsensusCommitUtils.prepareGetForStorage( + Get.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .clusteringKey(Key.ofInt("ck", 200)) + .where(column("col1").isEqualToInt(10)) + .and(column("col2").isGreaterThanInt(20)) + .build(), + metadata)) + .isEqualTo( + Get.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .clusteringKey(Key.ofInt("ck", 200)) + .where( + condition(column("col1").isEqualToInt(10)) + .and(column("col2").isGreaterThanInt(20)) + .build()) + .or( + condition(column(Attribute.BEFORE_PREFIX + "col1").isEqualToInt(10)) + .and(column(Attribute.BEFORE_PREFIX + "col2").isGreaterThanInt(20)) + .build()) + .consistency(Consistency.LINEARIZABLE) + .build()); + + // OR condition + assertThat( + ConsensusCommitUtils.prepareGetForStorage( + Get.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .clusteringKey(Key.ofInt("ck", 200)) + .where(column("col1").isEqualToInt(10)) + .or(column("col2").isEqualToInt(30)) + .build(), + metadata)) + .isEqualTo( + Get.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .clusteringKey(Key.ofInt("ck", 200)) + .where(column("col1").isEqualToInt(10)) + .or(column(Attribute.BEFORE_PREFIX + "col1").isEqualToInt(10)) + .or(column("col2").isEqualToInt(30)) + .or(column(Attribute.BEFORE_PREFIX + "col2").isEqualToInt(30)) + .consistency(Consistency.LINEARIZABLE) + .build()); + + // Complex condition (AND + OR) + assertThat( + ConsensusCommitUtils.prepareGetForStorage( + Get.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .clusteringKey(Key.ofInt("ck", 200)) + .where( + condition(column("col1").isGreaterThanInt(10)) + .and(column("col1").isLessThanInt(20)) + .build()) + .or( + condition(column("col2").isGreaterThanInt(30)) + .and(column("col2").isLessThanOrEqualToInt(40)) + .build()) + .build(), + metadata)) + .isEqualTo( + Get.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .clusteringKey(Key.ofInt("ck", 200)) + .where( + condition(column("col1").isGreaterThanInt(10)) + .and(column("col1").isLessThanInt(20)) + .build()) + .or( + condition(column(Attribute.BEFORE_PREFIX + "col1").isGreaterThanInt(10)) + .and(column(Attribute.BEFORE_PREFIX + "col1").isLessThanInt(20)) + .build()) + .or( + condition(column("col2").isGreaterThanInt(30)) + .and(column("col2").isLessThanOrEqualToInt(40)) + .build()) + .or( + condition(column(Attribute.BEFORE_PREFIX + "col2").isGreaterThanInt(30)) + .and(column(Attribute.BEFORE_PREFIX + "col2").isLessThanOrEqualToInt(40)) + .build()) + .consistency(Consistency.LINEARIZABLE) + .build()); + + // LIKE expression (should be converted) + assertThat( + ConsensusCommitUtils.prepareGetForStorage( + Get.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .clusteringKey(Key.ofInt("ck", 200)) + .where(column("col4").isLikeText("pattern%")) + .build(), + metadata)) + .isEqualTo( + Get.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .clusteringKey(Key.ofInt("ck", 200)) + .where(column("col4").isLikeText("pattern%")) + .or(column(Attribute.BEFORE_PREFIX + "col4").isLikeText("pattern%")) + .consistency(Consistency.LINEARIZABLE) + .build()); + + // LIKE expression with escape (should be converted) + assertThat( + ConsensusCommitUtils.prepareGetForStorage( + Get.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .clusteringKey(Key.ofInt("ck", 200)) + .where(column("col4").isLikeText("%pattern%", "\\")) + .build(), + metadata)) + .isEqualTo( + Get.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .clusteringKey(Key.ofInt("ck", 200)) + .where(column("col4").isLikeText("%pattern%", "\\")) + .or(column(Attribute.BEFORE_PREFIX + "col4").isLikeText("%pattern%", "\\")) + .consistency(Consistency.LINEARIZABLE) + .build()); + } + + @Test + public void prepareScanForStorage_ScanWithoutConjunctionsAndLimit_shouldReturnPreparedScan() { + // Arrange + Scan scan = + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofText("pk", "key1")) + .projections("col1", "col2") + .build(); + + TableMetadata metadata = + TableMetadata.newBuilder() + .addColumn("pk", DataType.TEXT) + .addColumn("col1", DataType.INT) + .addColumn("col2", DataType.TEXT) + .addPartitionKey("pk") + .build(); + + // Act + Scan actual = ConsensusCommitUtils.prepareScanForStorage(scan, metadata); + + // Assert + assertThat(actual) + .isEqualTo( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofText("pk", "key1")) + .consistency(Consistency.LINEARIZABLE) + .build()); + } + + @Test + public void prepareScanForStorage_ScanWithLimit_shouldRemoveLimit() { + // Arrange + Scan scan = + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofText("pk", "key1")) + .limit(10) + .build(); + + TableMetadata metadata = + TableMetadata.newBuilder().addColumn("pk", DataType.TEXT).addPartitionKey("pk").build(); + + // Act + Scan actual = ConsensusCommitUtils.prepareScanForStorage(scan, metadata); + + // Assert + assertThat(actual) + .isEqualTo( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofText("pk", "key1")) + .consistency(Consistency.LINEARIZABLE) + .build()); + } + + @Test + public void prepareScanForStorage_ScanWithConjunctions_shouldConvertConjunctions() { + // Arrange + TableMetadata metadata = + ConsensusCommitUtils.buildTransactionTableMetadata( + TableMetadata.newBuilder() + .addColumn("pk", DataType.INT) + .addColumn("ck", DataType.INT) + .addColumn("col1", DataType.INT) + .addColumn("col2", DataType.INT) + .addColumn("col3", DataType.INT) + .addColumn("col4", DataType.TEXT) + .addPartitionKey("pk") + .addClusteringKey("ck") + .addSecondaryIndex("col3") + .build()); + + // Act Assert + + // Single condition + assertThat( + ConsensusCommitUtils.prepareScanForStorage( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .where(column("col1").isEqualToInt(10)) + .build(), + metadata)) + .isEqualTo( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .where(column("col1").isEqualToInt(10)) + .or(column(Attribute.BEFORE_PREFIX + "col1").isEqualToInt(10)) + .consistency(Consistency.LINEARIZABLE) + .build()); + + // AND condition + assertThat( + ConsensusCommitUtils.prepareScanForStorage( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .where(column("col1").isEqualToInt(10)) + .and(column("col2").isGreaterThanInt(20)) + .build(), + metadata)) + .isEqualTo( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .where( + condition(column("col1").isEqualToInt(10)) + .and(column("col2").isGreaterThanInt(20)) + .build()) + .or( + condition(column(Attribute.BEFORE_PREFIX + "col1").isEqualToInt(10)) + .and(column(Attribute.BEFORE_PREFIX + "col2").isGreaterThanInt(20)) + .build()) + .consistency(Consistency.LINEARIZABLE) + .build()); + + // OR condition + assertThat( + ConsensusCommitUtils.prepareScanForStorage( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .where(column("col1").isEqualToInt(10)) + .or(column("col2").isEqualToInt(30)) + .build(), + metadata)) + .isEqualTo( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .where(column("col1").isEqualToInt(10)) + .or(column(Attribute.BEFORE_PREFIX + "col1").isEqualToInt(10)) + .or(column("col2").isEqualToInt(30)) + .or(column(Attribute.BEFORE_PREFIX + "col2").isEqualToInt(30)) + .consistency(Consistency.LINEARIZABLE) + .build()); + + // Complex condition (AND + OR) + assertThat( + ConsensusCommitUtils.prepareScanForStorage( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .where( + condition(column("col1").isGreaterThanInt(10)) + .and(column("col1").isLessThanInt(20)) + .build()) + .or( + condition(column("col2").isGreaterThanInt(30)) + .and(column("col2").isLessThanOrEqualToInt(40)) + .build()) + .build(), + metadata)) + .isEqualTo( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .where( + condition(column("col1").isGreaterThanInt(10)) + .and(column("col1").isLessThanInt(20)) + .build()) + .or( + condition(column(Attribute.BEFORE_PREFIX + "col1").isGreaterThanInt(10)) + .and(column(Attribute.BEFORE_PREFIX + "col1").isLessThanInt(20)) + .build()) + .or( + condition(column("col2").isGreaterThanInt(30)) + .and(column("col2").isLessThanOrEqualToInt(40)) + .build()) + .or( + condition(column(Attribute.BEFORE_PREFIX + "col2").isGreaterThanInt(30)) + .and(column(Attribute.BEFORE_PREFIX + "col2").isLessThanOrEqualToInt(40)) + .build()) + .consistency(Consistency.LINEARIZABLE) + .build()); + + // LIKE expression (should be converted) + assertThat( + ConsensusCommitUtils.prepareScanForStorage( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .where(column("col4").isLikeText("pattern%")) + .build(), + metadata)) + .isEqualTo( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .where(column("col4").isLikeText("pattern%")) + .or(column(Attribute.BEFORE_PREFIX + "col4").isLikeText("pattern%")) + .consistency(Consistency.LINEARIZABLE) + .build()); + + // LIKE expression with escape (should be converted) + assertThat( + ConsensusCommitUtils.prepareScanForStorage( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .where(column("col4").isLikeText("%pattern%", "\\")) + .build(), + metadata)) + .isEqualTo( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .partitionKey(Key.ofInt("pk", 100)) + .where(column("col4").isLikeText("%pattern%", "\\")) + .or(column(Attribute.BEFORE_PREFIX + "col4").isLikeText("%pattern%", "\\")) + .consistency(Consistency.LINEARIZABLE) + .build()); + + // Condition on partition key (should not be converted) + assertThat( + ConsensusCommitUtils.prepareScanForStorage( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .all() + .where(column("pk").isEqualToInt(100)) + .build(), + metadata)) + .isEqualTo( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .all() + .where(column("pk").isEqualToInt(100)) + .consistency(Consistency.LINEARIZABLE) + .build()); + + // Condition on partition key and other columns (only other columns should be converted) + assertThat( + ConsensusCommitUtils.prepareScanForStorage( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .all() + .where(column("pk").isEqualToInt(50)) + .and(column("col1").isEqualToInt(10)) + .build(), + metadata)) + .isEqualTo( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .all() + .where( + condition(column("pk").isEqualToInt(50)) + .and(column("col1").isEqualToInt(10)) + .build()) + .or( + condition(column("pk").isEqualToInt(50)) + .and(column(Attribute.BEFORE_PREFIX + "col1").isEqualToInt(10)) + .build()) + .consistency(Consistency.LINEARIZABLE) + .build()); + + assertThat( + ConsensusCommitUtils.prepareScanForStorage( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .all() + .where(column("pk").isEqualToInt(50)) + .or(column("col1").isEqualToInt(10)) + .build(), + metadata)) + .isEqualTo( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .all() + .where(column("pk").isEqualToInt(50)) + .or(column("col1").isEqualToInt(10)) + .or(column(Attribute.BEFORE_PREFIX + "col1").isEqualToInt(10)) + .consistency(Consistency.LINEARIZABLE) + .build()); + + // Condition on clustering key (should not be converted) + assertThat( + ConsensusCommitUtils.prepareScanForStorage( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .all() + .where(column("ck").isGreaterThanInt(150)) + .build(), + metadata)) + .isEqualTo( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .all() + .where(column("ck").isGreaterThanInt(150)) + .consistency(Consistency.LINEARIZABLE) + .build()); + + // Condition on clustering key and other columns (only other columns should be converted) + assertThat( + ConsensusCommitUtils.prepareScanForStorage( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .all() + .where(column("ck").isEqualToInt(150)) + .and(column("col1").isEqualToInt(10)) + .build(), + metadata)) + .isEqualTo( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .all() + .where( + condition(column("ck").isEqualToInt(150)) + .and(column("col1").isEqualToInt(10)) + .build()) + .or( + condition(column("ck").isEqualToInt(150)) + .and(column(Attribute.BEFORE_PREFIX + "col1").isEqualToInt(10)) + .build()) + .consistency(Consistency.LINEARIZABLE) + .build()); + + assertThat( + ConsensusCommitUtils.prepareScanForStorage( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .all() + .where(column("ck").isEqualToInt(150)) + .or(column("col1").isEqualToInt(10)) + .build(), + metadata)) + .isEqualTo( + Scan.newBuilder() + .namespace("ns") + .table("tbl") + .all() + .where(column("ck").isEqualToInt(150)) + .or(column("col1").isEqualToInt(10)) + .or(column(Attribute.BEFORE_PREFIX + "col1").isEqualToInt(10)) + .consistency(Consistency.LINEARIZABLE) + .build()); + } } diff --git a/core/src/test/java/com/scalar/db/transaction/consensuscommit/CrudHandlerTest.java b/core/src/test/java/com/scalar/db/transaction/consensuscommit/CrudHandlerTest.java index 18aaf365d3..8b21be0aa1 100644 --- a/core/src/test/java/com/scalar/db/transaction/consensuscommit/CrudHandlerTest.java +++ b/core/src/test/java/com/scalar/db/transaction/consensuscommit/CrudHandlerTest.java @@ -38,6 +38,7 @@ import com.scalar.db.exception.transaction.ValidationConflictException; import com.scalar.db.io.Column; import com.scalar.db.io.DataType; +import com.scalar.db.io.IntColumn; import com.scalar.db.io.Key; import com.scalar.db.io.TextColumn; import com.scalar.db.util.ScalarDbUtils; @@ -76,6 +77,8 @@ public class CrudHandlerTest { private static final String ANY_TEXT_3 = "text3"; private static final String ANY_TEXT_4 = "text4"; private static final String ANY_TEXT_5 = "text5"; + private static final int ANY_INT_1 = 100; + private static final int ANY_INT_2 = 200; private static final TableMetadata TABLE_METADATA = ConsensusCommitUtils.buildTransactionTableMetadata( @@ -141,7 +144,7 @@ private Scan prepareCrossPartitionScan() { .namespace(ANY_NAMESPACE_NAME) .table(ANY_TABLE_NAME) .all() - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) + .where(column(ANY_NAME_4).isEqualToInt(ANY_INT_1)) .build(); } @@ -160,10 +163,11 @@ private TransactionResult prepareResult( .put(ANY_NAME_1, TextColumn.of(ANY_NAME_1, partitionKeyColumnValue)) .put(ANY_NAME_2, TextColumn.of(ANY_NAME_2, clusteringKeyColumnValue)) .put(ANY_NAME_3, TextColumn.of(ANY_NAME_3, ANY_TEXT_3)) - .put(Attribute.ID, ScalarDbUtils.toColumn(Attribute.toIdValue(ANY_ID_2))) - .put(Attribute.STATE, ScalarDbUtils.toColumn(Attribute.toStateValue(state))) - .put(Attribute.VERSION, ScalarDbUtils.toColumn(Attribute.toVersionValue(2))) - .put(Attribute.BEFORE_ID, ScalarDbUtils.toColumn(Attribute.toBeforeIdValue(ANY_ID_1))) + .put(ANY_NAME_4, IntColumn.of(ANY_NAME_4, ANY_INT_1)) + .put(Attribute.ID, TextColumn.of(Attribute.ID, ANY_ID_2)) + .put(Attribute.STATE, IntColumn.of(Attribute.STATE, state.get())) + .put(Attribute.VERSION, IntColumn.of(Attribute.VERSION, 2)) + .put(Attribute.BEFORE_ID, TextColumn.of(Attribute.BEFORE_ID, ANY_ID_1)) .put( Attribute.BEFORE_STATE, ScalarDbUtils.toColumn(Attribute.toBeforeStateValue(TransactionState.COMMITTED))) @@ -177,11 +181,10 @@ private TransactionResult prepareResult( public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudException { // Arrange Get get = prepareGet(); - Get getForStorage = toGetForStorageFrom(get); Snapshot.Key key = new Snapshot.Key(get); Optional expected = Optional.of(prepareResult(TransactionState.COMMITTED)); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(true); - when(snapshot.getResult(key, getForStorage)).thenReturn(expected); + when(snapshot.containsKeyInGetSet(get)).thenReturn(true); + when(snapshot.getResult(key, get)).thenReturn(expected); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); @@ -205,10 +208,10 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept Get getForStorage = toGetForStorageFrom(get); Optional expected = Optional.of(prepareResult(TransactionState.COMMITTED)); Optional transactionResult = expected.map(e -> (TransactionResult) e); - Snapshot.Key key = new Snapshot.Key(getForStorage); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(false); + Snapshot.Key key = new Snapshot.Key(get); + when(snapshot.containsKeyInGetSet(get)).thenReturn(false); when(storage.get(getForStorage)).thenReturn(expected); - when(snapshot.getResult(key, getForStorage)).thenReturn(transactionResult); + when(snapshot.getResult(key, get)).thenReturn(transactionResult); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); @@ -235,10 +238,10 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept Get getForStorage = toGetForStorageFrom(get); Optional expected = Optional.of(prepareResult(TransactionState.COMMITTED)); Optional transactionResult = expected.map(e -> (TransactionResult) e); - Snapshot.Key key = new Snapshot.Key(getForStorage); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(false); + Snapshot.Key key = new Snapshot.Key(get); + when(snapshot.containsKeyInGetSet(get)).thenReturn(false); when(storage.get(getForStorage)).thenReturn(expected); - when(snapshot.getResult(key, getForStorage)).thenReturn(transactionResult); + when(snapshot.getResult(key, get)).thenReturn(transactionResult); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, true, false); @@ -265,10 +268,10 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept Get getForStorage = toGetForStorageFrom(get); Optional expected = Optional.of(prepareResult(TransactionState.COMMITTED)); Optional transactionResult = expected.map(e -> (TransactionResult) e); - Snapshot.Key key = new Snapshot.Key(getForStorage); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(false); + Snapshot.Key key = new Snapshot.Key(get); + when(snapshot.containsKeyInGetSet(get)).thenReturn(false); when(storage.get(getForStorage)).thenReturn(expected); - when(snapshot.mergeResult(key, transactionResult, getForStorage.getConjunctions())) + when(snapshot.mergeResult(key, transactionResult, get.getConjunctions())) .thenReturn(transactionResult); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, true, true); @@ -296,10 +299,10 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept Get getForStorage = toGetForStorageFrom(get); Optional expected = Optional.of(prepareResult(TransactionState.COMMITTED)); Optional transactionResult = expected.map(e -> (TransactionResult) e); - Snapshot.Key key = new Snapshot.Key(getForStorage); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(false); + Snapshot.Key key = new Snapshot.Key(get); + when(snapshot.containsKeyInGetSet(get)).thenReturn(false); when(storage.get(getForStorage)).thenReturn(expected); - when(snapshot.mergeResult(key, transactionResult, getForStorage.getConjunctions())) + when(snapshot.mergeResult(key, transactionResult, get.getConjunctions())) .thenReturn(transactionResult); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SERIALIZABLE, true, true); @@ -323,14 +326,19 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept get_GetWithConjunction_GetNotExistsInSnapshotAndRecordInStorageCommitted_InOneOperationMode_ValidationRequired_ShouldReturnFromStorageAndUpdateSnapshot() throws CrudException, ExecutionException { // Arrange - ConditionalExpression condition = column(ANY_NAME_3).isEqualToText(ANY_TEXT_3); + ConditionalExpression condition = column(ANY_NAME_4).isEqualToInt(ANY_INT_1); Get get = Get.newBuilder(prepareGet()).where(condition).build(); - Get getForStorage = toGetForStorageFrom(get); + Get getForStorage = + Get.newBuilder(toGetForStorageFrom(get)) + .clearConditions() + .where(condition) + .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .build(); Optional expected = Optional.of(prepareResult(TransactionState.COMMITTED)); Optional transactionResult = expected.map(e -> (TransactionResult) e); - Snapshot.Key key = new Snapshot.Key(getForStorage); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(false); - when(storage.get(any())).thenReturn(expected); + Snapshot.Key key = new Snapshot.Key(get); + when(snapshot.containsKeyInGetSet(get)).thenReturn(false); + when(storage.get(getForStorage)).thenReturn(expected); when(snapshot.mergeResult( key, transactionResult, Collections.singleton(Selection.Conjunction.of(condition)))) .thenReturn(transactionResult); @@ -346,13 +354,7 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept Optional.of( new FilteredResult( expected.get(), Collections.emptyList(), TABLE_METADATA, false))); - verify(storage) - .get( - Get.newBuilder(getForStorage) - .clearConditions() - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .build()); + verify(storage).get(getForStorage); verify(snapshot, never()).putIntoReadSet(any(), any()); verify(snapshot).putIntoGetSet(get, Optional.of((TransactionResult) expected.get())); } @@ -366,10 +368,10 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept Get getForStorage = toGetForStorageFrom(get); Optional expected = Optional.of(prepareResult(TransactionState.COMMITTED)); Optional transactionResult = expected.map(e -> (TransactionResult) e); - Snapshot.Key key = new Snapshot.Key(getForStorage); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(false); - when(storage.get(any())).thenReturn(expected); - when(snapshot.mergeResult(key, transactionResult, getForStorage.getConjunctions())) + Snapshot.Key key = new Snapshot.Key(get); + when(snapshot.containsKeyInGetSet(get)).thenReturn(false); + when(storage.get(getForStorage)).thenReturn(expected); + when(snapshot.mergeResult(key, transactionResult, get.getConjunctions())) .thenReturn(transactionResult); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.READ_COMMITTED, false, false); @@ -395,13 +397,12 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept // Arrange Get get = Get.newBuilder(prepareGet()).build(); Get getForStorage = toGetForStorageFrom(get); - Optional expected = Optional.of(prepareResult(TransactionState.COMMITTED)); Optional transactionResult = expected.map(e -> (TransactionResult) e); - Snapshot.Key key = new Snapshot.Key(getForStorage); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(false); - when(storage.get(any())).thenReturn(expected); - when(snapshot.mergeResult(key, transactionResult, getForStorage.getConjunctions())) + Snapshot.Key key = new Snapshot.Key(get); + when(snapshot.containsKeyInGetSet(get)).thenReturn(false); + when(storage.get(getForStorage)).thenReturn(expected); + when(snapshot.mergeResult(key, transactionResult, get.getConjunctions())) .thenReturn(transactionResult); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.READ_COMMITTED, true, false); @@ -430,13 +431,13 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept Get getForStorage = toGetForStorageFrom(get); result = prepareResult(TransactionState.PREPARED); when(storage.get(getForStorage)).thenReturn(Optional.of(result)); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(false); + when(snapshot.containsKeyInGetSet(get)).thenReturn(false); TransactionResult expected = mock(TransactionResult.class); when(expected.getContainedColumnNames()).thenReturn(Collections.singleton(ANY_NAME_1)); when(expected.getAsObject(ANY_NAME_1)).thenReturn(ANY_TEXT_1); - when(snapshot.getResult(key, getForStorage)).thenReturn(Optional.of(expected)); + when(snapshot.getResult(key, get)).thenReturn(Optional.of(expected)); TransactionResult recoveredResult = mock(TransactionResult.class); @SuppressWarnings("unchecked") @@ -444,7 +445,7 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept when(recoveryExecutor.execute( key, - getForStorage, + get, new TransactionResult(result), ANY_ID_1, RecoveryExecutor.RecoveryType.RETURN_LATEST_RESULT_AND_RECOVER)) @@ -461,12 +462,12 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept verify(recoveryExecutor) .execute( key, - getForStorage, + get, new TransactionResult(result), ANY_ID_1, RecoveryExecutor.RecoveryType.RETURN_LATEST_RESULT_AND_RECOVER); verify(snapshot).putIntoReadSet(key, Optional.of(recoveredResult)); - verify(snapshot).putIntoGetSet(getForStorage, Optional.of(recoveredResult)); + verify(snapshot).putIntoGetSet(get, Optional.of(recoveredResult)); assertThat(actual) .isEqualTo( @@ -484,7 +485,7 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept Get getForStorage = toGetForStorageFrom(get); result = prepareResult(TransactionState.PREPARED); when(storage.get(getForStorage)).thenReturn(Optional.of(result)); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(false); + when(snapshot.containsKeyInGetSet(get)).thenReturn(false); TransactionResult expected = mock(TransactionResult.class); when(expected.getContainedColumnNames()).thenReturn(Collections.singleton(ANY_NAME_1)); @@ -498,13 +499,13 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept when(recoveryExecutor.execute( key, - getForStorage, + get, transactionResult, ANY_ID_1, RecoveryExecutor.RecoveryType.RETURN_COMMITTED_RESULT_AND_RECOVER)) .thenReturn(new RecoveryExecutor.Result(key, Optional.of(recoveredResult), recoveryFuture)); - when(snapshot.mergeResult(key, Optional.of(recoveredResult), getForStorage.getConjunctions())) + when(snapshot.mergeResult(key, Optional.of(recoveredResult), get.getConjunctions())) .thenReturn(Optional.of(expected)); TransactionContext context = @@ -518,7 +519,7 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept verify(recoveryExecutor) .execute( key, - getForStorage, + get, transactionResult, ANY_ID_1, RecoveryExecutor.RecoveryType.RETURN_COMMITTED_RESULT_AND_RECOVER); @@ -541,7 +542,7 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept Get getForStorage = toGetForStorageFrom(get); result = prepareResult(TransactionState.PREPARED); when(storage.get(getForStorage)).thenReturn(Optional.of(result)); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(false); + when(snapshot.containsKeyInGetSet(get)).thenReturn(false); TransactionResult expected = mock(TransactionResult.class); when(expected.getContainedColumnNames()).thenReturn(Collections.singleton(ANY_NAME_1)); @@ -555,13 +556,13 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept when(recoveryExecutor.execute( key, - getForStorage, + get, transactionResult, ANY_ID_1, RecoveryExecutor.RecoveryType.RETURN_COMMITTED_RESULT_AND_NOT_RECOVER)) .thenReturn(new RecoveryExecutor.Result(key, Optional.of(recoveredResult), recoveryFuture)); - when(snapshot.mergeResult(key, Optional.of(recoveredResult), getForStorage.getConjunctions())) + when(snapshot.mergeResult(key, Optional.of(recoveredResult), get.getConjunctions())) .thenReturn(Optional.of(expected)); TransactionContext context = @@ -575,7 +576,7 @@ public void get_GetExistsInSnapshot_ShouldReturnFromSnapshot() throws CrudExcept verify(recoveryExecutor) .execute( key, - getForStorage, + get, transactionResult, ANY_ID_1, RecoveryExecutor.RecoveryType.RETURN_COMMITTED_RESULT_AND_NOT_RECOVER); @@ -594,7 +595,7 @@ public void get_GetNotExistsInSnapshotAndRecordNotExistsInStorage_ShouldReturnEm // Arrange Get get = prepareGet(); Get getForStorage = toGetForStorageFrom(get); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(false); + when(snapshot.containsKeyInGetSet(get)).thenReturn(false); when(storage.get(getForStorage)).thenReturn(Optional.empty()); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); @@ -603,6 +604,9 @@ public void get_GetNotExistsInSnapshotAndRecordNotExistsInStorage_ShouldReturnEm Optional result = handler.get(get, context); // Assert + verify(storage).get(getForStorage); + verify(snapshot).putIntoReadSet(new Snapshot.Key(get), Optional.empty()); + verify(snapshot).putIntoGetSet(get, Optional.empty()); assertThat(result.isPresent()).isFalse(); } @@ -612,7 +616,7 @@ public void get_GetNotExistsInSnapshotAndExceptionThrownInStorage_ShouldThrowCru // Arrange Get get = prepareGet(); Get getForStorage = toGetForStorageFrom(get); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(false); + when(snapshot.containsKeyInGetSet(get)).thenReturn(false); ExecutionException toThrow = mock(ExecutionException.class); when(storage.get(getForStorage)).thenThrow(toThrow); TransactionContext context = @@ -628,22 +632,21 @@ public void get_GetNotExistsInSnapshotAndExceptionThrownInStorage_ShouldThrowCru public void get_CalledTwice_SecondTimeShouldReturnTheSameFromSnapshot() throws ExecutionException, CrudException { // Arrange - Get originalGet = prepareGet(); - Get getForStorage = toGetForStorageFrom(originalGet); - Get get1 = prepareGet(); - Get get2 = prepareGet(); + Get get = prepareGet(); + Get getForStorage = toGetForStorageFrom(get); + Get anotherGet = prepareGet(); Result result = prepareResult(TransactionState.COMMITTED); Optional expected = Optional.of(new TransactionResult(result)); - Snapshot.Key key = new Snapshot.Key(getForStorage); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(false).thenReturn(true); - when(snapshot.getResult(key, getForStorage)).thenReturn(expected).thenReturn(expected); + Snapshot.Key key = new Snapshot.Key(get); + when(snapshot.containsKeyInGetSet(get)).thenReturn(false).thenReturn(true); + when(snapshot.getResult(key, get)).thenReturn(expected).thenReturn(expected); when(storage.get(getForStorage)).thenReturn(Optional.of(result)); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - Optional results1 = handler.get(get1, context); - Optional results2 = handler.get(get2, context); + Optional results1 = handler.get(get, context); + Optional results2 = handler.get(anotherGet, context); // Assert verify(snapshot).putIntoReadSet(key, expected); @@ -653,7 +656,7 @@ public void get_CalledTwice_SecondTimeShouldReturnTheSameFromSnapshot() new FilteredResult( expected.get(), Collections.emptyList(), TABLE_METADATA, false))); assertThat(results1).isEqualTo(results2); - verify(storage, never()).get(originalGet); + verify(storage, never()).get(get); verify(storage).get(getForStorage); } @@ -661,24 +664,23 @@ public void get_CalledTwice_SecondTimeShouldReturnTheSameFromSnapshot() public void get_CalledTwice_ReadCommittedIsolation_BothShouldReturnFromStorage() throws ExecutionException, CrudException { // Arrange - Get originalGet = prepareGet(); - Get getForStorage = toGetForStorageFrom(originalGet); - Get get1 = prepareGet(); - Get get2 = prepareGet(); + Get get = prepareGet(); + Get getForStorage = toGetForStorageFrom(get); + Get anotherGet = prepareGet(); Result result = prepareResult(TransactionState.COMMITTED); Optional expected = Optional.of(new TransactionResult(result)); - Snapshot.Key key = new Snapshot.Key(getForStorage); - when(snapshot.containsKeyInGetSet(getForStorage)).thenReturn(false); + Snapshot.Key key = new Snapshot.Key(get); + when(snapshot.containsKeyInGetSet(get)).thenReturn(false); when(snapshot.mergeResult( - key, Optional.of(new TransactionResult(result)), getForStorage.getConjunctions())) + key, Optional.of(new TransactionResult(result)), get.getConjunctions())) .thenReturn(expected); when(storage.get(getForStorage)).thenReturn(Optional.of(result)); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.READ_COMMITTED, false, false); // Act - Optional results1 = handler.get(get1, context); - Optional results2 = handler.get(get2, context); + Optional results1 = handler.get(get, context); + Optional results2 = handler.get(anotherGet, context); // Assert verify(storage, times(2)).get(getForStorage); @@ -695,10 +697,9 @@ public void get_CalledTwice_ReadCommittedIsolation_BothShouldReturnFromStorage() public void get_CalledTwiceUnderRealSnapshot_SecondTimeShouldReturnTheSameFromSnapshot() throws ExecutionException, CrudException { // Arrange - Get originalGet = prepareGet(); - Get getForStorage = toGetForStorageFrom(originalGet); - Get get1 = prepareGet(); - Get get2 = prepareGet(); + Get get = prepareGet(); + Get getForStorage = toGetForStorageFrom(get); + Get anotherGet = prepareGet(); Result result = prepareResult(TransactionState.COMMITTED); Optional expected = Optional.of(new TransactionResult(result)); snapshot = new Snapshot(ANY_ID_1, Isolation.SNAPSHOT, tableMetadataManager, parallelExecutor); @@ -707,8 +708,8 @@ public void get_CalledTwiceUnderRealSnapshot_SecondTimeShouldReturnTheSameFromSn new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - Optional results1 = handler.get(get1, context); - Optional results2 = handler.get(get2, context); + Optional results1 = handler.get(get, context); + Optional results2 = handler.get(anotherGet, context); // Assert assertThat(results1) @@ -717,7 +718,7 @@ public void get_CalledTwiceUnderRealSnapshot_SecondTimeShouldReturnTheSameFromSn new FilteredResult( expected.get(), Collections.emptyList(), TABLE_METADATA, false))); assertThat(results1).isEqualTo(results2); - verify(storage, never()).get(originalGet); + verify(storage, never()).get(get); verify(storage).get(getForStorage); } @@ -725,10 +726,9 @@ public void get_CalledTwiceUnderRealSnapshot_SecondTimeShouldReturnTheSameFromSn public void get_CalledTwiceUnderRealSnapshot_ReadCommittedIsolation_BothShouldReturnFromStorage() throws ExecutionException, CrudException { // Arrange - Get originalGet = prepareGet(); - Get getForStorage = toGetForStorageFrom(originalGet); - Get get1 = prepareGet(); - Get get2 = prepareGet(); + Get get = prepareGet(); + Get getForStorage = toGetForStorageFrom(get); + Get anotherGet = prepareGet(); Result result = prepareResult(TransactionState.COMMITTED); Optional expected = Optional.of(new TransactionResult(result)); snapshot = @@ -738,8 +738,8 @@ public void get_CalledTwiceUnderRealSnapshot_ReadCommittedIsolation_BothShouldRe new TransactionContext(ANY_ID_1, snapshot, Isolation.READ_COMMITTED, false, false); // Act - Optional results1 = handler.get(get1, context); - Optional results2 = handler.get(get2, context); + Optional results1 = handler.get(get, context); + Optional results2 = handler.get(anotherGet, context); // Assert verify(storage, times(2)).get(getForStorage); @@ -780,14 +780,14 @@ public void get_DifferentGetButSameRecordReturned_ShouldNotOverwriteReadSet() throws ExecutionException, CrudException { // Arrange Get get1 = prepareGet(); - Get get2 = Get.newBuilder(get1).where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)).build(); + Get get2 = Get.newBuilder(get1).where(column(ANY_NAME_4).isEqualToInt(ANY_INT_1)).build(); Get getForStorage1 = toGetForStorageFrom(get1); Get getForStorage2 = Get.newBuilder(get2) .clearProjections() .clearConditions() - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) + .where(column(ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isEqualToInt(ANY_INT_1)) .consistency(Consistency.LINEARIZABLE) .build(); Result result = prepareResult(TransactionState.COMMITTED); @@ -838,6 +838,7 @@ void scanOrGetScanner_ResultGivenFromStorage_ShouldUpdateSnapshotAndReturn(ScanT List results = scanOrGetScanner(scan, scanType, context); // Assert + verify(storage).scan(scanForStorage); verify(scanner).close(); verify(snapshot).putIntoReadSet(key, Optional.of(expected)); verify(snapshot).putIntoScanSet(scan, Maps.newLinkedHashMap(ImmutableMap.of(key, expected))); @@ -870,6 +871,7 @@ void scanOrGetScanner_ResultGivenFromStorage_InReadOnlyMode_ShouldUpdateSnapshot List results = scanOrGetScanner(scan, scanType, context); // Assert + verify(storage).scan(scanForStorage); verify(snapshot, never()).putIntoReadSet(any(), any()); verify(snapshot).putIntoScanSet(scan, Maps.newLinkedHashMap(ImmutableMap.of(key, expected))); verify(snapshot, never()).verifyNoOverlap(any(), any()); @@ -901,6 +903,7 @@ void scanOrGetScanner_ResultGivenFromStorage_InReadOnlyMode_ShouldUpdateSnapshot List results = scanOrGetScanner(scan, scanType, context); // Assert + verify(storage).scan(scanForStorage); verify(snapshot, never()).putIntoReadSet(any(), any()); verify(snapshot, never()).putIntoScanSet(any(), any()); verify(snapshot, never()).verifyNoOverlap(any(), any()); @@ -933,6 +936,7 @@ void scanOrGetScanner_ResultGivenFromStorage_InReadOnlyMode_ShouldUpdateSnapshot List results = scanOrGetScanner(scan, scanType, context); // Assert + verify(storage).scan(scanForStorage); verify(snapshot, never()).putIntoReadSet(any(), any()); verify(snapshot).putIntoScanSet(scan, Maps.newLinkedHashMap(ImmutableMap.of(key, expected))); verify(snapshot, never()).verifyNoOverlap(any(), any()); @@ -970,7 +974,7 @@ void scanOrGetScanner_ResultGivenFromStorage_InReadOnlyMode_ShouldUpdateSnapshot when(recoveryExecutor.execute( key, - scanForStorage, + scan, new TransactionResult(result), ANY_ID_1, RecoveryExecutor.RecoveryType.RETURN_LATEST_RESULT_AND_RECOVER)) @@ -983,12 +987,12 @@ void scanOrGetScanner_ResultGivenFromStorage_InReadOnlyMode_ShouldUpdateSnapshot List results = scanOrGetScanner(scan, scanType, context); // Assert + verify(storage).scan(scanForStorage); verify(scanner).close(); verify(snapshot).putIntoReadSet(key, Optional.of(recoveredResult)); verify(snapshot) - .putIntoScanSet( - scanForStorage, Maps.newLinkedHashMap(ImmutableMap.of(key, recoveredResult))); - verify(snapshot).verifyNoOverlap(scanForStorage, ImmutableMap.of(key, recoveredResult)); + .putIntoScanSet(scan, Maps.newLinkedHashMap(ImmutableMap.of(key, recoveredResult))); + verify(snapshot).verifyNoOverlap(scan, ImmutableMap.of(key, recoveredResult)); assertThat(results) .containsExactly( @@ -1024,7 +1028,7 @@ void scanOrGetScanner_ResultGivenFromStorage_InReadOnlyMode_ShouldUpdateSnapshot when(recoveryExecutor.execute( key, - scanForStorage, + scan, new TransactionResult(result), ANY_ID_1, RecoveryExecutor.RecoveryType.RETURN_COMMITTED_RESULT_AND_RECOVER)) @@ -1037,10 +1041,11 @@ void scanOrGetScanner_ResultGivenFromStorage_InReadOnlyMode_ShouldUpdateSnapshot List results = scanOrGetScanner(scan, scanType, context); // Assert + verify(storage).scan(scanForStorage); verify(scanner).close(); verify(snapshot).putIntoReadSet(key, Optional.of(recoveredResult)); verify(snapshot, never()).putIntoScanSet(any(), any()); - verify(snapshot).verifyNoOverlap(scanForStorage, ImmutableMap.of(key, recoveredResult)); + verify(snapshot).verifyNoOverlap(scan, ImmutableMap.of(key, recoveredResult)); assertThat(results) .containsExactly( @@ -1076,7 +1081,7 @@ void scanOrGetScanner_ResultGivenFromStorage_InReadOnlyMode_ShouldUpdateSnapshot when(recoveryExecutor.execute( key, - scanForStorage, + scan, new TransactionResult(result), ANY_ID_1, RecoveryExecutor.RecoveryType.RETURN_COMMITTED_RESULT_AND_NOT_RECOVER)) @@ -1089,6 +1094,7 @@ void scanOrGetScanner_ResultGivenFromStorage_InReadOnlyMode_ShouldUpdateSnapshot List results = scanOrGetScanner(scan, scanType, context); // Assert + verify(storage).scan(scanForStorage); verify(scanner).close(); verify(snapshot, never()).putIntoReadSet(any(), any()); verify(snapshot, never()).putIntoScanSet(any(), any()); @@ -1104,10 +1110,9 @@ void scanOrGetScanner_ResultGivenFromStorage_InReadOnlyMode_ShouldUpdateSnapshot void scanOrGetScanner_CalledTwice_SecondTimeShouldReturnTheSameFromSnapshot(ScanType scanType) throws ExecutionException, CrudException, IOException { // Arrange - Scan originalScan = prepareScan(); - Scan scanForStorage = toScanForStorageFrom(originalScan); - Scan scan1 = prepareScan(); - Scan scan2 = prepareScan(); + Scan scan = prepareScan(); + Scan scanForStorage = toScanForStorageFrom(scan); + Scan anotherScan = prepareScan(); result = prepareResult(TransactionState.COMMITTED); TransactionResult expected = new TransactionResult(result); if (scanType == ScanType.SCAN) { @@ -1116,27 +1121,26 @@ void scanOrGetScanner_CalledTwice_SecondTimeShouldReturnTheSameFromSnapshot(Scan when(scanner.one()).thenReturn(Optional.of(result)).thenReturn(Optional.empty()); } when(storage.scan(scanForStorage)).thenReturn(scanner); - Snapshot.Key key = new Snapshot.Key(scanForStorage, result, TABLE_METADATA); - when(snapshot.getResults(scanForStorage)) + Snapshot.Key key = new Snapshot.Key(scan, result, TABLE_METADATA); + when(snapshot.getResults(scan)) .thenReturn(Optional.empty()) .thenReturn(Optional.of(Maps.newLinkedHashMap(ImmutableMap.of(key, expected)))); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - List results1 = scanOrGetScanner(scan1, scanType, context); - List results2 = scanOrGetScanner(scan2, scanType, context); + List results1 = scanOrGetScanner(scan, scanType, context); + List results2 = scanOrGetScanner(anotherScan, scanType, context); // Assert verify(scanner).close(); verify(snapshot).putIntoReadSet(key, Optional.of(expected)); - verify(snapshot) - .putIntoScanSet(scanForStorage, Maps.newLinkedHashMap(ImmutableMap.of(key, expected))); + verify(snapshot).putIntoScanSet(scan, Maps.newLinkedHashMap(ImmutableMap.of(key, expected))); assertThat(results1.size()).isEqualTo(1); assertThat(results1.get(0)) .isEqualTo(new FilteredResult(expected, Collections.emptyList(), TABLE_METADATA, false)); assertThat(results1).isEqualTo(results2); - verify(storage, never()).scan(originalScan); + verify(storage, never()).scan(scan); verify(storage).scan(scanForStorage); } @@ -1145,10 +1149,9 @@ void scanOrGetScanner_CalledTwice_SecondTimeShouldReturnTheSameFromSnapshot(Scan void scan_CalledTwiceUnderRealSnapshot_SecondTimeShouldReturnTheSameFromSnapshot( ScanType scanType) throws ExecutionException, CrudException, IOException { // Arrange - Scan originalScan = prepareScan(); - Scan scanForStorage = toScanForStorageFrom(originalScan); - Scan scan1 = prepareScan(); - Scan scan2 = prepareScan(); + Scan scan = prepareScan(); + Scan scanForStorage = toScanForStorageFrom(scan); + Scan anotherScan = prepareScan(); result = prepareResult(TransactionState.COMMITTED); TransactionResult expected = new TransactionResult(result); snapshot = new Snapshot(ANY_ID_1, Isolation.SNAPSHOT, tableMetadataManager, parallelExecutor); @@ -1162,8 +1165,8 @@ void scan_CalledTwiceUnderRealSnapshot_SecondTimeShouldReturnTheSameFromSnapshot new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - List results1 = scanOrGetScanner(scan1, scanType, context); - List results2 = scanOrGetScanner(scan2, scanType, context); + List results1 = scanOrGetScanner(scan, scanType, context); + List results2 = scanOrGetScanner(anotherScan, scanType, context); // Assert assertThat(results1.size()).isEqualTo(1); @@ -1172,7 +1175,7 @@ void scan_CalledTwiceUnderRealSnapshot_SecondTimeShouldReturnTheSameFromSnapshot assertThat(results1).isEqualTo(results2); verify(scanner).close(); - verify(storage, never()).scan(originalScan); + verify(storage, never()).scan(scan); verify(storage).scan(scanForStorage); } @@ -1219,7 +1222,8 @@ void scanOrGetScanner_GetCalledAfterScan_ShouldReturnFromStorage(ScanType scanTy void scanOrGetScanner_GetCalledAfterScanUnderRealSnapshot_ShouldReturnFromStorage( ScanType scanType) throws ExecutionException, CrudException, IOException { // Arrange - Scan scan = toScanForStorageFrom(prepareScan()); + Scan scan = prepareScan(); + Scan scanForStorage = toScanForStorageFrom(scan); result = prepareResult(TransactionState.COMMITTED); snapshot = new Snapshot(ANY_ID_1, Isolation.SNAPSHOT, tableMetadataManager, parallelExecutor); if (scanType == ScanType.SCAN) { @@ -1227,7 +1231,8 @@ void scanOrGetScanner_GetCalledAfterScanUnderRealSnapshot_ShouldReturnFromStorag } else { when(scanner.one()).thenReturn(Optional.of(result)).thenReturn(Optional.empty()); } - when(storage.scan(scan)).thenReturn(scanner); + when(storage.scan(scanForStorage)).thenReturn(scanner); + Get get = prepareGet(); Get getForStorage = toGetForStorageFrom(get); when(storage.get(getForStorage)).thenReturn(Optional.of(result)); @@ -1239,8 +1244,8 @@ void scanOrGetScanner_GetCalledAfterScanUnderRealSnapshot_ShouldReturnFromStorag Optional result = handler.get(get, context); // Assert - verify(storage).scan(scan); - verify(storage).get(get); + verify(storage).scan(scanForStorage); + verify(storage).get(getForStorage); verify(scanner).close(); assertThat(results.size()).isEqualTo(1); @@ -1327,6 +1332,12 @@ void scanOrGetScanner_CalledAfterDeleteUnderRealSnapshot_ShouldThrowIllegalArgum ScanType scanType) throws ExecutionException, CrudException, IOException { // Arrange Scan scan = prepareCrossPartitionScan(); + Scan scanForStorage = + Scan.newBuilder(toScanForStorageFrom(scan)) + .clearConditions() + .where(column(ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .build(); result = prepareResult(TransactionState.COMMITTED); Snapshot.Key key = new Snapshot.Key(scan, result, TABLE_METADATA); if (scanType == ScanType.SCAN) { @@ -1343,6 +1354,7 @@ void scanOrGetScanner_CalledAfterDeleteUnderRealSnapshot_ShouldThrowIllegalArgum List results = scanOrGetScanner(scan, scanType, context); // Assert + verify(storage).scan(scanForStorage); verify(scanner).close(); verify(snapshot).putIntoReadSet(key, Optional.of(transactionResult)); verify(snapshot) @@ -1361,10 +1373,15 @@ void scanOrGetScanner_CalledAfterDeleteUnderRealSnapshot_ShouldThrowIllegalArgum ScanType scanType) throws ExecutionException, IOException, CrudException { // Arrange Scan scan = prepareCrossPartitionScan(); - Scan scanForStorage = toScanForStorageFrom(scan); + Scan scanForStorage = + Scan.newBuilder(toScanForStorageFrom(scan)) + .clearConditions() + .where(column(ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .build(); result = prepareResult(TransactionState.PREPARED); - Snapshot.Key key = new Snapshot.Key(scanForStorage, result, TABLE_METADATA); + Snapshot.Key key = new Snapshot.Key(scan, result, TABLE_METADATA); if (scanType == ScanType.SCAN) { when(scanner.iterator()).thenReturn(Collections.singletonList(result).iterator()); } else { @@ -1374,10 +1391,10 @@ void scanOrGetScanner_CalledAfterDeleteUnderRealSnapshot_ShouldThrowIllegalArgum TransactionResult recoveredResult = mock(TransactionResult.class); - when(recoveredResult.getContainedColumnNames()).thenReturn(Collections.singleton(ANY_NAME_3)); - when(recoveredResult.getAsObject(ANY_NAME_3)).thenReturn(ANY_TEXT_3); + when(recoveredResult.getContainedColumnNames()).thenReturn(Collections.singleton(ANY_NAME_4)); + when(recoveredResult.getAsObject(ANY_NAME_4)).thenReturn(ANY_INT_1); when(recoveredResult.getColumns()) - .thenReturn(ImmutableMap.of(ANY_NAME_3, TextColumn.of(ANY_NAME_3, ANY_TEXT_3))); + .thenReturn(ImmutableMap.of(ANY_NAME_4, IntColumn.of(ANY_NAME_4, ANY_INT_1))); when(snapshot.getResult(key)).thenReturn(Optional.of(new TransactionResult(result))); @@ -1386,7 +1403,7 @@ void scanOrGetScanner_CalledAfterDeleteUnderRealSnapshot_ShouldThrowIllegalArgum when(recoveryExecutor.execute( key, - scanForStorage, + scan, new TransactionResult(result), ANY_ID_1, RecoveryExecutor.RecoveryType.RETURN_LATEST_RESULT_AND_RECOVER)) @@ -1396,22 +1413,15 @@ void scanOrGetScanner_CalledAfterDeleteUnderRealSnapshot_ShouldThrowIllegalArgum new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - List results = scanOrGetScanner(scanForStorage, scanType, context); + List results = scanOrGetScanner(scan, scanType, context); // Assert - verify(storage) - .scan( - Scan.newBuilder(scanForStorage) - .clearConditions() - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .build()); + verify(storage).scan(scanForStorage); verify(scanner).close(); verify(snapshot).putIntoReadSet(key, Optional.of(recoveredResult)); verify(snapshot) - .putIntoScanSet( - scanForStorage, Maps.newLinkedHashMap(ImmutableMap.of(key, recoveredResult))); - verify(snapshot).verifyNoOverlap(scanForStorage, ImmutableMap.of(key, recoveredResult)); + .putIntoScanSet(scan, Maps.newLinkedHashMap(ImmutableMap.of(key, recoveredResult))); + verify(snapshot).verifyNoOverlap(scan, ImmutableMap.of(key, recoveredResult)); assertThat(results) .containsExactly( @@ -1425,10 +1435,15 @@ void scanOrGetScanner_CalledAfterDeleteUnderRealSnapshot_ShouldThrowIllegalArgum ScanType scanType) throws ExecutionException, IOException, CrudException { // Arrange Scan scan = prepareCrossPartitionScan(); - Scan scanForStorage = toScanForStorageFrom(scan); + Scan scanForStorage = + Scan.newBuilder(toScanForStorageFrom(scan)) + .clearConditions() + .where(column(ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .build(); result = prepareResult(TransactionState.PREPARED); - Snapshot.Key key = new Snapshot.Key(scanForStorage, result, TABLE_METADATA); + Snapshot.Key key = new Snapshot.Key(scan, result, TABLE_METADATA); if (scanType == ScanType.SCAN) { when(scanner.iterator()).thenReturn(Collections.singletonList(result).iterator()); } else { @@ -1438,10 +1453,10 @@ void scanOrGetScanner_CalledAfterDeleteUnderRealSnapshot_ShouldThrowIllegalArgum TransactionResult recoveredResult = mock(TransactionResult.class); - when(recoveredResult.getContainedColumnNames()).thenReturn(Collections.singleton(ANY_NAME_3)); - when(recoveredResult.getAsObject(ANY_NAME_3)).thenReturn(ANY_TEXT_4); + when(recoveredResult.getContainedColumnNames()).thenReturn(Collections.singleton(ANY_NAME_4)); + when(recoveredResult.getAsObject(ANY_NAME_4)).thenReturn(ANY_INT_2); when(recoveredResult.getColumns()) - .thenReturn(ImmutableMap.of(ANY_NAME_3, TextColumn.of(ANY_NAME_3, ANY_TEXT_4))); + .thenReturn(ImmutableMap.of(ANY_NAME_4, IntColumn.of(ANY_NAME_4, ANY_INT_2))); when(snapshot.getResult(key)).thenReturn(Optional.of(new TransactionResult(result))); @@ -1450,7 +1465,7 @@ void scanOrGetScanner_CalledAfterDeleteUnderRealSnapshot_ShouldThrowIllegalArgum when(recoveryExecutor.execute( key, - scanForStorage, + scan, new TransactionResult(result), ANY_ID_1, RecoveryExecutor.RecoveryType.RETURN_LATEST_RESULT_AND_RECOVER)) @@ -1460,20 +1475,14 @@ void scanOrGetScanner_CalledAfterDeleteUnderRealSnapshot_ShouldThrowIllegalArgum new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - List results = scanOrGetScanner(scanForStorage, scanType, context); + List results = scanOrGetScanner(scan, scanType, context); // Assert - verify(storage) - .scan( - Scan.newBuilder(scanForStorage) - .clearConditions() - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .build()); + verify(storage).scan(scanForStorage); verify(scanner).close(); verify(snapshot, never()).putIntoReadSet(any(), any()); - verify(snapshot).putIntoScanSet(scanForStorage, Maps.newLinkedHashMap()); - verify(snapshot).verifyNoOverlap(scanForStorage, ImmutableMap.of()); + verify(snapshot).putIntoScanSet(scan, Maps.newLinkedHashMap()); + verify(snapshot).verifyNoOverlap(scan, ImmutableMap.of()); assertThat(results).isEmpty(); } @@ -1522,6 +1531,7 @@ void scanOrGetScanner_WithLimit_ShouldReturnLimitedResults(ScanType scanType) .isEqualTo( new FilteredResult(transactionResult2, Collections.emptyList(), TABLE_METADATA, false)); + verify(storage).scan(scanForStorage); verify(scanner).close(); verify(snapshot).putIntoReadSet(key1, Optional.of(transactionResult1)); verify(snapshot).putIntoReadSet(key2, Optional.of(transactionResult2)); @@ -1541,10 +1551,9 @@ void scanOrGetScanner_WithLimit_ShouldReturnLimitedResults(ScanType scanType) void scanOrGetScanner_WithLimitExceedingAvailableResults_ShouldReturnAllAvailableResults( ScanType scanType) throws CrudException, ExecutionException, IOException { // Arrange - Scan scanWithoutLimit = prepareScan(); Scan scanWithLimit = - Scan.newBuilder(scanWithoutLimit).limit(5).build(); // Limit higher than available results - Scan scanForStorage = toScanForStorageFrom(scanWithoutLimit); + Scan.newBuilder(prepareScan()).limit(5).build(); // Limit higher than available results + Scan scanForStorage = Scan.newBuilder(toScanForStorageFrom(scanWithLimit)).limit(0).build(); Result result = prepareResult(TransactionState.COMMITTED); Snapshot.Key key1 = new Snapshot.Key(scanWithLimit, result, TABLE_METADATA); @@ -1566,6 +1575,7 @@ void scanOrGetScanner_WithLimitExceedingAvailableResults_ShouldReturnAllAvailabl // Assert assertThat(results).hasSize(1); + verify(storage).scan(scanForStorage); verify(scanner).close(); verify(snapshot).putIntoReadSet(key1, Optional.of(transactionResult1)); } @@ -1576,10 +1586,8 @@ void scanOrGetScanner_WithLimitExceedingAvailableResults_ShouldReturnAllAvailabl scanOrGetScanner_WithLimit_UncommittedResult_ShouldCallRecoveryExecutorWithReturnLatestResultAndRecover( ScanType scanType) throws ExecutionException, IOException, CrudException { // Arrange - Scan scanWithoutLimit = prepareScan(); - Scan scanWithLimit = Scan.newBuilder(scanWithoutLimit).limit(2).build(); - Scan scanForStorageWithLimit = toScanForStorageFrom(scanWithLimit); - Scan scanForStorageWithoutLimit = toScanForStorageFrom(scanWithoutLimit); + Scan scanWithLimit = Scan.newBuilder(prepareScan()).limit(2).build(); + Scan scanForStorage = Scan.newBuilder(toScanForStorageFrom(scanWithLimit)).limit(0).build(); Result uncommittedResult1 = prepareResult(ANY_TEXT_1, ANY_TEXT_2, TransactionState.DELETED); Result uncommittedResult2 = prepareResult(ANY_TEXT_1, ANY_TEXT_3, TransactionState.PREPARED); @@ -1601,7 +1609,7 @@ void scanOrGetScanner_WithLimitExceedingAvailableResults_ShouldReturnAllAvailabl .thenReturn(Optional.of(uncommittedResult3)) .thenReturn(Optional.empty()); } - when(storage.scan(scanForStorageWithoutLimit)).thenReturn(scanner); + when(storage.scan(scanForStorage)).thenReturn(scanner); TransactionResult recoveredResult1 = mock(TransactionResult.class); when(recoveredResult1.getContainedColumnNames()).thenReturn(Collections.singleton(ANY_NAME_3)); @@ -1620,14 +1628,14 @@ void scanOrGetScanner_WithLimitExceedingAvailableResults_ShouldReturnAllAvailabl when(recoveryExecutor.execute( key1, - scanForStorageWithLimit, + scanWithLimit, new TransactionResult(uncommittedResult1), ANY_ID_1, RecoveryExecutor.RecoveryType.RETURN_LATEST_RESULT_AND_RECOVER)) .thenReturn(new RecoveryExecutor.Result(key1, Optional.empty(), recoveryFuture)); when(recoveryExecutor.execute( key2, - scanForStorageWithLimit, + scanWithLimit, new TransactionResult(uncommittedResult2), ANY_ID_1, RecoveryExecutor.RecoveryType.RETURN_LATEST_RESULT_AND_RECOVER)) @@ -1635,7 +1643,7 @@ void scanOrGetScanner_WithLimitExceedingAvailableResults_ShouldReturnAllAvailabl new RecoveryExecutor.Result(key2, Optional.of(recoveredResult1), recoveryFuture)); when(recoveryExecutor.execute( key3, - scanForStorageWithLimit, + scanWithLimit, new TransactionResult(uncommittedResult3), ANY_ID_1, RecoveryExecutor.RecoveryType.RETURN_LATEST_RESULT_AND_RECOVER)) @@ -1649,18 +1657,17 @@ void scanOrGetScanner_WithLimitExceedingAvailableResults_ShouldReturnAllAvailabl List results = scanOrGetScanner(scanWithLimit, scanType, context); // Assert - verify(storage).scan(scanForStorageWithoutLimit); + verify(storage).scan(scanForStorage); verify(scanner).close(); verify(snapshot).putIntoReadSet(key2, Optional.of(recoveredResult1)); verify(snapshot).putIntoReadSet(key3, Optional.of(recoveredResult2)); verify(snapshot) .putIntoScanSet( - scanForStorageWithLimit, + scanWithLimit, Maps.newLinkedHashMap(ImmutableMap.of(key2, recoveredResult1, key3, recoveredResult2))); verify(snapshot) .verifyNoOverlap( - scanForStorageWithLimit, - ImmutableMap.of(key2, recoveredResult1, key3, recoveredResult2)); + scanWithLimit, ImmutableMap.of(key2, recoveredResult1, key3, recoveredResult2)); assertThat(results) .containsExactly( @@ -1763,6 +1770,7 @@ public void getScanner_ExecutionExceptionThrownByScannerOne_ShouldThrowCrudExcep actualScanner.close(); // Assert + verify(storage).scan(scanForStorage); verify(scanner).close(); verify(snapshot).putIntoReadSet(key1, Optional.of(txResult1)); verify(snapshot) @@ -1797,6 +1805,7 @@ public void getScanner_ExecutionExceptionThrownByScannerOne_ShouldThrowCrudExcep actualScanner.close(); // Assert + verify(storage).scan(scanForStorage); verify(snapshot, never()).putIntoReadSet(any(), any()); verify(snapshot, never()).putIntoScannerSet(any(), any()); verify(snapshot, never()).verifyNoOverlap(any(), any()); @@ -1830,6 +1839,7 @@ public void getScanner_ExecutionExceptionThrownByScannerOne_ShouldThrowCrudExcep actualScanner.close(); // Assert + verify(storage).scan(scanForStorage); verify(snapshot, never()).putIntoReadSet(any(), any()); verify(snapshot) .putIntoScannerSet(scan, Maps.newLinkedHashMap(ImmutableMap.of(key1, txResult1))); @@ -1852,7 +1862,7 @@ public void put_PutWithoutConditionGiven_ShouldCallAppropriateMethods() throws C spied.put(put, context); // Assert - verify(spied, never()).readUnread(any(), any(), any()); + verify(spied, never()).readUnread(any(), any(), any(), any()); verify(snapshot, never()).getResult(any()); verify(mutationConditionsValidator, never()) .checkIfConditionIsSatisfied(any(Put.class), any(), any()); @@ -1878,13 +1888,6 @@ public void put_PutWithoutConditionGiven_ShouldCallAppropriateMethods() throws C when(result.isCommitted()).thenReturn(true); when(snapshot.getResult(key)).thenReturn(Optional.of(result)); - Get getForKey = - Get.newBuilder() - .namespace(key.getNamespace()) - .table(key.getTable()) - .partitionKey(key.getPartitionKey()) - .build(); - CrudHandler spied = spy(handler); TransactionContext context = @@ -1894,7 +1897,7 @@ public void put_PutWithoutConditionGiven_ShouldCallAppropriateMethods() throws C spied.put(put, context); // Assert - verify(spied, never()).readUnread(key, getForKey, context); + verify(spied, never()).readUnread(any(), any(), any(), any()); verify(snapshot).getResult(key); verify(mutationConditionsValidator).checkIfConditionIsSatisfied(put, result, context); verify(snapshot).putIntoWriteSet(key, put); @@ -1932,13 +1935,13 @@ public void put_PutWithoutConditionGiven_ShouldCallAppropriateMethods() throws C TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); - doReturn(Optional.empty()).when(spied).getFromStorage(getForKey, context); + doReturn(Optional.empty()).when(spied).getFromStorage(getForKey, TABLE_METADATA, ANY_ID_1); // Act spied.put(put, context); // Assert - verify(spied).read(key, getForKey, context); + verify(spied).read(key, getForKey, context, TABLE_METADATA); verify(snapshot).getResult(key); verify(mutationConditionsValidator).checkIfConditionIsSatisfied(put, result, context); verify(snapshot).putIntoWriteSet(key, put); @@ -1962,13 +1965,6 @@ public void put_PutWithoutConditionGiven_ShouldCallAppropriateMethods() throws C when(result.isCommitted()).thenReturn(true); when(snapshot.getResult(key)).thenReturn(Optional.of(result)); - Get getForKey = - Get.newBuilder() - .namespace(key.getNamespace()) - .table(key.getTable()) - .partitionKey(key.getPartitionKey()) - .build(); - CrudHandler spied = spy(handler); TransactionContext context = @@ -1978,7 +1974,7 @@ public void put_PutWithoutConditionGiven_ShouldCallAppropriateMethods() throws C spied.put(put, context); // Assert - verify(spied, never()).readUnread(key, getForKey, context); + verify(spied, never()).readUnread(any(), any(), any(), any()); verify(snapshot).getResult(key); verify(mutationConditionsValidator).checkIfConditionIsSatisfied(put, result, context); verify(snapshot).putIntoWriteSet(key, put); @@ -2023,7 +2019,7 @@ public void delete_DeleteWithoutConditionGiven_ShouldCallAppropriateMethods() spied.delete(delete, context); // Assert - verify(spied, never()).readUnread(any(), any(), any()); + verify(spied, never()).readUnread(any(), any(), any(), any()); verify(snapshot, never()).getResult(any()); verify(mutationConditionsValidator, never()) .checkIfConditionIsSatisfied(any(Delete.class), any(), any()); @@ -2047,13 +2043,6 @@ public void delete_DeleteWithConditionGiven_WithResultInReadSet_ShouldCallApprop when(result.isCommitted()).thenReturn(true); when(snapshot.getResult(key)).thenReturn(Optional.of(result)); - Get getForKey = - Get.newBuilder() - .namespace(key.getNamespace()) - .table(key.getTable()) - .partitionKey(key.getPartitionKey()) - .build(); - CrudHandler spied = spy(handler); TransactionContext context = @@ -2063,7 +2052,7 @@ public void delete_DeleteWithConditionGiven_WithResultInReadSet_ShouldCallApprop spied.delete(delete, context); // Assert - verify(spied, never()).readUnread(key, getForKey, context); + verify(spied, never()).readUnread(any(), any(), any(), any()); verify(snapshot).getResult(key); verify(mutationConditionsValidator).checkIfConditionIsSatisfied(delete, result, context); verify(snapshot).putIntoDeleteSet(key, delete); @@ -2097,13 +2086,13 @@ public void delete_DeleteWithConditionGiven_WithoutResultInReadSet_ShouldCallApp TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); - doReturn(Optional.empty()).when(spied).getFromStorage(getForKey, context); + doReturn(Optional.empty()).when(spied).getFromStorage(getForKey, TABLE_METADATA, ANY_ID_1); // Act spied.delete(delete, context); // Assert - verify(spied).read(key, getForKey, context); + verify(spied).read(key, getForKey, context, TABLE_METADATA); verify(snapshot).getResult(key); verify(mutationConditionsValidator).checkIfConditionIsSatisfied(delete, null, context); verify(snapshot).putIntoDeleteSet(key, delete); @@ -2124,15 +2113,16 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() .table(key.getTable()) .partitionKey(key.getPartitionKey()) .build(); + Get getForStorage = toGetForStorageFrom(getForKey); when(snapshot.containsKeyInGetSet(getForKey)).thenReturn(true); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - handler.readUnread(key, getForKey, context); + handler.readUnread(key, getForKey, context, TABLE_METADATA); // Assert - verify(storage, never()).get(any()); + verify(storage, never()).get(getForStorage); verify(snapshot, never()).putIntoReadSet(any(Snapshot.Key.class), any(Optional.class)); verify(snapshot, never()).putIntoGetSet(any(Get.class), any(Optional.class)); } @@ -2152,16 +2142,17 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() .table(key.getTable()) .partitionKey(key.getPartitionKey()) .build(); + Get getForStorage = toGetForStorageFrom(getForKey); when(snapshot.containsKeyInGetSet(getForKey)).thenReturn(false); - when(storage.get(any())).thenReturn(Optional.empty()); + when(storage.get(getForStorage)).thenReturn(Optional.empty()); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - handler.readUnread(key, getForKey, context); + handler.readUnread(key, getForKey, context, TABLE_METADATA); // Assert - verify(storage).get(any()); + verify(storage).get(getForStorage); verify(snapshot).putIntoReadSet(key, Optional.empty()); verify(snapshot).putIntoGetSet(getForKey, Optional.empty()); } @@ -2180,26 +2171,24 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() .namespace(key.getNamespace()) .table(key.getTable()) .partitionKey(key.getPartitionKey()) - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_1)) + .where(column(ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .build(); + Get getForStorage = + Get.newBuilder(toGetForStorageFrom(getForKey)) + .clearConditions() + .where(column(ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isEqualToInt(ANY_INT_1)) .build(); when(snapshot.containsKeyInGetSet(getForKey)).thenReturn(false); - when(storage.get(any())).thenReturn(Optional.empty()); + when(storage.get(getForStorage)).thenReturn(Optional.empty()); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - handler.readUnread(key, getForKey, context); + handler.readUnread(key, getForKey, context, TABLE_METADATA); // Assert - verify(storage) - .get( - Get.newBuilder() - .namespace(key.getNamespace()) - .table(key.getTable()) - .partitionKey(key.getPartitionKey()) - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_1)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_1)) - .build()); + verify(storage).get(getForStorage); verify(snapshot, never()).putIntoReadSet(key, Optional.empty()); verify(snapshot).putIntoGetSet(getForKey, Optional.empty()); } @@ -2213,26 +2202,25 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() when(key.getNamespace()).thenReturn(ANY_NAMESPACE_NAME); when(key.getTable()).thenReturn(ANY_TABLE_NAME); when(key.getPartitionKey()).thenReturn(Key.ofText(ANY_NAME_1, ANY_TEXT_1)); - - when(result.getInt(Attribute.STATE)).thenReturn(TransactionState.COMMITTED.get()); - when(storage.get(any())).thenReturn(Optional.of(result)); - Get getForKey = Get.newBuilder() .namespace(key.getNamespace()) .table(key.getTable()) .partitionKey(key.getPartitionKey()) .build(); + Get getForStorage = toGetForStorageFrom(getForKey); when(snapshot.containsKeyInGetSet(getForKey)).thenReturn(false); + when(result.getInt(Attribute.STATE)).thenReturn(TransactionState.COMMITTED.get()); + when(storage.get(getForStorage)).thenReturn(Optional.of(result)); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - handler.readUnread(key, getForKey, context); + handler.readUnread(key, getForKey, context, TABLE_METADATA); // Assert - verify(storage).get(any()); + verify(storage).get(getForStorage); verify(snapshot).putIntoReadSet(key, Optional.of(new TransactionResult(result))); verify(snapshot).putIntoGetSet(getForKey, Optional.of(new TransactionResult(result))); } @@ -2246,17 +2234,16 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() when(key.getNamespace()).thenReturn(ANY_NAMESPACE_NAME); when(key.getTable()).thenReturn(ANY_TABLE_NAME); when(key.getPartitionKey()).thenReturn(Key.ofText(ANY_NAME_1, ANY_TEXT_1)); - - when(result.getInt(Attribute.STATE)).thenReturn(TransactionState.PREPARED.get()); - when(storage.get(any())).thenReturn(Optional.of(result)); - Get getForKey = Get.newBuilder() .namespace(key.getNamespace()) .table(key.getTable()) .partitionKey(key.getPartitionKey()) .build(); + Get getForStorage = toGetForStorageFrom(getForKey); when(snapshot.containsKeyInGetSet(getForKey)).thenReturn(false); + when(result.getInt(Attribute.STATE)).thenReturn(TransactionState.PREPARED.get()); + when(storage.get(getForStorage)).thenReturn(Optional.of(result)); TransactionResult recoveredResult = mock(TransactionResult.class); @SuppressWarnings("unchecked") @@ -2274,10 +2261,10 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - handler.readUnread(key, getForKey, context); + handler.readUnread(key, getForKey, context, TABLE_METADATA); // Assert - verify(storage).get(getForKey); + verify(storage).get(getForStorage); verify(recoveryExecutor) .execute( key, @@ -2298,17 +2285,17 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() when(key.getNamespace()).thenReturn(ANY_NAMESPACE_NAME); when(key.getTable()).thenReturn(ANY_TABLE_NAME); when(key.getPartitionKey()).thenReturn(Key.ofText(ANY_NAME_1, ANY_TEXT_1)); - - when(result.getInt(Attribute.STATE)).thenReturn(TransactionState.PREPARED.get()); - when(storage.get(any())).thenReturn(Optional.of(result)); - Get getForKey = Get.newBuilder() .namespace(key.getNamespace()) .table(key.getTable()) .partitionKey(key.getPartitionKey()) + .consistency(Consistency.LINEARIZABLE) .build(); + Get getForStorage = toGetForStorageFrom(getForKey); when(snapshot.containsKeyInGetSet(getForKey)).thenReturn(false); + when(result.getInt(Attribute.STATE)).thenReturn(TransactionState.PREPARED.get()); + when(storage.get(getForStorage)).thenReturn(Optional.of(result)); Optional recoveredRecord = Optional.empty(); @SuppressWarnings("unchecked") @@ -2326,10 +2313,10 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - handler.readUnread(key, getForKey, context); + handler.readUnread(key, getForKey, context, TABLE_METADATA); // Assert - verify(storage).get(getForKey); + verify(storage).get(getForStorage); verify(recoveryExecutor) .execute( key, @@ -2350,25 +2337,30 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() when(key.getNamespace()).thenReturn(ANY_NAMESPACE_NAME); when(key.getTable()).thenReturn(ANY_TABLE_NAME); when(key.getPartitionKey()).thenReturn(Key.ofText(ANY_NAME_1, ANY_TEXT_1)); - - when(result.getInt(Attribute.STATE)).thenReturn(TransactionState.PREPARED.get()); - when(storage.get(any())).thenReturn(Optional.of(result)); - Get getWithConjunction = Get.newBuilder() .namespace(key.getNamespace()) .table(key.getTable()) .partitionKey(key.getPartitionKey()) - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) + .where(column(ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .build(); + Get getForStorage = + Get.newBuilder(getWithConjunction) + .clearConditions() + .where(column(ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .consistency(Consistency.LINEARIZABLE) .build(); when(snapshot.containsKeyInGetSet(getWithConjunction)).thenReturn(false); + when(result.getInt(Attribute.STATE)).thenReturn(TransactionState.PREPARED.get()); + when(storage.get(getForStorage)).thenReturn(Optional.of(result)); TransactionResult recoveredResult = mock(TransactionResult.class); - when(recoveredResult.getContainedColumnNames()).thenReturn(Collections.singleton(ANY_NAME_3)); - when(recoveredResult.getAsObject(ANY_NAME_3)).thenReturn(ANY_TEXT_3); + when(recoveredResult.getContainedColumnNames()).thenReturn(Collections.singleton(ANY_NAME_4)); + when(recoveredResult.getAsObject(ANY_NAME_4)).thenReturn(ANY_INT_1); when(recoveredResult.getColumns()) - .thenReturn(ImmutableMap.of(ANY_NAME_3, TextColumn.of(ANY_NAME_3, ANY_TEXT_3))); + .thenReturn(ImmutableMap.of(ANY_NAME_4, IntColumn.of(ANY_NAME_4, ANY_INT_1))); @SuppressWarnings("unchecked") Future recoveryFuture = mock(Future.class); @@ -2385,16 +2377,10 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - handler.readUnread(key, getWithConjunction, context); + handler.readUnread(key, getWithConjunction, context, TABLE_METADATA); // Assert - verify(storage) - .get( - Get.newBuilder(getWithConjunction) - .clearConditions() - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .build()); + verify(storage).get(getForStorage); verify(recoveryExecutor) .execute( key, @@ -2415,25 +2401,29 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() when(key.getNamespace()).thenReturn(ANY_NAMESPACE_NAME); when(key.getTable()).thenReturn(ANY_TABLE_NAME); when(key.getPartitionKey()).thenReturn(Key.ofText(ANY_NAME_1, ANY_TEXT_1)); - - when(result.getInt(Attribute.STATE)).thenReturn(TransactionState.PREPARED.get()); - when(storage.get(any())).thenReturn(Optional.of(result)); - Get getWithConjunction = Get.newBuilder() .namespace(key.getNamespace()) .table(key.getTable()) .partitionKey(key.getPartitionKey()) - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) + .where(column(ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .build(); + Get getForStorage = + Get.newBuilder(toGetForStorageFrom(getWithConjunction)) + .clearConditions() + .where(column(ANY_NAME_4).isEqualToInt(ANY_INT_1)) + .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isEqualToInt(ANY_INT_1)) .build(); when(snapshot.containsKeyInGetSet(getWithConjunction)).thenReturn(false); + when(result.getInt(Attribute.STATE)).thenReturn(TransactionState.PREPARED.get()); + when(storage.get(getForStorage)).thenReturn(Optional.of(result)); TransactionResult recoveredResult = mock(TransactionResult.class); - when(recoveredResult.getContainedColumnNames()).thenReturn(Collections.singleton(ANY_NAME_3)); - when(recoveredResult.getAsObject(ANY_NAME_3)).thenReturn(ANY_TEXT_4); + when(recoveredResult.getContainedColumnNames()).thenReturn(Collections.singleton(ANY_NAME_4)); + when(recoveredResult.getAsObject(ANY_NAME_4)).thenReturn(ANY_INT_2); when(recoveredResult.getColumns()) - .thenReturn(ImmutableMap.of(ANY_NAME_3, TextColumn.of(ANY_NAME_3, ANY_TEXT_4))); + .thenReturn(ImmutableMap.of(ANY_NAME_4, IntColumn.of(ANY_NAME_4, ANY_INT_2))); @SuppressWarnings("unchecked") Future recoveryFuture = mock(Future.class); @@ -2450,16 +2440,10 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - handler.readUnread(key, getWithConjunction, context); + handler.readUnread(key, getWithConjunction, context, TABLE_METADATA); // Assert - verify(storage) - .get( - Get.newBuilder(getWithConjunction) - .clearConditions() - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .build()); + verify(storage).get(getForStorage); verify(recoveryExecutor) .execute( key, @@ -2482,16 +2466,17 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() .table(ANY_TABLE_NAME) .indexKey(Key.ofText(ANY_NAME_3, ANY_TEXT_1)) .build(); + Get getForStorage = toGetForStorageFrom(getWithIndex); when(snapshot.containsKeyInGetSet(getWithIndex)).thenReturn(false); - when(storage.get(any())).thenReturn(Optional.empty()); + when(storage.get(getForStorage)).thenReturn(Optional.empty()); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - handler.readUnread(null, getWithIndex, context); + handler.readUnread(null, getWithIndex, context, TABLE_METADATA); // Assert - verify(storage).get(any()); + verify(storage).get(getForStorage); verify(snapshot, never()).putIntoReadSet(any(), any()); verify(snapshot).putIntoGetSet(getWithIndex, Optional.empty()); } @@ -2507,24 +2492,24 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() ImmutableMap.of( ANY_NAME_1, TextColumn.of(ANY_NAME_1, ANY_TEXT_1), ANY_NAME_2, TextColumn.of(ANY_NAME_2, ANY_TEXT_2))); - when(storage.get(any())).thenReturn(Optional.of(result)); - Get getWithIndex = Get.newBuilder() .namespace(ANY_NAMESPACE_NAME) .table(ANY_TABLE_NAME) .indexKey(Key.ofText(ANY_NAME_3, ANY_TEXT_1)) .build(); + Get getForStorage = toGetForStorageFrom(getWithIndex); when(snapshot.containsKeyInGetSet(getWithIndex)).thenReturn(false); + when(storage.get(getForStorage)).thenReturn(Optional.of(result)); TransactionContext context = new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - handler.readUnread(null, getWithIndex, context); + handler.readUnread(null, getWithIndex, context, TABLE_METADATA); // Assert - verify(storage).get(any()); + verify(storage).get(getForStorage); verify(snapshot) .putIntoReadSet( new Snapshot.Key(getWithIndex, result, TABLE_METADATA), @@ -2543,15 +2528,15 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() ImmutableMap.of( ANY_NAME_1, TextColumn.of(ANY_NAME_1, ANY_TEXT_1), ANY_NAME_2, TextColumn.of(ANY_NAME_2, ANY_TEXT_2))); - when(storage.get(any())).thenReturn(Optional.of(result)); - Get getWithIndex = Get.newBuilder() .namespace(ANY_NAMESPACE_NAME) .table(ANY_TABLE_NAME) .indexKey(Key.ofText(ANY_NAME_3, ANY_TEXT_1)) .build(); + Get getForStorage = toGetForStorageFrom(getWithIndex); when(snapshot.containsKeyInGetSet(getWithIndex)).thenReturn(false); + when(storage.get(getForStorage)).thenReturn(Optional.of(result)); Snapshot.Key key = new Snapshot.Key(getWithIndex, result, TABLE_METADATA); @@ -2571,10 +2556,10 @@ public void readUnread_GetContainedInGetSet_ShouldCallAppropriateMethods() new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); // Act - handler.readUnread(key, getWithIndex, context); + handler.readUnread(key, getWithIndex, context, TABLE_METADATA); // Assert - verify(storage).get(getWithIndex); + verify(storage).get(getForStorage); verify(recoveryExecutor) .execute( key, @@ -2738,478 +2723,6 @@ public void readIfImplicitPreReadEnabled_ShouldCallAppropriateMethods() assertThat(transactionIdCaptor.getValue()).isEqualTo(ANY_ID_1); } - @Test - public void get_WithConjunctions_ShouldConvertConjunctions() - throws CrudException, ExecutionException { - // Arrange - when(result.getInt(Attribute.STATE)).thenReturn(TransactionState.COMMITTED.get()); - when(storage.get(any())).thenReturn(Optional.of(result)); - TransactionContext context = - new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); - - // Act - handler.get( - Get.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .clusteringKey(Key.ofText(ANY_NAME_2, ANY_TEXT_2)) - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .build(), - context); - handler.get( - Get.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .clusteringKey(Key.ofText(ANY_NAME_2, ANY_TEXT_2)) - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .and(column(ANY_NAME_4).isEqualToInt(10)) - .build(), - context); - handler.get( - Get.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .clusteringKey(Key.ofText(ANY_NAME_2, ANY_TEXT_2)) - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(ANY_NAME_4).isEqualToInt(20)) - .build(), - context); - handler.get( - Get.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .clusteringKey(Key.ofText(ANY_NAME_2, ANY_TEXT_2)) - .where( - condition(column(ANY_NAME_3).isNotEqualToText(ANY_TEXT_3)) - .and(column(ANY_NAME_3).isNotEqualToText(ANY_TEXT_4)) - .build()) - .or( - condition(column(ANY_NAME_4).isGreaterThanInt(30)) - .and(column(ANY_NAME_4).isLessThanOrEqualToInt(40)) - .build()) - .build(), - context); - handler.get( - Get.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .clusteringKey(Key.ofText(ANY_NAME_2, ANY_TEXT_2)) - .where( - condition(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(ANY_NAME_3).isEqualToText(ANY_TEXT_4)) - .build()) - .and( - condition(column(ANY_NAME_4).isLessThanOrEqualToInt(50)) - .or(column(ANY_NAME_4).isGreaterThanInt(60)) - .build()) - .build(), - context); - handler.get( - Get.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .clusteringKey(Key.ofText(ANY_NAME_2, ANY_TEXT_2)) - .where(column(ANY_NAME_3).isLikeText(ANY_TEXT_3)) - .or(column(ANY_NAME_3).isLikeText(ANY_TEXT_4)) - .build(), - context); - - // Assert - verify(storage) - .get( - Get.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .clusteringKey(Key.ofText(ANY_NAME_2, ANY_TEXT_2)) - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .consistency(Consistency.LINEARIZABLE) - .build()); - verify(storage) - .get( - Get.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .clusteringKey(Key.ofText(ANY_NAME_2, ANY_TEXT_2)) - .where( - condition(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .and(column(ANY_NAME_4).isEqualToInt(10)) - .build()) - .or( - condition( - column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .and(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isEqualToInt(10)) - .build()) - .consistency(Consistency.LINEARIZABLE) - .build()); - verify(storage) - .get( - Get.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .clusteringKey(Key.ofText(ANY_NAME_2, ANY_TEXT_2)) - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(ANY_NAME_4).isEqualToInt(20)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isEqualToInt(20)) - .consistency(Consistency.LINEARIZABLE) - .build()); - verify(storage) - .get( - Get.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .clusteringKey(Key.ofText(ANY_NAME_2, ANY_TEXT_2)) - .where( - condition(column(ANY_NAME_3).isNotEqualToText(ANY_TEXT_3)) - .and(column(ANY_NAME_3).isNotEqualToText(ANY_TEXT_4)) - .build()) - .or( - condition(column(ANY_NAME_4).isGreaterThanInt(30)) - .and(column(ANY_NAME_4).isLessThanOrEqualToInt(40)) - .build()) - .or( - condition( - column(Attribute.BEFORE_PREFIX + ANY_NAME_3) - .isNotEqualToText(ANY_TEXT_3)) - .and( - column(Attribute.BEFORE_PREFIX + ANY_NAME_3) - .isNotEqualToText(ANY_TEXT_4)) - .build()) - .or( - condition(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isGreaterThanInt(30)) - .and( - column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isLessThanOrEqualToInt(40)) - .build()) - .consistency(Consistency.LINEARIZABLE) - .build()); - verify(storage) - .get( - Get.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .clusteringKey(Key.ofText(ANY_NAME_2, ANY_TEXT_2)) - .where( - condition(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .and(column(ANY_NAME_4).isLessThanOrEqualToInt(50)) - .build()) - .or( - condition(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .and(column(ANY_NAME_4).isGreaterThanInt(60)) - .build()) - .or( - condition(column(ANY_NAME_3).isEqualToText(ANY_TEXT_4)) - .and(column(ANY_NAME_4).isLessThanOrEqualToInt(50)) - .build()) - .or( - condition(column(ANY_NAME_3).isEqualToText(ANY_TEXT_4)) - .and(column(ANY_NAME_4).isGreaterThanInt(60)) - .build()) - .or( - condition( - column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .and( - column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isLessThanOrEqualToInt(50)) - .build()) - .or( - condition( - column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .and(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isGreaterThanInt(60)) - .build()) - .or( - condition( - column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_4)) - .and( - column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isLessThanOrEqualToInt(50)) - .build()) - .or( - condition( - column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_4)) - .and(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isGreaterThanInt(60)) - .build()) - .consistency(Consistency.LINEARIZABLE) - .build()); - verify(storage) - .get( - Get.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .clusteringKey(Key.ofText(ANY_NAME_2, ANY_TEXT_2)) - .where(column(ANY_NAME_3).isLikeText(ANY_TEXT_3)) - .or(column(ANY_NAME_3).isLikeText(ANY_TEXT_4)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isLikeText(ANY_TEXT_3)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isLikeText(ANY_TEXT_4)) - .consistency(Consistency.LINEARIZABLE) - .build()); - } - - @Test - public void scan_WithConjunctions_ShouldConvertConjunctions() - throws CrudException, ExecutionException { - // Arrange - when(result.getInt(Attribute.STATE)).thenReturn(TransactionState.COMMITTED.get()); - when(result.getColumns()) - .thenReturn( - ImmutableMap.of( - ANY_NAME_1, TextColumn.of(ANY_NAME_1, ANY_TEXT_1), - ANY_NAME_2, TextColumn.of(ANY_NAME_2, ANY_TEXT_2))); - when(scanner.iterator()).thenReturn(Collections.singletonList(result).iterator()); - when(storage.scan(any())).thenReturn(scanner); - TransactionContext context = - new TransactionContext(ANY_ID_1, snapshot, Isolation.SNAPSHOT, false, false); - - // Act - handler.scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .build(), - context); - handler.scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .and(column(ANY_NAME_4).isEqualToInt(10)) - .build(), - context); - handler.scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(ANY_NAME_4).isEqualToInt(20)) - .build(), - context); - handler.scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .where( - condition(column(ANY_NAME_3).isNotEqualToText(ANY_TEXT_3)) - .and(column(ANY_NAME_3).isNotEqualToText(ANY_TEXT_4)) - .build()) - .or( - condition(column(ANY_NAME_4).isGreaterThanInt(30)) - .and(column(ANY_NAME_4).isLessThanOrEqualToInt(40)) - .build()) - .build(), - context); - handler.scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .where( - condition(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(ANY_NAME_3).isEqualToText(ANY_TEXT_4)) - .build()) - .and( - condition(column(ANY_NAME_4).isLessThanOrEqualToInt(50)) - .or(column(ANY_NAME_4).isGreaterThanInt(60)) - .build()) - .build(), - context); - handler.scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .where(column(ANY_NAME_3).isLikeText(ANY_TEXT_3)) - .or(column(ANY_NAME_3).isLikeText(ANY_TEXT_4)) - .build(), - context); - handler.scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .all() - .where(column(ANY_NAME_1).isGreaterThanText(ANY_TEXT_3)) - .and(column(ANY_NAME_2).isLessThanOrEqualToText(ANY_TEXT_4)) - .build(), - context); - handler.scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .all() - .where(column(ANY_NAME_1).isGreaterThanText(ANY_TEXT_3)) - .and(column(ANY_NAME_3).isEqualToText(ANY_TEXT_4)) - .build(), - context); - - // Assert - verify(storage) - .scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .consistency(Consistency.LINEARIZABLE) - .build()); - verify(storage) - .scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .where( - condition(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .and(column(ANY_NAME_4).isEqualToInt(10)) - .build()) - .or( - condition( - column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .and(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isEqualToInt(10)) - .build()) - .consistency(Consistency.LINEARIZABLE) - .build()); - verify(storage) - .scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .where(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(ANY_NAME_4).isEqualToInt(20)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isEqualToInt(20)) - .consistency(Consistency.LINEARIZABLE) - .build()); - verify(storage) - .scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .where( - condition(column(ANY_NAME_3).isNotEqualToText(ANY_TEXT_3)) - .and(column(ANY_NAME_3).isNotEqualToText(ANY_TEXT_4)) - .build()) - .or( - condition(column(ANY_NAME_4).isGreaterThanInt(30)) - .and(column(ANY_NAME_4).isLessThanOrEqualToInt(40)) - .build()) - .or( - condition( - column(Attribute.BEFORE_PREFIX + ANY_NAME_3) - .isNotEqualToText(ANY_TEXT_3)) - .and( - column(Attribute.BEFORE_PREFIX + ANY_NAME_3) - .isNotEqualToText(ANY_TEXT_4)) - .build()) - .or( - condition(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isGreaterThanInt(30)) - .and( - column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isLessThanOrEqualToInt(40)) - .build()) - .consistency(Consistency.LINEARIZABLE) - .build()); - verify(storage) - .scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .where( - condition(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .and(column(ANY_NAME_4).isLessThanOrEqualToInt(50)) - .build()) - .or( - condition(column(ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .and(column(ANY_NAME_4).isGreaterThanInt(60)) - .build()) - .or( - condition(column(ANY_NAME_3).isEqualToText(ANY_TEXT_4)) - .and(column(ANY_NAME_4).isLessThanOrEqualToInt(50)) - .build()) - .or( - condition(column(ANY_NAME_3).isEqualToText(ANY_TEXT_4)) - .and(column(ANY_NAME_4).isGreaterThanInt(60)) - .build()) - .or( - condition( - column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .and( - column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isLessThanOrEqualToInt(50)) - .build()) - .or( - condition( - column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_3)) - .and(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isGreaterThanInt(60)) - .build()) - .or( - condition( - column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_4)) - .and( - column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isLessThanOrEqualToInt(50)) - .build()) - .or( - condition( - column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_4)) - .and(column(Attribute.BEFORE_PREFIX + ANY_NAME_4).isGreaterThanInt(60)) - .build()) - .consistency(Consistency.LINEARIZABLE) - .build()); - verify(storage) - .scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .where(column(ANY_NAME_3).isLikeText(ANY_TEXT_3)) - .or(column(ANY_NAME_3).isLikeText(ANY_TEXT_4)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isLikeText(ANY_TEXT_3)) - .or(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isLikeText(ANY_TEXT_4)) - .consistency(Consistency.LINEARIZABLE) - .build()); - verify(storage) - .scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .all() - .where(column(ANY_NAME_1).isGreaterThanText(ANY_TEXT_3)) - .and(column(ANY_NAME_2).isLessThanOrEqualToText(ANY_TEXT_4)) - .consistency(Consistency.LINEARIZABLE) - .build()); - verify(storage) - .scan( - Scan.newBuilder() - .namespace(ANY_NAMESPACE_NAME) - .table(ANY_TABLE_NAME) - .all() - .where( - condition(column(ANY_NAME_1).isGreaterThanText(ANY_TEXT_3)) - .and(column(ANY_NAME_3).isEqualToText(ANY_TEXT_4)) - .build()) - .or( - condition(column(ANY_NAME_1).isGreaterThanText(ANY_TEXT_3)) - .and(column(Attribute.BEFORE_PREFIX + ANY_NAME_3).isEqualToText(ANY_TEXT_4)) - .build()) - .consistency(Consistency.LINEARIZABLE) - .build()); - } - private List scanOrGetScanner(Scan scan, ScanType scanType, TransactionContext context) throws CrudException { if (scanType == ScanType.SCAN) { diff --git a/core/src/test/java/com/scalar/db/transaction/consensuscommit/SnapshotTest.java b/core/src/test/java/com/scalar/db/transaction/consensuscommit/SnapshotTest.java index 0970977ce9..3fc1b2c7c3 100644 --- a/core/src/test/java/com/scalar/db/transaction/consensuscommit/SnapshotTest.java +++ b/core/src/test/java/com/scalar/db/transaction/consensuscommit/SnapshotTest.java @@ -174,21 +174,25 @@ private TransactionResult prepareResultWithNullMetadata() { } private Get prepareGet() { - Key partitionKey = new Key(ANY_NAME_1, ANY_TEXT_1); - Key clusteringKey = new Key(ANY_NAME_2, ANY_TEXT_2); - return new Get(partitionKey, clusteringKey) - .withConsistency(Consistency.LINEARIZABLE) - .forNamespace(ANY_NAMESPACE_NAME) - .forTable(ANY_TABLE_NAME); + Key partitionKey = Key.ofText(ANY_NAME_1, ANY_TEXT_1); + Key clusteringKey = Key.ofText(ANY_NAME_2, ANY_TEXT_2); + return Get.newBuilder() + .namespace(ANY_NAMESPACE_NAME) + .table(ANY_TABLE_NAME) + .partitionKey(partitionKey) + .clusteringKey(clusteringKey) + .build(); } private Get prepareAnotherGet() { - Key partitionKey = new Key(ANY_NAME_5, ANY_TEXT_5); - Key clusteringKey = new Key(ANY_NAME_6, ANY_TEXT_6); - return new Get(partitionKey, clusteringKey) - .withConsistency(Consistency.LINEARIZABLE) - .forNamespace(ANY_NAMESPACE_NAME) - .forTable(ANY_TABLE_NAME); + Key partitionKey = Key.ofText(ANY_NAME_5, ANY_TEXT_5); + Key clusteringKey = Key.ofText(ANY_NAME_6, ANY_TEXT_6); + return Get.newBuilder() + .namespace(ANY_NAMESPACE_NAME) + .table(ANY_TABLE_NAME) + .partitionKey(partitionKey) + .clusteringKey(clusteringKey) + .build(); } private Get prepareGetWithIndex() { @@ -201,13 +205,14 @@ private Get prepareGetWithIndex() { } private Scan prepareScan() { - Key partitionKey = new Key(ANY_NAME_1, ANY_TEXT_1); - Key clusteringKey = new Key(ANY_NAME_2, ANY_TEXT_2); - return new Scan(partitionKey) - .withStart(clusteringKey) - .withConsistency(Consistency.LINEARIZABLE) - .forNamespace(ANY_NAMESPACE_NAME) - .forTable(ANY_TABLE_NAME); + Key partitionKey = Key.ofText(ANY_NAME_1, ANY_TEXT_1); + Key clusteringKey = Key.ofText(ANY_NAME_2, ANY_TEXT_2); + return Scan.newBuilder() + .namespace(ANY_NAMESPACE_NAME) + .table(ANY_TABLE_NAME) + .partitionKey(partitionKey) + .start(clusteringKey) + .build(); } private Scan prepareScanWithLimit(int limit) { @@ -216,7 +221,6 @@ private Scan prepareScanWithLimit(int limit) { .table(ANY_TABLE_NAME) .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) .limit(limit) - .consistency(Consistency.LINEARIZABLE) .build(); } @@ -254,27 +258,18 @@ private Put preparePut(String partitionKeyColumnValue, String clusteringKeyColum .clusteringKey(Key.ofText(ANY_NAME_2, clusteringKeyColumnValue)) .textValue(ANY_NAME_3, ANY_TEXT_3) .textValue(ANY_NAME_4, ANY_TEXT_4) - .consistency(Consistency.LINEARIZABLE) .build(); } - private Put prepareAnotherPut() { - Key partitionKey = new Key(ANY_NAME_5, ANY_TEXT_5); - Key clusteringKey = new Key(ANY_NAME_6, ANY_TEXT_6); - return new Put(partitionKey, clusteringKey) - .withConsistency(Consistency.LINEARIZABLE) - .forNamespace(ANY_NAMESPACE_NAME) - .forTable(ANY_TABLE_NAME); - } - private Put preparePutWithPartitionKeyOnly() { - Key partitionKey = new Key(ANY_NAME_1, ANY_TEXT_1); - return new Put(partitionKey) - .withConsistency(Consistency.LINEARIZABLE) - .forNamespace(ANY_NAMESPACE_NAME) - .forTable(ANY_TABLE_NAME) - .withValue(ANY_NAME_3, ANY_TEXT_3) - .withValue(ANY_NAME_4, ANY_TEXT_4); + Key partitionKey = Key.ofText(ANY_NAME_1, ANY_TEXT_1); + return Put.newBuilder() + .namespace(ANY_NAMESPACE_NAME) + .table(ANY_TABLE_NAME) + .partitionKey(partitionKey) + .textValue(ANY_NAME_3, ANY_TEXT_3) + .textValue(ANY_NAME_4, ANY_TEXT_4) + .build(); } private Put preparePutWithIntColumns() { @@ -289,7 +284,6 @@ private Put preparePutWithIntColumns() { .value(IntColumn.of(ANY_NAME_6, ANY_INT_1)) .value(IntColumn.of(ANY_NAME_7, ANY_INT_1)) .value(IntColumn.ofNull(ANY_NAME_8)) - .consistency(Consistency.LINEARIZABLE) .build(); } @@ -314,17 +308,18 @@ private Delete prepareDelete(String partitionKeyColumnValue, String clusteringKe .table(ANY_TABLE_NAME) .partitionKey(Key.ofText(ANY_NAME_1, partitionKeyColumnValue)) .clusteringKey(Key.ofText(ANY_NAME_2, clusteringKeyColumnValue)) - .consistency(Consistency.LINEARIZABLE) .build(); } private Delete prepareAnotherDelete() { - Key partitionKey = new Key(ANY_NAME_5, ANY_TEXT_5); - Key clusteringKey = new Key(ANY_NAME_6, ANY_TEXT_6); - return new Delete(partitionKey, clusteringKey) - .withConsistency(Consistency.LINEARIZABLE) - .forNamespace(ANY_NAMESPACE_NAME) - .forTable(ANY_TABLE_NAME); + Key partitionKey = Key.ofText(ANY_NAME_5, ANY_TEXT_5); + Key clusteringKey = Key.ofText(ANY_NAME_6, ANY_TEXT_6); + return Delete.newBuilder() + .namespace(ANY_NAMESPACE_NAME) + .table(ANY_TABLE_NAME) + .partitionKey(partitionKey) + .clusteringKey(clusteringKey) + .build(); } private void configureBehavior() throws ExecutionException { @@ -988,14 +983,15 @@ public void toSerializable_ReadSetNotChanged_ShouldProcessWithoutExceptions() snapshot.putIntoGetSet(get, Optional.of(txResult)); snapshot.putIntoWriteSet(new Snapshot.Key(put), put); DistributedStorage storage = mock(DistributedStorage.class); - Get getWithProjections = prepareAnotherGet().withProjection(Attribute.ID); - when(storage.get(getWithProjections)).thenReturn(Optional.of(txResult)); + Get getForStorage = + Get.newBuilder(prepareAnotherGet()).consistency(Consistency.LINEARIZABLE).build(); + when(storage.get(getForStorage)).thenReturn(Optional.of(txResult)); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); // Assert - verify(storage).get(getWithProjections); + verify(storage).get(getForStorage); } @Test @@ -1010,15 +1006,16 @@ public void toSerializable_ReadSetUpdated_ShouldThrowValidationConflictException snapshot.putIntoWriteSet(new Snapshot.Key(put), put); DistributedStorage storage = mock(DistributedStorage.class); TransactionResult changedTxResult = prepareResult(ANY_ID + "x"); - Get getWithProjections = prepareAnotherGet().withProjection(Attribute.ID); - when(storage.get(getWithProjections)).thenReturn(Optional.of(changedTxResult)); + Get getForStorage = + Get.newBuilder(prepareAnotherGet()).consistency(Consistency.LINEARIZABLE).build(); + when(storage.get(getForStorage)).thenReturn(Optional.of(changedTxResult)); // Act Assert assertThatThrownBy(() -> snapshot.toSerializable(storage)) .isInstanceOf(ValidationConflictException.class); // Assert - verify(storage).get(getWithProjections); + verify(storage).get(getForStorage); } @Test @@ -1032,15 +1029,16 @@ public void toSerializable_ReadSetExtended_ShouldThrowValidationConflictExceptio snapshot.putIntoWriteSet(new Snapshot.Key(put), put); DistributedStorage storage = mock(DistributedStorage.class); TransactionResult txResult = prepareResult(ANY_ID); - Get getWithProjections = prepareAnotherGet().withProjection(Attribute.ID); - when(storage.get(getWithProjections)).thenReturn(Optional.of(txResult)); + Get getForStorage = + Get.newBuilder(prepareAnotherGet()).consistency(Consistency.LINEARIZABLE).build(); + when(storage.get(getForStorage)).thenReturn(Optional.of(txResult)); // Act Assert assertThatThrownBy(() -> snapshot.toSerializable(storage)) .isInstanceOf(ValidationConflictException.class); // Assert - verify(storage).get(getWithProjections); + verify(storage).get(getForStorage); } @Test @@ -1052,17 +1050,18 @@ public void toSerializable_GetSetWithGetWithIndex_ShouldProcessWithoutExceptions TransactionResult txResult = prepareResult(ANY_ID + "x"); snapshot.putIntoGetSet(getWithIndex, Optional.of(txResult)); DistributedStorage storage = mock(DistributedStorage.class); - Scan scanWithIndex = - prepareScanWithIndex().withProjections(Arrays.asList(Attribute.ID, ANY_NAME_1, ANY_NAME_2)); + Scan scanForStorage = + Scan.newBuilder(prepareScanWithIndex()).consistency(Consistency.LINEARIZABLE).build(); + Scanner scanner = mock(Scanner.class); when(scanner.one()).thenReturn(Optional.of(txResult)).thenReturn(Optional.empty()); - when(storage.scan(scanWithIndex)).thenReturn(scanner); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); // Assert - verify(storage).scan(scanWithIndex); + verify(storage).scan(scanForStorage); } @Test @@ -1076,21 +1075,22 @@ public void toSerializable_GetSetWithGetWithIndex_ShouldProcessWithoutExceptions TransactionResult result2 = prepareResult(ANY_ID + "xx", ANY_TEXT_1, ANY_TEXT_3); snapshot.putIntoGetSet(getWithIndex, Optional.of(result1)); DistributedStorage storage = mock(DistributedStorage.class); - Scan scanWithIndex = - prepareScanWithIndex().withProjections(Arrays.asList(Attribute.ID, ANY_NAME_1, ANY_NAME_2)); + Scan scanForStorage = + Scan.newBuilder(prepareScanWithIndex()).consistency(Consistency.LINEARIZABLE).build(); + Scanner scanner = mock(Scanner.class); when(scanner.one()) .thenReturn(Optional.of(result1)) .thenReturn(Optional.of(result2)) .thenReturn(Optional.empty()); - when(storage.scan(scanWithIndex)).thenReturn(scanner); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatThrownBy(() -> snapshot.toSerializable(storage)) .isInstanceOf(ValidationConflictException.class); // Assert - verify(storage).scan(scanWithIndex); + verify(storage).scan(scanForStorage); } @Test @@ -1104,20 +1104,21 @@ public void toSerializable_GetSetWithGetWithIndex_ShouldProcessWithoutExceptions TransactionResult result2 = prepareResult(ANY_ID, ANY_TEXT_1, ANY_TEXT_3); snapshot.putIntoGetSet(getWithIndex, Optional.of(result1)); DistributedStorage storage = mock(DistributedStorage.class); - Scan scanWithIndex = - prepareScanWithIndex().withProjections(Arrays.asList(Attribute.ID, ANY_NAME_1, ANY_NAME_2)); + Scan scanForStorage = + Scan.newBuilder(prepareScanWithIndex()).consistency(Consistency.LINEARIZABLE).build(); + Scanner scanner = mock(Scanner.class); when(scanner.one()) .thenReturn(Optional.of(result1)) .thenReturn(Optional.of(result2)) .thenReturn(Optional.empty()); - when(storage.scan(scanWithIndex)).thenReturn(scanner); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); // Assert - verify(storage).scan(scanWithIndex); + verify(storage).scan(scanForStorage); } @Test @@ -1132,15 +1133,15 @@ public void toSerializable_ScanSetNotChanged_ShouldProcessWithoutExceptions() DistributedStorage storage = mock(DistributedStorage.class); Scanner scanner = mock(Scanner.class); when(scanner.one()).thenReturn(Optional.of(txResult)).thenReturn(Optional.empty()); - Scan scanWithProjections = - prepareScan().withProjections(Arrays.asList(Attribute.ID, ANY_NAME_1, ANY_NAME_2)); - when(storage.scan(scanWithProjections)).thenReturn(scanner); + Scan scanForStorage = + Scan.newBuilder(prepareScan()).consistency(Consistency.LINEARIZABLE).build(); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); // Assert - verify(storage).scan(scanWithProjections); + verify(storage).scan(scanForStorage); } @Test @@ -1156,16 +1157,16 @@ public void toSerializable_ScanSetUpdated_ShouldThrowValidationConflictException TransactionResult changedTxResult = prepareResult(ANY_ID + "x"); Scanner scanner = mock(Scanner.class); when(scanner.one()).thenReturn(Optional.of(changedTxResult)).thenReturn(Optional.empty()); - Scan scanWithProjections = - prepareScan().withProjections(Arrays.asList(Attribute.ID, ANY_NAME_1, ANY_NAME_2)); - when(storage.scan(scanWithProjections)).thenReturn(scanner); + Scan scanForStorage = + Scan.newBuilder(prepareScan()).consistency(Consistency.LINEARIZABLE).build(); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatThrownBy(() -> snapshot.toSerializable(storage)) .isInstanceOf(ValidationConflictException.class); // Assert - verify(storage).scan(scanWithProjections); + verify(storage).scan(scanForStorage); } @Test @@ -1181,15 +1182,15 @@ public void toSerializable_ScanSetUpdatedByMyself_ShouldProcessWithoutExceptions TransactionResult changedTxResult = prepareResult(ANY_ID); Scanner scanner = mock(Scanner.class); when(scanner.one()).thenReturn(Optional.of(changedTxResult)).thenReturn(Optional.empty()); - Scan scanWithProjections = - prepareScan().withProjections(Arrays.asList(Attribute.ID, ANY_NAME_1, ANY_NAME_2)); - when(storage.scan(scanWithProjections)).thenReturn(scanner); + Scan scanForStorage = + Scan.newBuilder(prepareScan()).consistency(Consistency.LINEARIZABLE).build(); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); // Assert - verify(storage).scan(scanWithProjections); + verify(storage).scan(scanForStorage); } @Test @@ -1204,16 +1205,16 @@ public void toSerializable_ScanSetExtended_ShouldThrowValidationConflictExceptio TransactionResult txResult = new TransactionResult(result); Scanner scanner = mock(Scanner.class); when(scanner.one()).thenReturn(Optional.of(txResult)).thenReturn(Optional.empty()); - Scan scanWithProjections = - prepareScan().withProjections(Arrays.asList(Attribute.ID, ANY_NAME_1, ANY_NAME_2)); - when(storage.scan(scanWithProjections)).thenReturn(scanner); + Scan scanForStorage = + Scan.newBuilder(prepareScan()).consistency(Consistency.LINEARIZABLE).build(); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatThrownBy(() -> snapshot.toSerializable(storage)) .isInstanceOf(ValidationConflictException.class); // Assert - verify(storage).scan(scanWithProjections); + verify(storage).scan(scanForStorage); } @Test @@ -1233,16 +1234,16 @@ public void toSerializable_ScanSetExtended_ShouldThrowValidationConflictExceptio .thenReturn(Optional.of(result1)) .thenReturn(Optional.of(result2)) .thenReturn(Optional.empty()); - Scan scanWithProjections = - prepareScan().withProjections(Arrays.asList(Attribute.ID, ANY_NAME_1, ANY_NAME_2)); - when(storage.scan(scanWithProjections)).thenReturn(scanner); + Scan scanForStorage = + Scan.newBuilder(prepareScan()).consistency(Consistency.LINEARIZABLE).build(); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatThrownBy(() -> snapshot.toSerializable(storage)) .isInstanceOf(ValidationConflictException.class); // Assert - verify(storage).scan(scanWithProjections); + verify(storage).scan(scanForStorage); } @Test @@ -1257,15 +1258,15 @@ public void toSerializable_ScanSetExtendedByMyself_ShouldProcessWithoutException TransactionResult txResult = new TransactionResult(result); Scanner scanner = mock(Scanner.class); when(scanner.one()).thenReturn(Optional.of(txResult)).thenReturn(Optional.empty()); - Scan scanWithProjections = - prepareScan().withProjections(Arrays.asList(Attribute.ID, ANY_NAME_1, ANY_NAME_2)); - when(storage.scan(scanWithProjections)).thenReturn(scanner); + Scan scanForStorage = + Scan.newBuilder(prepareScan()).consistency(Consistency.LINEARIZABLE).build(); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); // Assert - verify(storage).scan(scanWithProjections); + verify(storage).scan(scanForStorage); } @Test @@ -1285,15 +1286,15 @@ public void toSerializable_ScanSetExtendedByMyself_ShouldProcessWithoutException .thenReturn(Optional.of(result1)) .thenReturn(Optional.of(result2)) .thenReturn(Optional.empty()); - Scan scanWithProjections = - prepareScan().withProjections(Arrays.asList(Attribute.ID, ANY_NAME_1, ANY_NAME_2)); - when(storage.scan(scanWithProjections)).thenReturn(scanner); + Scan scanForStorage = + Scan.newBuilder(prepareScan()).consistency(Consistency.LINEARIZABLE).build(); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); // Assert - verify(storage).scan(scanWithProjections); + verify(storage).scan(scanForStorage); } @Test @@ -1308,16 +1309,16 @@ public void toSerializable_ScanSetDeleted_ShouldThrowValidationConflictException DistributedStorage storage = mock(DistributedStorage.class); Scanner scanner = mock(Scanner.class); when(scanner.one()).thenReturn(Optional.empty()); - Scan scanWithProjections = - prepareScan().withProjections(Arrays.asList(Attribute.ID, ANY_NAME_1, ANY_NAME_2)); - when(storage.scan(scanWithProjections)).thenReturn(scanner); + Scan scanForStorage = + Scan.newBuilder(prepareScan()).consistency(Consistency.LINEARIZABLE).build(); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatThrownBy(() -> snapshot.toSerializable(storage)) .isInstanceOf(ValidationConflictException.class); // Assert - verify(storage).scan(scanWithProjections); + verify(storage).scan(scanForStorage); } @Test @@ -1337,16 +1338,16 @@ public void toSerializable_ScanSetDeleted_ShouldThrowValidationConflictException DistributedStorage storage = mock(DistributedStorage.class); Scanner scanner = mock(Scanner.class); when(scanner.one()).thenReturn(Optional.of(result2)).thenReturn(Optional.empty()); - Scan scanWithProjections = - prepareScan().withProjections(Arrays.asList(Attribute.ID, ANY_NAME_1, ANY_NAME_2)); - when(storage.scan(scanWithProjections)).thenReturn(scanner); + Scan scanForStorage = + Scan.newBuilder(prepareScan()).consistency(Consistency.LINEARIZABLE).build(); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatThrownBy(() -> snapshot.toSerializable(storage)) .isInstanceOf(ValidationConflictException.class); // Assert - verify(storage).scan(scanWithProjections); + verify(storage).scan(scanForStorage); } @Test @@ -1356,17 +1357,19 @@ public void toSerializable_MultipleScansInScanSetExist_ShouldProcessWithoutExcep snapshot = prepareSnapshot(Isolation.SERIALIZABLE); Scan scan1 = - new Scan(new Key(ANY_NAME_1, ANY_TEXT_1)) - .withStart(new Key(ANY_NAME_2, ANY_TEXT_2)) - .withConsistency(Consistency.LINEARIZABLE) - .forNamespace(ANY_NAMESPACE_NAME) - .forTable(ANY_TABLE_NAME); + Scan.newBuilder() + .namespace(ANY_NAMESPACE_NAME) + .table(ANY_TABLE_NAME) + .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) + .start(Key.ofText(ANY_NAME_2, ANY_TEXT_2)) + .build(); Scan scan2 = - new Scan(new Key(ANY_NAME_1, ANY_TEXT_2)) - .withStart(new Key(ANY_NAME_2, ANY_TEXT_1)) - .withConsistency(Consistency.LINEARIZABLE) - .forNamespace(ANY_NAMESPACE_NAME) - .forTable(ANY_TABLE_NAME); + Scan.newBuilder() + .namespace(ANY_NAMESPACE_NAME) + .table(ANY_TABLE_NAME) + .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_2)) + .start(Key.ofText(ANY_NAME_2, ANY_TEXT_1)) + .build(); Result result1 = new TransactionResult( @@ -1406,25 +1409,27 @@ public void toSerializable_MultipleScansInScanSetExist_ShouldProcessWithoutExcep Scanner scanner1 = mock(Scanner.class); when(scanner1.one()).thenReturn(Optional.of(result1)).thenReturn(Optional.empty()); - Scan scan1WithProjections = - new Scan(new Key(ANY_NAME_1, ANY_TEXT_1)) - .withStart(new Key(ANY_NAME_2, ANY_TEXT_2)) - .withConsistency(Consistency.LINEARIZABLE) - .forNamespace(ANY_NAMESPACE_NAME) - .forTable(ANY_TABLE_NAME) - .withProjections(Arrays.asList(Attribute.ID, ANY_NAME_1, ANY_NAME_2)); - when(storage.scan(scan1WithProjections)).thenReturn(scanner1); + Scan scan1ForStorage = + Scan.newBuilder() + .namespace(ANY_NAMESPACE_NAME) + .table(ANY_TABLE_NAME) + .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) + .start(Key.ofText(ANY_NAME_2, ANY_TEXT_2)) + .consistency(Consistency.LINEARIZABLE) + .build(); + when(storage.scan(scan1ForStorage)).thenReturn(scanner1); Scanner scanner2 = mock(Scanner.class); when(scanner2.one()).thenReturn(Optional.of(result2)).thenReturn(Optional.empty()); - Scan scan2WithProjections = - new Scan(new Key(ANY_NAME_1, ANY_TEXT_2)) - .withStart(new Key(ANY_NAME_2, ANY_TEXT_1)) - .withConsistency(Consistency.LINEARIZABLE) - .forNamespace(ANY_NAMESPACE_NAME) - .forTable(ANY_TABLE_NAME) - .withProjections(Arrays.asList(Attribute.ID, ANY_NAME_1, ANY_NAME_2)); - when(storage.scan(scan2WithProjections)).thenReturn(scanner2); + Scan scan2ForStorage = + Scan.newBuilder() + .namespace(ANY_NAMESPACE_NAME) + .table(ANY_TABLE_NAME) + .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_2)) + .start(Key.ofText(ANY_NAME_2, ANY_TEXT_1)) + .consistency(Consistency.LINEARIZABLE) + .build(); + when(storage.scan(scan2ForStorage)).thenReturn(scanner2); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); @@ -1442,14 +1447,14 @@ public void toSerializable_NullMetadataInReadSetNotChanged_ShouldProcessWithoutE snapshot.putIntoGetSet(get, Optional.of(result)); snapshot.putIntoWriteSet(new Snapshot.Key(put), put); DistributedStorage storage = mock(DistributedStorage.class); - Get getWithProjections = Get.newBuilder(get).projection(Attribute.ID).build(); - when(storage.get(getWithProjections)).thenReturn(Optional.of(txResult)); + Get getForStorage = Get.newBuilder(get).consistency(Consistency.LINEARIZABLE).build(); + when(storage.get(getForStorage)).thenReturn(Optional.of(txResult)); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); // Assert - verify(storage).get(getWithProjections); + verify(storage).get(getForStorage); } @Test @@ -1464,15 +1469,15 @@ public void toSerializable_NullMetadataInReadSetChanged_ShouldThrowValidationCon snapshot.putIntoGetSet(get, Optional.of(result)); snapshot.putIntoWriteSet(new Snapshot.Key(put), put); DistributedStorage storage = mock(DistributedStorage.class); - Get getWithProjections = Get.newBuilder(get).projection(Attribute.ID).build(); - when(storage.get(getWithProjections)).thenReturn(Optional.of(changedResult)); + Get getForStorage = Get.newBuilder(get).consistency(Consistency.LINEARIZABLE).build(); + when(storage.get(getForStorage)).thenReturn(Optional.of(changedResult)); // Act Assert assertThatThrownBy(() -> snapshot.toSerializable(storage)) .isInstanceOf(ValidationConflictException.class); // Assert - verify(storage).get(getWithProjections); + verify(storage).get(getForStorage); } @Test @@ -1486,20 +1491,20 @@ public void toSerializable_ScanWithLimitInScanSet_ShouldProcessWithoutExceptions Snapshot.Key key1 = new Snapshot.Key(scan, result1, TABLE_METADATA); snapshot.putIntoScanSet(scan, Maps.newLinkedHashMap(Collections.singletonMap(key1, result1))); DistributedStorage storage = mock(DistributedStorage.class); - Scan scanWithProjectionsWithoutLimit = - Scan.newBuilder(scan).projections(Attribute.ID, ANY_NAME_1, ANY_NAME_2).limit(0).build(); + Scan scanForStorage = + Scan.newBuilder(scan).limit(0).consistency(Consistency.LINEARIZABLE).build(); Scanner scanner = mock(Scanner.class); when(scanner.one()) .thenReturn(Optional.of(result1)) .thenReturn(Optional.of(result2)) .thenReturn(Optional.empty()); - when(storage.scan(scanWithProjectionsWithoutLimit)).thenReturn(scanner); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); // Assert - verify(storage).scan(scanWithProjectionsWithoutLimit); + verify(storage).scan(scanForStorage); } @Test @@ -1515,22 +1520,22 @@ public void toSerializable_ScanWithLimitInScanSet_ShouldProcessWithoutExceptions Snapshot.Key key1 = new Snapshot.Key(scan, result1, TABLE_METADATA); snapshot.putIntoScanSet(scan, Maps.newLinkedHashMap(ImmutableMap.of(key1, result1))); DistributedStorage storage = mock(DistributedStorage.class); - Scan scanWithProjectionsWithoutLimit = - Scan.newBuilder(scan).projections(Attribute.ID, ANY_NAME_1, ANY_NAME_2).limit(0).build(); + Scan scanForStorage = + Scan.newBuilder(scan).limit(0).consistency(Consistency.LINEARIZABLE).build(); Scanner scanner = mock(Scanner.class); when(scanner.one()) .thenReturn(Optional.of(insertedResult)) .thenReturn(Optional.of(result1)) .thenReturn(Optional.of(result2)) .thenReturn(Optional.empty()); - when(storage.scan(scanWithProjectionsWithoutLimit)).thenReturn(scanner); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatThrownBy(() -> snapshot.toSerializable(storage)) .isInstanceOf(ValidationConflictException.class); // Assert - verify(storage).scan(scanWithProjectionsWithoutLimit); + verify(storage).scan(scanForStorage); } @Test @@ -1546,21 +1551,21 @@ public void toSerializable_ScanWithLimitInScanSet_ShouldProcessWithoutExceptions Snapshot.Key key1 = new Snapshot.Key(scan, result1, TABLE_METADATA); snapshot.putIntoScanSet(scan, Maps.newLinkedHashMap(ImmutableMap.of(key1, result1))); DistributedStorage storage = mock(DistributedStorage.class); - Scan scanWithProjectionsWithoutLimit = - Scan.newBuilder(scan).projections(Attribute.ID, ANY_NAME_1, ANY_NAME_2).limit(0).build(); + Scan scanForStorage = + Scan.newBuilder(scan).limit(0).consistency(Consistency.LINEARIZABLE).build(); Scanner scanner = mock(Scanner.class); when(scanner.one()) .thenReturn(Optional.of(insertedResult)) .thenReturn(Optional.of(result1)) .thenReturn(Optional.of(result2)) .thenReturn(Optional.empty()); - when(storage.scan(scanWithProjectionsWithoutLimit)).thenReturn(scanner); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); // Assert - verify(storage).scan(scanWithProjectionsWithoutLimit); + verify(storage).scan(scanForStorage); } @Test @@ -1578,22 +1583,22 @@ public void toSerializable_ScanWithLimitInScanSet_ShouldProcessWithoutExceptions snapshot.putIntoScanSet( scan, Maps.newLinkedHashMap(ImmutableMap.of(key1, result1, key2, result2))); DistributedStorage storage = mock(DistributedStorage.class); - Scan scanWithProjectionsWithoutLimit = - Scan.newBuilder(scan).projections(Attribute.ID, ANY_NAME_1, ANY_NAME_2).limit(0).build(); + Scan scanForStorage = + Scan.newBuilder(scan).limit(0).consistency(Consistency.LINEARIZABLE).build(); Scanner scanner = mock(Scanner.class); when(scanner.one()) .thenReturn(Optional.of(result1)) .thenReturn(Optional.of(result2)) .thenReturn(Optional.of(insertedResult)) .thenReturn(Optional.empty()); - when(storage.scan(scanWithProjectionsWithoutLimit)).thenReturn(scanner); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatThrownBy(() -> snapshot.toSerializable(storage)) .isInstanceOf(ValidationConflictException.class); // Assert - verify(storage).scan(scanWithProjectionsWithoutLimit); + verify(storage).scan(scanForStorage); } @Test @@ -1611,21 +1616,22 @@ public void toSerializable_ScanWithLimitInScanSet_ShouldProcessWithoutExceptions snapshot.putIntoScanSet( scan, Maps.newLinkedHashMap(ImmutableMap.of(key1, result1, key2, result2))); DistributedStorage storage = mock(DistributedStorage.class); - Scan scanWithProjectionsWithoutLimit = - Scan.newBuilder(scan).projections(Attribute.ID, ANY_NAME_1, ANY_NAME_2).limit(0).build(); + Scan scanForStorage = + Scan.newBuilder(scan).limit(0).consistency(Consistency.LINEARIZABLE).build(); + Scanner scanner = mock(Scanner.class); when(scanner.one()) .thenReturn(Optional.of(result1)) .thenReturn(Optional.of(result2)) .thenReturn(Optional.of(insertedResult)) .thenReturn(Optional.empty()); - when(storage.scan(scanWithProjectionsWithoutLimit)).thenReturn(scanner); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); // Assert - verify(storage).scan(scanWithProjectionsWithoutLimit); + verify(storage).scan(scanForStorage); } @Test @@ -1649,16 +1655,17 @@ public void toSerializable_ScanWithLimitInScanSet_ShouldProcessWithoutExceptions when(scanner.one()).thenReturn(Optional.of(result2)).thenReturn(Optional.empty()); DistributedStorage storage = mock(DistributedStorage.class); - Scan scanWithProjections = - Scan.newBuilder(scan).projections(Attribute.ID, ANY_NAME_1, ANY_NAME_2).limit(0).build(); - when(storage.scan(scanWithProjections)).thenReturn(scanner); + Scan scanForStorage = + Scan.newBuilder(scan).limit(0).consistency(Consistency.LINEARIZABLE).build(); + + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatThrownBy(() -> snapshot.toSerializable(storage)) .isInstanceOf(ValidationConflictException.class); // Assert - verify(storage).scan(scanWithProjections); + verify(storage).scan(scanForStorage); } @Test @@ -1684,15 +1691,16 @@ public void toSerializable_ScanWithLimitInScanSet_ShouldProcessWithoutExceptions when(scanner.one()).thenReturn(Optional.of(result2)).thenReturn(Optional.empty()); DistributedStorage storage = mock(DistributedStorage.class); - Scan scanWithProjections = - Scan.newBuilder(scan).projections(Attribute.ID, ANY_NAME_1, ANY_NAME_2).limit(0).build(); - when(storage.scan(scanWithProjections)).thenReturn(scanner); + Scan scanForStorage = + Scan.newBuilder(scan).limit(0).consistency(Consistency.LINEARIZABLE).build(); + + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); // Assert - verify(storage).scan(scanWithProjections); + verify(storage).scan(scanForStorage); } @Test @@ -1718,15 +1726,15 @@ public void toSerializable_ScanWithLimitInScanSet_ShouldProcessWithoutExceptions when(scanner.one()).thenReturn(Optional.of(result2)).thenReturn(Optional.empty()); DistributedStorage storage = mock(DistributedStorage.class); - Scan scanWithProjections = - Scan.newBuilder(scan).projections(Attribute.ID, ANY_NAME_1, ANY_NAME_2).limit(0).build(); - when(storage.scan(scanWithProjections)).thenReturn(scanner); + Scan scanForStorage = + Scan.newBuilder(scan).limit(0).consistency(Consistency.LINEARIZABLE).build(); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); // Assert - verify(storage).scan(scanWithProjections); + verify(storage).scan(scanForStorage); } @Test @@ -1740,20 +1748,19 @@ public void toSerializable_ScannerSetNotChanged_ShouldProcessWithoutExceptions() Snapshot.Key key1 = new Snapshot.Key(scan, result1, TABLE_METADATA); snapshot.putIntoScannerSet(scan, Maps.newLinkedHashMap(ImmutableMap.of(key1, result1))); DistributedStorage storage = mock(DistributedStorage.class); - Scan scanWithProjections = - Scan.newBuilder(scan).projections(Attribute.ID, ANY_NAME_1, ANY_NAME_2).build(); + Scan scanForStorage = Scan.newBuilder(scan).consistency(Consistency.LINEARIZABLE).build(); Scanner scanner = mock(Scanner.class); when(scanner.one()) .thenReturn(Optional.of(result1)) .thenReturn(Optional.of(result2)) .thenReturn(Optional.empty()); - when(storage.scan(scanWithProjections)).thenReturn(scanner); + when(storage.scan(scanForStorage)).thenReturn(scanner); // Act Assert assertThatCode(() -> snapshot.toSerializable(storage)).doesNotThrowAnyException(); // Assert - verify(storage).scan(scanWithProjections); + verify(storage).scan(scanForStorage); } @Test @@ -1849,7 +1856,6 @@ public void toSerializable_ScannerSetNotChanged_ShouldProcessWithoutExceptions() .namespace(ANY_NAMESPACE_NAME) .table(ANY_TABLE_NAME) .partitionKey(Key.ofText(ANY_NAME_1, ANY_TEXT_1)) - .consistency(Consistency.LINEARIZABLE) .where(ConditionBuilder.column(ANY_NAME_3).isEqualToText(ANY_TEXT_4)) .build(); @@ -2153,11 +2159,8 @@ public void verifyNoOverlap_ScanAllGivenAndPutInWriteSetInSameTable_ShouldThrowE Put put = preparePut(); Snapshot.Key putKey = new Snapshot.Key(put); snapshot.putIntoWriteSet(putKey, put); - ScanAll scanAll = - new ScanAll() - .withConsistency(Consistency.LINEARIZABLE) - .forNamespace(ANY_NAMESPACE_NAME) - .forTable(ANY_TABLE_NAME); + Scan scanAll = + ScanAll.newBuilder().namespace(ANY_NAMESPACE_NAME).table(ANY_TABLE_NAME).all().build(); TransactionResult result = prepareResult(ANY_ID); Snapshot.Key key = new Snapshot.Key(scanAll, result, TABLE_METADATA); @@ -2179,11 +2182,8 @@ public void verifyNoOverlap_ScanAllGivenAndPutInWriteSetInSameTable_ShouldThrowE Put put = preparePut(); Snapshot.Key putKey = new Snapshot.Key(put); snapshot.putIntoWriteSet(putKey, put); - ScanAll scanAll = - new ScanAll() - .withConsistency(Consistency.LINEARIZABLE) - .forNamespace(ANY_NAMESPACE_NAME_2) - .forTable(ANY_TABLE_NAME_2); + Scan scanAll = + ScanAll.newBuilder().namespace(ANY_NAMESPACE_NAME_2).table(ANY_TABLE_NAME_2).all().build(); TransactionResult result = prepareResult(ANY_ID); Snapshot.Key key = new Snapshot.Key(scanAll, result, TABLE_METADATA); diff --git a/integration-test/src/main/java/com/scalar/db/transaction/consensuscommit/ConsensusCommitSpecificIntegrationTestBase.java b/integration-test/src/main/java/com/scalar/db/transaction/consensuscommit/ConsensusCommitSpecificIntegrationTestBase.java index 01565763bf..bc1cb1daae 100644 --- a/integration-test/src/main/java/com/scalar/db/transaction/consensuscommit/ConsensusCommitSpecificIntegrationTestBase.java +++ b/integration-test/src/main/java/com/scalar/db/transaction/consensuscommit/ConsensusCommitSpecificIntegrationTestBase.java @@ -6948,7 +6948,7 @@ void scanAndUpdate_ScanWithIndexGiven_ShouldUpdate(Isolation isolation) // Act Assert DistributedTransaction transaction = begin(manager, readOnly); Optional actual = - manager.get( + transaction.get( Get.newBuilder() .namespace(namespace1) .table(TABLE_1)