From 39b7dfb8c88cf7a99c3f91e8a97cf46c9ed216ca Mon Sep 17 00:00:00 2001 From: nicktorwald <nicktorwald@gmail.com> Date: Wed, 19 Jun 2019 02:58:58 +0700 Subject: [PATCH] Soft automatic schema reload Now client keeps actual schema metadata and sends schemaId header to be checked against current Tarantool schema version. If client version mismatches DB version client does schema reloading in the background. Client operation interface was reworked in scope of support not only number identifiers for spaces and indexes but also their string names. This also includes set of request builders that can be used as a public API to construct requests. The main idea here is to provide more natural DSL-like approach to build operations instead of current abstract types like List<?> or List<Object>. Closes: #7, #137 Affects: #212 --- README.md | 44 ++ .../org/tarantool/AbstractTarantoolOps.java | 150 ++++- src/main/java/org/tarantool/Iterator.java | 10 + src/main/java/org/tarantool/MsgPackLite.java | 7 + .../java/org/tarantool/RequestArguments.java | 79 +++ .../java/org/tarantool/TarantoolBase.java | 13 +- .../java/org/tarantool/TarantoolClient.java | 12 +- .../org/tarantool/TarantoolClientImpl.java | 529 +++++++++++------- .../org/tarantool/TarantoolClientOps.java | 33 +- .../org/tarantool/TarantoolClusterClient.java | 76 +-- .../org/tarantool/TarantoolConnection.java | 13 +- .../org/tarantool/TarantoolException.java | 12 +- src/main/java/org/tarantool/TarantoolOp.java | 57 ++ .../java/org/tarantool/TarantoolRequest.java | 174 ++++++ .../tarantool/TarantoolRequestArgument.java | 23 + ...antoolClusterStoredFunctionDiscoverer.java | 18 +- .../tarantool/dsl/AbstractRequestSpec.java | 46 ++ .../org/tarantool/dsl/CallRequestSpec.java | 63 +++ .../org/tarantool/dsl/DeleteRequestSpec.java | 66 +++ .../org/tarantool/dsl/EvalRequestSpec.java | 54 ++ .../org/tarantool/dsl/ExecuteRequestSpec.java | 91 +++ .../dsl/InsertOrReplaceRequestSpec.java | 81 +++ .../java/org/tarantool/dsl/Operation.java | 65 +++ .../java/org/tarantool/dsl/Operations.java | 48 ++ src/main/java/org/tarantool/dsl/Operator.java | 33 ++ .../org/tarantool/dsl/PingRequestSpec.java | 11 + src/main/java/org/tarantool/dsl/Requests.java | 109 ++++ .../org/tarantool/dsl/SelectRequestSpec.java | 111 ++++ .../org/tarantool/dsl/SpaceRequestSpec.java | 46 ++ .../dsl/TarantoolRequestConvertible.java | 21 + .../org/tarantool/dsl/UpdateRequestSpec.java | 73 +++ .../org/tarantool/dsl/UpsertRequestSpec.java | 88 +++ .../org/tarantool/jdbc/SQLConnection.java | 24 +- .../tarantool/protocol/ProtoConstants.java | 19 + .../org/tarantool/protocol/ProtoUtils.java | 18 +- .../tarantool/protocol/TarantoolPacket.java | 15 +- .../tarantool/schema/TarantoolIndexMeta.java | 128 +++++ .../TarantoolIndexNotFoundException.java | 16 + .../schema/TarantoolMetaSpacesCache.java | 116 ++++ .../schema/TarantoolSchemaException.java | 15 + .../tarantool/schema/TarantoolSchemaMeta.java | 41 ++ .../tarantool/schema/TarantoolSpaceMeta.java | 99 ++++ .../TarantoolSpaceNotFoundException.java | 9 + .../java/org/tarantool/util/TupleTwo.java | 4 +- .../tarantool/ClientAsyncOperationsIT.java | 281 +++++++++- .../org/tarantool/ClientOperationsIT.java | 183 ++++++ .../tarantool/ClientReconnectClusterIT.java | 28 +- .../java/org/tarantool/ClientReconnectIT.java | 10 +- .../FireAndForgetClientOperationsIT.java | 40 +- src/test/java/org/tarantool/IteratorTest.java | 35 -- .../org/tarantool/TarantoolClientOpsIT.java | 24 +- src/test/java/org/tarantool/TestUtils.java | 2 +- ...sterServiceStoredFunctionDiscovererIT.java | 3 +- .../jdbc/JdbcConnectionTimeoutIT.java | 5 +- .../jdbc/JdbcExceptionHandlingTest.java | 6 +- .../schema/ClientReconnectSchemaIT.java | 91 +++ .../org/tarantool/schema/ClientSchemaIT.java | 247 ++++++++ .../schema/ClientThreadSafeSchemaIT.java | 105 ++++ 58 files changed, 3396 insertions(+), 424 deletions(-) create mode 100644 src/main/java/org/tarantool/RequestArguments.java create mode 100644 src/main/java/org/tarantool/TarantoolOp.java create mode 100644 src/main/java/org/tarantool/TarantoolRequest.java create mode 100644 src/main/java/org/tarantool/TarantoolRequestArgument.java create mode 100644 src/main/java/org/tarantool/dsl/AbstractRequestSpec.java create mode 100644 src/main/java/org/tarantool/dsl/CallRequestSpec.java create mode 100644 src/main/java/org/tarantool/dsl/DeleteRequestSpec.java create mode 100644 src/main/java/org/tarantool/dsl/EvalRequestSpec.java create mode 100644 src/main/java/org/tarantool/dsl/ExecuteRequestSpec.java create mode 100644 src/main/java/org/tarantool/dsl/InsertOrReplaceRequestSpec.java create mode 100644 src/main/java/org/tarantool/dsl/Operation.java create mode 100644 src/main/java/org/tarantool/dsl/Operations.java create mode 100644 src/main/java/org/tarantool/dsl/Operator.java create mode 100644 src/main/java/org/tarantool/dsl/PingRequestSpec.java create mode 100644 src/main/java/org/tarantool/dsl/Requests.java create mode 100644 src/main/java/org/tarantool/dsl/SelectRequestSpec.java create mode 100644 src/main/java/org/tarantool/dsl/SpaceRequestSpec.java create mode 100644 src/main/java/org/tarantool/dsl/TarantoolRequestConvertible.java create mode 100644 src/main/java/org/tarantool/dsl/UpdateRequestSpec.java create mode 100644 src/main/java/org/tarantool/dsl/UpsertRequestSpec.java create mode 100644 src/main/java/org/tarantool/protocol/ProtoConstants.java create mode 100644 src/main/java/org/tarantool/schema/TarantoolIndexMeta.java create mode 100644 src/main/java/org/tarantool/schema/TarantoolIndexNotFoundException.java create mode 100644 src/main/java/org/tarantool/schema/TarantoolMetaSpacesCache.java create mode 100644 src/main/java/org/tarantool/schema/TarantoolSchemaException.java create mode 100644 src/main/java/org/tarantool/schema/TarantoolSchemaMeta.java create mode 100644 src/main/java/org/tarantool/schema/TarantoolSpaceMeta.java create mode 100644 src/main/java/org/tarantool/schema/TarantoolSpaceNotFoundException.java delete mode 100644 src/test/java/org/tarantool/IteratorTest.java create mode 100644 src/test/java/org/tarantool/schema/ClientReconnectSchemaIT.java create mode 100644 src/test/java/org/tarantool/schema/ClientSchemaIT.java create mode 100644 src/test/java/org/tarantool/schema/ClientThreadSafeSchemaIT.java diff --git a/README.md b/README.md index d05922a2..ec5eeb4e 100644 --- a/README.md +++ b/README.md @@ -130,6 +130,50 @@ all the results, you could override this: protected void complete(TarantoolPacket packet, TarantoolOp<?> future); ``` +## String space/index resolution + +Each operation that requires space or index to be executed, can work with +number ID as well as string name of a space or an index. +Assume, we have `my_space` space with space ID `512` and its primary index +`primary` with index ID `0`. Then, for instance, `select` operations can be +performed as the following: + +```java +client.syncOps().select(512, 0, Collections.singletonList(1), 0, 1, Iterator.EQ); +// or using more convenient way +client.syncOps().select("my_space", "primary", Collections.singletonList(1), 0, 1, Iterator.EQ); +``` + +Because _iproto_ has not yet supported string spaces and indexes, a client caches current server +schema in memory. The client relies on protocol SCHEMA_ID and sends each request with respect to +cached schema version. The schema is used primarily to resolve string names of spaces or indexes +against its integer IDs. + +### Schema update + +1. Just after a (re-)connection to the Tarantool instance. + The client cannot guarantee that new instance is the same and has same schema, + thus, the client drops the cached schema and fetches new one; +2. receiving a schema version error as a response to our request. + It's possible some request can be rejected by server because of schema + mismatching between client and server. In this case the schema will be + reloaded and the refused request will be resent using the updated schema + version; +3. sending a DDL request and receiving a new version in a response; +4. sending a request against a non-existent space/index name. + The client cannot exactly know whether name was not found because of + it does not exist or it has not the latest schema version. + +### Schema support caveats + +1. Each schema reloading requires at least two extra requests to fetch spaces and + indexes metadata respectively. There is also a ping request followed by reloading + of the schema to check whether the client has outdated version (see point 4 in + Schema update section). +2. In some circumstance, requests can be rejected several times until both client's + and server's versions matches. It may take significant amount of time or even be + a cause of request timeout. + ## Spring NamedParameterJdbcTemplate usage example The JDBC driver uses `TarantoolClient` implementation to provide a communication with server. diff --git a/src/main/java/org/tarantool/AbstractTarantoolOps.java b/src/main/java/org/tarantool/AbstractTarantoolOps.java index 8cefb379..6195447a 100644 --- a/src/main/java/org/tarantool/AbstractTarantoolOps.java +++ b/src/main/java/org/tarantool/AbstractTarantoolOps.java @@ -1,62 +1,158 @@ package org.tarantool; +import static org.tarantool.dsl.Requests.callRequest; +import static org.tarantool.dsl.Requests.deleteRequest; +import static org.tarantool.dsl.Requests.evalRequest; +import static org.tarantool.dsl.Requests.insertRequest; +import static org.tarantool.dsl.Requests.pingRequest; +import static org.tarantool.dsl.Requests.replaceRequest; +import static org.tarantool.dsl.Requests.selectRequest; +import static org.tarantool.dsl.Requests.updateRequest; +import static org.tarantool.dsl.Requests.upsertRequest; -public abstract class AbstractTarantoolOps<Space, Tuple, Operation, Result> - implements TarantoolClientOps<Space, Tuple, Operation, Result> { +import org.tarantool.dsl.Operation; +import org.tarantool.dsl.TarantoolRequestConvertible; +import org.tarantool.schema.TarantoolSchemaMeta; + +import java.util.Arrays; +import java.util.List; + +public abstract class AbstractTarantoolOps<Result> + implements TarantoolClientOps<List<?>, Object, Result> { private Code callCode = Code.CALL; - protected abstract Result exec(Code code, Object... args); + protected abstract Result exec(TarantoolRequest request); + + protected abstract TarantoolSchemaMeta getSchemaMeta(); + + public Result select(Integer space, Integer index, List<?> key, int offset, int limit, Iterator iterator) { + return execute( + selectRequest(space, index) + .key(key) + .offset(offset).limit(limit) + .iterator(iterator) + ); + } + + @Override + public Result select(String space, String index, List<?> key, int offset, int limit, Iterator iterator) { + return execute( + selectRequest(space, index) + .key(key) + .offset(offset).limit(limit) + .iterator(iterator) + ); + } - public Result select(Space space, Space index, Tuple key, int offset, int limit, Iterator iterator) { - return select(space, index, key, offset, limit, iterator.getValue()); + @Override + public Result select(Integer space, Integer index, List<?> key, int offset, int limit, int iterator) { + return execute( + selectRequest(space, index) + .key(key) + .offset(offset).limit(limit) + .iterator(iterator) + ); } - public Result select(Space space, Space index, Tuple key, int offset, int limit, int iterator) { - return exec( - Code.SELECT, - Key.SPACE, space, - Key.INDEX, index, - Key.KEY, key, - Key.ITERATOR, iterator, - Key.LIMIT, limit, - Key.OFFSET, offset + @Override + public Result select(String space, String index, List<?> key, int offset, int limit, int iterator) { + return execute( + selectRequest(space, index) + .key(key) + .offset(offset).limit(limit) + .iterator(iterator) ); } - public Result insert(Space space, Tuple tuple) { - return exec(Code.INSERT, Key.SPACE, space, Key.TUPLE, tuple); + @Override + public Result insert(Integer space, List<?> tuple) { + return execute(insertRequest(space, tuple)); + } + + @Override + public Result insert(String space, List<?> tuple) { + return execute(insertRequest(space, tuple)); } - public Result replace(Space space, Tuple tuple) { - return exec(Code.REPLACE, Key.SPACE, space, Key.TUPLE, tuple); + @Override + public Result replace(Integer space, List<?> tuple) { + return execute(replaceRequest(space, tuple)); } - public Result update(Space space, Tuple key, Operation... args) { - return exec(Code.UPDATE, Key.SPACE, space, Key.KEY, key, Key.TUPLE, args); + @Override + public Result replace(String space, List<?> tuple) { + return execute(replaceRequest(space, tuple)); } - public Result upsert(Space space, Tuple key, Tuple def, Operation... args) { - return exec(Code.UPSERT, Key.SPACE, space, Key.KEY, key, Key.TUPLE, def, Key.UPSERT_OPS, args); + @Override + public Result update(Integer space, List<?> key, Object... operations) { + Operation[] ops = Arrays.stream(operations) + .map(Operation::fromArray) + .toArray(org.tarantool.dsl.Operation[]::new); + return execute(updateRequest(space, key, ops)); } - public Result delete(Space space, Tuple key) { - return exec(Code.DELETE, Key.SPACE, space, Key.KEY, key); + @Override + public Result update(String space, List<?> key, Object... operations) { + Operation[] ops = Arrays.stream(operations) + .map(Operation::fromArray) + .toArray(org.tarantool.dsl.Operation[]::new); + return execute(updateRequest(space, key, ops)); } + @Override + public Result upsert(Integer space, List<?> key, List<?> defTuple, Object... operations) { + Operation[] ops = Arrays.stream(operations) + .map(Operation::fromArray) + .toArray(Operation[]::new); + return execute(upsertRequest(space, key, defTuple, ops)); + } + + @Override + public Result upsert(String space, List<?> key, List<?> defTuple, Object... operations) { + Operation[] ops = Arrays.stream(operations) + .map(Operation::fromArray) + .toArray(Operation[]::new); + return execute(upsertRequest(space, key, defTuple, ops)); + } + + @Override + public Result delete(Integer space, List<?> key) { + return execute(deleteRequest(space, key)); + } + + @Override + public Result delete(String space, List<?> key) { + return execute(deleteRequest(space, key)); + } + + @Override public Result call(String function, Object... args) { - return exec(callCode, Key.FUNCTION, function, Key.TUPLE, args); + return execute( + callRequest(function) + .arguments(args) + .useCall16(callCode == Code.OLD_CALL) + ); } + @Override public Result eval(String expression, Object... args) { - return exec(Code.EVAL, Key.EXPRESSION, expression, Key.TUPLE, args); + return execute(evalRequest(expression).arguments(args)); } + @Override public void ping() { - exec(Code.PING); + execute(pingRequest()); + } + + @Override + public Result execute(TarantoolRequestConvertible requestSpec) { + return exec(requestSpec.toTarantoolRequest(getSchemaMeta())); } public void setCallCode(Code callCode) { this.callCode = callCode; } + } diff --git a/src/main/java/org/tarantool/Iterator.java b/src/main/java/org/tarantool/Iterator.java index 4452a744..c013530a 100644 --- a/src/main/java/org/tarantool/Iterator.java +++ b/src/main/java/org/tarantool/Iterator.java @@ -1,5 +1,7 @@ package org.tarantool; +import java.util.Arrays; + // Iterator info was taken from here https://github.com/tarantool/tarantool/blob/f66584c3bcdffe61d6d99a4868a9b72d62338a11/src/box/iterator_type.h#L62 public enum Iterator { EQ(0), // key == x ASC order @@ -24,4 +26,12 @@ public enum Iterator { public int getValue() { return value; } + + public static Iterator valueOf(int value) { + return Arrays.stream(Iterator.values()) + .filter(v -> value == v.getValue()) + .findFirst() + .orElseThrow(IllegalArgumentException::new); + } + } diff --git a/src/main/java/org/tarantool/MsgPackLite.java b/src/main/java/org/tarantool/MsgPackLite.java index f16f66fa..d48aabfb 100644 --- a/src/main/java/org/tarantool/MsgPackLite.java +++ b/src/main/java/org/tarantool/MsgPackLite.java @@ -1,5 +1,7 @@ package org.tarantool; +import org.tarantool.util.TupleTwo; + import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; @@ -226,6 +228,11 @@ public void pack(Object item, OutputStream os) throws IOException { pack(kvp.getKey(), out); pack(kvp.getValue(), out); } + } else if (item instanceof TupleTwo) { + TupleTwo<?, ?> tuple = (TupleTwo<?, ?>) item; + out.write(1 | MP_FIXMAP); + pack(tuple.getFirst(), out); + pack(tuple.getSecond(), out); } else { throw new IllegalArgumentException("Cannot msgpack object of type " + item.getClass().getCanonicalName()); } diff --git a/src/main/java/org/tarantool/RequestArguments.java b/src/main/java/org/tarantool/RequestArguments.java new file mode 100644 index 00000000..fdf8bc2a --- /dev/null +++ b/src/main/java/org/tarantool/RequestArguments.java @@ -0,0 +1,79 @@ +package org.tarantool; + +import java.util.Objects; +import java.util.function.Supplier; + +/** + * Request argument factory. + * + * @see TarantoolRequestArgument + */ +public class RequestArguments { + + private RequestArguments() { + } + + public static TarantoolRequestArgument value(Object value) { + return new SimpleArgument(value); + } + + public static TarantoolRequestArgument cacheLookupValue(Supplier<Object> supplier) { + return new LookupArgument(supplier); + } + + /** + * Simple wrapper that holds the original value. + */ + private static class SimpleArgument implements TarantoolRequestArgument { + + private Object value; + + SimpleArgument(Object value) { + Objects.requireNonNull(value); + this.value = value; + } + + @Override + public boolean isSerializable() { + return true; + } + + @Override + public Object getValue() { + return value; + } + + } + + /** + * Wrapper that evaluates the value each time + * it is requested. + * <p> + * It works like a function, where {@code argument = f(key)}. + */ + private static class LookupArgument implements TarantoolRequestArgument { + + Supplier<Object> lookup; + + LookupArgument(Supplier<Object> lookup) { + this.lookup = Objects.requireNonNull(lookup); + } + + @Override + public boolean isSerializable() { + try { + lookup.get(); + } catch (Exception ignored) { + return false; + } + return true; + } + + @Override + public synchronized Object getValue() { + return lookup.get(); + } + + } + +} diff --git a/src/main/java/org/tarantool/TarantoolBase.java b/src/main/java/org/tarantool/TarantoolBase.java index c74647ae..26656112 100644 --- a/src/main/java/org/tarantool/TarantoolBase.java +++ b/src/main/java/org/tarantool/TarantoolBase.java @@ -6,10 +6,9 @@ import java.io.IOException; import java.net.Socket; import java.nio.channels.SocketChannel; -import java.util.List; import java.util.concurrent.atomic.AtomicLong; -public abstract class TarantoolBase<Result> extends AbstractTarantoolOps<Integer, List<?>, Object, Result> { +public abstract class TarantoolBase<Result> extends AbstractTarantoolOps<Result> { protected String serverVersion; protected MsgPackLite msgPackLite = MsgPackLite.INSTANCE; protected AtomicLong syncId = new AtomicLong(); @@ -42,16 +41,6 @@ protected void closeChannel(SocketChannel channel) { } } - protected void validateArgs(Object[] args) { - if (args != null) { - for (int i = 0; i < args.length; i += 2) { - if (args[i + 1] == null) { - throw new NullPointerException(((Key) args[i]).name() + " should not be null"); - } - } - } - } - public void setInitialRequestSize(int initialRequestSize) { this.initialRequestSize = initialRequestSize; } diff --git a/src/main/java/org/tarantool/TarantoolClient.java b/src/main/java/org/tarantool/TarantoolClient.java index 2ad0c84c..70167383 100644 --- a/src/main/java/org/tarantool/TarantoolClient.java +++ b/src/main/java/org/tarantool/TarantoolClient.java @@ -1,5 +1,7 @@ package org.tarantool; +import org.tarantool.schema.TarantoolSchemaMeta; + import java.util.List; import java.util.Map; import java.util.concurrent.CompletionStage; @@ -7,13 +9,13 @@ import java.util.concurrent.TimeUnit; public interface TarantoolClient { - TarantoolClientOps<Integer, List<?>, Object, List<?>> syncOps(); + TarantoolClientOps<List<?>, Object, List<?>> syncOps(); - TarantoolClientOps<Integer, List<?>, Object, Future<List<?>>> asyncOps(); + TarantoolClientOps<List<?>, Object, Future<List<?>>> asyncOps(); - TarantoolClientOps<Integer, List<?>, Object, CompletionStage<List<?>>> composableAsyncOps(); + TarantoolClientOps<List<?>, Object, CompletionStage<List<?>>> composableAsyncOps(); - TarantoolClientOps<Integer, List<?>, Object, Long> fireAndForgetOps(); + TarantoolClientOps<List<?>, Object, Long> fireAndForgetOps(); TarantoolSQLOps<Object, Long, List<Map<String, Object>>> sqlSyncOps(); @@ -29,4 +31,6 @@ public interface TarantoolClient { boolean waitAlive(long timeout, TimeUnit unit) throws InterruptedException; + TarantoolSchemaMeta getSchemaMeta(); + } diff --git a/src/main/java/org/tarantool/TarantoolClientImpl.java b/src/main/java/org/tarantool/TarantoolClientImpl.java index dcb7a8ba..b5848041 100644 --- a/src/main/java/org/tarantool/TarantoolClientImpl.java +++ b/src/main/java/org/tarantool/TarantoolClientImpl.java @@ -1,30 +1,41 @@ package org.tarantool; +import static org.tarantool.dsl.Requests.executeRequest; +import static org.tarantool.dsl.Requests.pingRequest; + +import org.tarantool.protocol.ProtoConstants; import org.tarantool.protocol.ProtoUtils; import org.tarantool.protocol.ReadableViaSelectorChannel; import org.tarantool.protocol.TarantoolGreeting; import org.tarantool.protocol.TarantoolPacket; +import org.tarantool.schema.TarantoolMetaSpacesCache; +import org.tarantool.schema.TarantoolSchemaMeta; +import org.tarantool.util.TupleTwo; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.SocketChannel; +import java.time.Duration; +import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.concurrent.CompletableFuture; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletionStage; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; import java.util.concurrent.Future; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.PriorityBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.locks.StampedLock; public class TarantoolClientImpl extends TarantoolBase<Future<?>> implements TarantoolClient { @@ -32,7 +43,7 @@ public class TarantoolClientImpl extends TarantoolBase<Future<?>> implements Tar = new CommunicationException("Not connected, initializing connection"); protected TarantoolClientConfig config; - protected long operationTimeout; + protected Duration operationTimeout; /** * External. @@ -43,7 +54,12 @@ public class TarantoolClientImpl extends TarantoolBase<Future<?>> implements Tar protected volatile Exception thumbstone; - protected Map<Long, TarantoolOp<?>> futures; + protected ScheduledExecutorService workExecutor; + + protected StampedLock schemaLock = new StampedLock(); + protected BlockingQueue<TarantoolRequest> delayedOpsQueue; + + protected Map<Long, TarantoolRequest> futures; protected AtomicInteger pendingResponsesCount = new AtomicInteger(); /** @@ -63,6 +79,7 @@ public class TarantoolClientImpl extends TarantoolBase<Future<?>> implements Tar protected SyncOps syncOps; protected FireAndForgetOps fireAndForgetOps; protected ComposableAsyncOps composableAsyncOps; + protected UnsafeSchemaOps unsafeSchemaOps; /** * Inner. @@ -72,6 +89,9 @@ public class TarantoolClientImpl extends TarantoolBase<Future<?>> implements Tar protected Thread reader; protected Thread writer; + protected TarantoolSchemaMeta schemaMeta = new TarantoolMetaSpacesCache(this); + protected volatile Long lastRecheduledOperationId = null; + protected Thread connector = new Thread(new Runnable() { @Override public void run() { @@ -104,10 +124,13 @@ private void initClient(SocketChannelProvider socketProvider, TarantoolClientCon this.thumbstone = NOT_INIT_EXCEPTION; this.config = config; this.initialRequestSize = config.defaultRequestSize; - this.operationTimeout = config.operationExpiryTimeMillis; + this.operationTimeout = Duration.ofMillis(config.operationExpiryTimeMillis); this.socketProvider = socketProvider; this.stats = new TarantoolClientStats(); this.futures = new ConcurrentHashMap<>(config.predictedFutures); + this.delayedOpsQueue = new PriorityBlockingQueue<>(128); + this.workExecutor = + Executors.newSingleThreadScheduledExecutor(new TarantoolThreadDaemonFactory("tarantool-worker")); this.sharedBuffer = ByteBuffer.allocateDirect(config.sharedBufferSize); this.writerBuffer = ByteBuffer.allocateDirect(sharedBuffer.capacity()); this.connector.setDaemon(true); @@ -115,6 +138,7 @@ private void initClient(SocketChannelProvider socketProvider, TarantoolClientCon this.syncOps = new SyncOps(); this.composableAsyncOps = new ComposableAsyncOps(); this.fireAndForgetOps = new FireAndForgetOps(); + this.unsafeSchemaOps = new UnsafeSchemaOps(); if (!config.useNewCall) { setCallCode(Code.OLD_CALL); this.syncOps.setCallCode(Code.OLD_CALL); @@ -192,6 +216,7 @@ protected void connect(final SocketChannel channel) throws Exception { } this.thumbstone = null; startThreads(channel.socket().getRemoteSocketAddress().toString()); + updateSchema(); } protected void startThreads(String threadName) throws InterruptedException { @@ -203,7 +228,7 @@ protected void startThreads(String threadName) throws InterruptedException { try { readThread(); } finally { - state.release(StateHelper.READING); + state.release(StateHelper.READING | StateHelper.SCHEMA_UPDATING); // only last of two IO-threads can signal for reconnection if (leftIoThreads.decrementAndGet() == 0) { state.trySignalForReconnection(); @@ -217,7 +242,7 @@ protected void startThreads(String threadName) throws InterruptedException { try { writeThread(); } finally { - state.release(StateHelper.WRITING); + state.release(StateHelper.WRITING | StateHelper.SCHEMA_UPDATING); // only last of two IO-threads can signal for reconnection if (leftIoThreads.decrementAndGet() == 0) { state.trySignalForReconnection(); @@ -240,60 +265,67 @@ protected void configureThreads(String threadName) { reader.setPriority(config.readerThreadPriority); } + @Override + public TarantoolSchemaMeta getSchemaMeta() { + return schemaMeta; + } + /** * Executes an operation with default timeout. * - * @param code operation code - * @param args operation arguments + * @param request operation data * * @return deferred result * * @see #setOperationTimeout(long) */ - protected Future<?> exec(Code code, Object... args) { - return doExec(operationTimeout, code, args); + @Override + protected Future<?> exec(TarantoolRequest request) { + return doExec(request).getResult(); } - /** - * Executes an operation with the given timeout. - * {@code timeoutMillis} will override the default - * timeout. 0 means the limitless operation. - * - * @param code operation code - * @param args operation arguments - * - * @return deferred result - */ - protected Future<?> exec(long timeoutMillis, Code code, Object... args) { - return doExec(timeoutMillis, code, args); + protected TarantoolRequest doExec(TarantoolRequest request) { + long stamp = schemaLock.readLock(); + try { + // space or index names could not be found in cache + // it's possible the client keeps the outdated schema + request.begin(syncId.incrementAndGet(), operationTimeout); + if (!request.isSerializable()) { + delayedOpsQueue.add(request); + // the schema is already reloading, skip sync-op + if (!state.isStateSet(StateHelper.SCHEMA_UPDATING)) { + optimisticSchemaUpdate(request); + } + return request; + } + // postpone operation if the schema is reloading + if (state.isStateSet(StateHelper.SCHEMA_UPDATING)) { + delayedOpsQueue.add(request); + return request; + } + return registerOperation(request, schemaMeta.getSchemaVersion()); + } finally { + schemaLock.unlockRead(stamp); + } } - protected TarantoolOp<?> doExec(long timeoutMillis, Code code, Object[] args) { - validateArgs(args); - long sid = syncId.incrementAndGet(); - - TarantoolOp<?> future = makeNewOperation(timeoutMillis, sid, code, args); - - if (isDead(future)) { - return future; + protected TarantoolRequest registerOperation(TarantoolRequest request, long schemaId) { + if (isDead(request)) { + return request; } - futures.put(sid, future); - if (isDead(future)) { - futures.remove(sid); - return future; + futures.put(request.getId(), request); + if (isDead(request)) { + futures.remove(request.getId()); + return request; } try { - write(code, sid, null, args); + request.setStartedSchemaId(schemaId); + write(request.getCode(), request.getId(), schemaId, request.getArguments().toArray()); } catch (Exception e) { - futures.remove(sid); - fail(future, e); + futures.remove(request.getId()); + fail(request, e); } - return future; - } - - protected TarantoolOp<?> makeNewOperation(long timeoutMillis, long sid, Code code, Object[] args) { - return new TarantoolOp<>(sid, code, args) - .orTimeout(timeoutMillis, TimeUnit.MILLISECONDS); + return request; } protected synchronized void die(String message, Exception cause) { @@ -303,16 +335,22 @@ protected synchronized void die(String message, Exception cause) { final CommunicationException error = new CommunicationException(message, cause); this.thumbstone = error; while (!futures.isEmpty()) { - Iterator<Map.Entry<Long, TarantoolOp<?>>> iterator = futures.entrySet().iterator(); + Iterator<Map.Entry<Long, TarantoolRequest>> iterator = futures.entrySet().iterator(); while (iterator.hasNext()) { - Map.Entry<Long, TarantoolOp<?>> elem = iterator.next(); + Map.Entry<Long, TarantoolRequest> elem = iterator.next(); if (elem != null) { - TarantoolOp<?> future = elem.getValue(); - fail(future, error); + TarantoolRequest request = elem.getValue(); + fail(request, error); } iterator.remove(); } } + + TarantoolRequest request; + while ((request = delayedOpsQueue.poll()) != null) { + fail(request, error); + } + pendingResponsesCount.set(0); bufferLock.lock(); @@ -326,7 +364,7 @@ protected synchronized void die(String message, Exception cause) { } public void ping() { - syncGet(exec(Code.PING)); + syncGet(execute(pingRequest())); } protected void write(Code code, Long syncId, Long schemaId, Object... args) @@ -418,10 +456,10 @@ protected void readThread() { Map<Integer, Object> headers = packet.getHeaders(); Long syncId = (Long) headers.get(Key.SYNC.getId()); - TarantoolOp<?> future = futures.remove(syncId); + TarantoolRequest request = futures.remove(syncId); stats.received++; pendingResponsesCount.decrementAndGet(); - complete(packet, future); + complete(packet, request); } catch (Exception e) { die("Cant read answer", e); return; @@ -461,33 +499,181 @@ protected void writeThread() { } } - protected void fail(TarantoolOp<?> future, Exception e) { - future.completeExceptionally(e); + protected void fail(TarantoolRequest request, Exception e) { + request.getResult().completeExceptionally(e); + } + + protected void complete(TarantoolPacket packet, TarantoolRequest request) { + TarantoolOp<?> result = request.getResult(); + if (result.isDone()) { + return; + } + + long code = packet.getCode(); + long schemaId = packet.getSchemaId(); + if (code == ProtoConstants.SUCCESS) { + request.setCompletedSchemaId(schemaId); + if (request.isSync()) { + completeSync(request); + } else if (request.getCode() == Code.EXECUTE) { + completeSql(request, packet); + } else { + ((TarantoolOp) result).complete(packet.getData()); + } + } else if (code == ProtoConstants.ERR_WRONG_SCHEMA_VERSION) { + if (schemaId > schemaMeta.getSchemaVersion()) { + delayedOpsQueue.add(request); + } else { + registerOperation(request, schemaMeta.getSchemaVersion()); + } + } else { + Object error = packet.getError(); + fail(request, serverError(code, error)); + } + + if (request.getStartedSchemaId() == 0) { + return; + } + // it's possible to receive bigger version than current + // i.e. after DDL operation or wrong schema version response + if (schemaId > schemaMeta.getSchemaVersion()) { + updateSchema(); + } + // if the previous rescheduling was interrupted by + // this operation then continue the process + if (lastRecheduledOperationId != null && + request.getId() == lastRecheduledOperationId) { + updateSchema(); + } + } + + /** + * When a sync is successful it's required + * to re-evaluate dependent request and complete + * exceptionally with a proper error. + * + * @param request sync request + */ + private void completeSync(TarantoolRequest request) { + TarantoolRequest source = request.getSyncDependent(); + if (delayedOpsQueue.remove(source)) { + try { + source.getArguments(); + } catch (Exception e) { + fail(source, e); + } + } + } + + /** + * Optimistic means the client issues ping request + * to check whether it keeps an actual schema version or not. + * If the schema is outdated, client uses refreshes its schema + * cache. + * + * It's suitable when we don't refresh all the cache directly because + * of the schema can be large enough to be loaded fast. + * + * @see #updateSchema() + */ + private void optimisticSchemaUpdate(TarantoolRequest source) { + TarantoolRequest pingRequest = new TarantoolRequest(Code.PING); + pingRequest.beginSync(syncId.incrementAndGet(), operationTimeout, source); + registerOperation(pingRequest, schemaMeta.getSchemaVersion()); } - protected void complete(TarantoolPacket packet, TarantoolOp<?> future) { - if (future != null) { - long code = packet.getCode(); - if (code == 0) { - if (future.getCode() == Code.EXECUTE) { - completeSql(future, packet); + private void updateSchema() { + performSchemaAction(() -> { + if (state.acquire(StateHelper.SCHEMA_UPDATING)) { + lastRecheduledOperationId = null; + workExecutor.execute(createUpdateSchemaTask()); + } + }); + } + + private Runnable createUpdateSchemaTask() { + return () -> { + try { + schemaMeta.refresh(); + } finally { + performSchemaAction(() -> { + try { + rescheduleDelayedOperations(); + } finally { + state.release(StateHelper.SCHEMA_UPDATING); + } + }); + } + }; + } + + private void rescheduleDelayedOperations() { + List<TarantoolRequest> delayedRequests = new ArrayList<>(delayedOpsQueue.size()); + TarantoolRequest request; + // it's possible that DDL operation and following dependent + // unserializable request were delayed. + // thus, the client will take operations until + // both probable DDL and next unserializable ops are met + // [..., <create space A>, ..., <insert into A>, ...] + // | | | + // first DDL any serializable ops stop here, after first unserializable + boolean wasSchemaUnsafeRequest = false; + boolean wasUnserializableAfterUnsafe = false; + Long firstUnsafeOpId = null; + while ((request = delayedOpsQueue.poll()) != null) { + TarantoolOp<?> op = request.getResult(); + if (!op.isDone()) { + wasSchemaUnsafeRequest |= request.canModifySchema(); + if (firstUnsafeOpId == null && wasSchemaUnsafeRequest) { + firstUnsafeOpId = request.getId(); + registerOperation(request, schemaMeta.getSchemaVersion()); + if (!request.isSerializable()) { + firstUnsafeOpId = null; + wasSchemaUnsafeRequest = false; + } + } else if (wasSchemaUnsafeRequest) { + wasUnserializableAfterUnsafe |= !request.isSerializable(); + if (wasUnserializableAfterUnsafe) { + delayedRequests.add(request); + } else { + registerOperation(request, schemaMeta.getSchemaVersion()); + } } else { - ((TarantoolOp) future).complete(packet.getBody().get(Key.DATA.getId())); + registerOperation(request, schemaMeta.getSchemaVersion()); } - } else { - Object error = packet.getBody().get(Key.ERROR.getId()); - fail(future, serverError(code, error)); } } + // re-issue all operations starting from + // first unserializable request after possibly DDL request + if (!delayedRequests.isEmpty()) { + lastRecheduledOperationId = firstUnsafeOpId; + delayedOpsQueue.addAll(delayedRequests); + } } - protected void completeSql(TarantoolOp<?> future, TarantoolPacket pack) { + protected void completeSql(TarantoolRequest request, TarantoolPacket pack) { Long rowCount = SqlProtoUtils.getSQLRowCount(pack); + TarantoolOp<?> result = request.getResult(); if (rowCount != null) { - ((TarantoolOp) future).complete(rowCount); + ((TarantoolOp) result).complete(rowCount); } else { List<Map<String, Object>> values = SqlProtoUtils.readSqlResult(pack); - ((TarantoolOp) future).complete(values); + ((TarantoolOp) result).complete(values); + } + } + + /** + * Convenient guard scope that executes given runnable + * inside schema write lock. + * + * @param action to be executed + */ + protected void performSchemaAction(Runnable action) { + long stamp = schemaLock.writeLock(); + try { + action.run(); + } finally { + schemaLock.unlockWrite(stamp); } } @@ -523,6 +709,9 @@ public void close() { protected void close(Exception e) { if (state.close()) { + if (workExecutor != null) { + workExecutor.shutdownNow(); + } connector.interrupt(); die(e.getMessage(), e); } @@ -551,7 +740,7 @@ protected void stopIO() { * @return timeout in millis */ public long getOperationTimeout() { - return operationTimeout; + return operationTimeout.toMillis(); } /** @@ -560,17 +749,17 @@ public long getOperationTimeout() { * @param operationTimeout timeout in millis */ public void setOperationTimeout(long operationTimeout) { - this.operationTimeout = operationTimeout; + this.operationTimeout = Duration.ofMillis(operationTimeout); } @Override public boolean isAlive() { - return state.getState() == StateHelper.ALIVE && thumbstone == null; + return state.isStateSet(StateHelper.ALIVE) && thumbstone == null; } @Override public boolean isClosed() { - return state.getState() == StateHelper.CLOSED; + return state.isStateSet(StateHelper.CLOSED); } @Override @@ -584,36 +773,40 @@ public boolean waitAlive(long timeout, TimeUnit unit) throws InterruptedExceptio } @Override - public TarantoolClientOps<Integer, List<?>, Object, List<?>> syncOps() { + public TarantoolClientOps<List<?>, Object, List<?>> syncOps() { return syncOps; } @Override - public TarantoolClientOps<Integer, List<?>, Object, Future<List<?>>> asyncOps() { + public TarantoolClientOps<List<?>, Object, Future<List<?>>> asyncOps() { return (TarantoolClientOps) this; } @Override - public TarantoolClientOps<Integer, List<?>, Object, CompletionStage<List<?>>> composableAsyncOps() { + public TarantoolClientOps<List<?>, Object, CompletionStage<List<?>>> composableAsyncOps() { return composableAsyncOps; } @Override - public TarantoolClientOps<Integer, List<?>, Object, Long> fireAndForgetOps() { + public TarantoolClientOps<List<?>, Object, Long> fireAndForgetOps() { return fireAndForgetOps; } + public TarantoolClientOps<List<?>, Object, TupleTwo<List<?>, Long>> unsafeSchemaOps() { + return unsafeSchemaOps; + } + @Override public TarantoolSQLOps<Object, Long, List<Map<String, Object>>> sqlSyncOps() { return new TarantoolSQLOps<Object, Long, List<Map<String, Object>>>() { @Override public Long update(String sql, Object... bind) { - return (Long) syncGet(exec(Code.EXECUTE, Key.SQL_TEXT, sql, Key.SQL_BIND, bind)); + return (Long) syncGet(execute(executeRequest(sql).ordinalParameters(bind))); } @Override public List<Map<String, Object>> query(String sql, Object... bind) { - return (List<Map<String, Object>>) syncGet(exec(Code.EXECUTE, Key.SQL_TEXT, sql, Key.SQL_BIND, bind)); + return (List<Map<String, Object>>) syncGet(execute(executeRequest(sql).ordinalParameters(bind))); } }; } @@ -623,39 +816,32 @@ public TarantoolSQLOps<Object, Future<Long>, Future<List<Map<String, Object>>>> return new TarantoolSQLOps<Object, Future<Long>, Future<List<Map<String, Object>>>>() { @Override public Future<Long> update(String sql, Object... bind) { - return (Future<Long>) exec(Code.EXECUTE, Key.SQL_TEXT, sql, Key.SQL_BIND, bind); + return (Future<Long>) execute(executeRequest(sql).ordinalParameters(bind)); } @Override public Future<List<Map<String, Object>>> query(String sql, Object... bind) { - return (Future<List<Map<String, Object>>>) exec(Code.EXECUTE, Key.SQL_TEXT, sql, Key.SQL_BIND, bind); + return (Future<List<Map<String, Object>>>) execute(executeRequest(sql).ordinalParameters(bind)); } }; } - protected class SyncOps extends AbstractTarantoolOps<Integer, List<?>, Object, List<?>> { + protected class SyncOps extends BaseClientOps<List<?>> { @Override - public List exec(Code code, Object... args) { - return (List) syncGet(TarantoolClientImpl.this.exec(code, args)); - } - - @Override - public void close() { - throw new IllegalStateException("You should close TarantoolClient instead."); + protected List<?> exec(TarantoolRequest request) { + return (List) syncGet(TarantoolClientImpl.this.exec(request)); } } - protected class FireAndForgetOps extends AbstractTarantoolOps<Integer, List<?>, Object, Long> { + protected class FireAndForgetOps extends BaseClientOps<Long> { @Override - public Long exec(Code code, Object... args) { + protected Long exec(TarantoolRequest request) { if (thumbstone == null) { try { - long syncId = TarantoolClientImpl.this.syncId.incrementAndGet(); - write(code, syncId, null, args); - return syncId; + return doExec(request).getId(); } catch (Exception e) { throw new CommunicationException("Execute failed", e); } @@ -664,104 +850,16 @@ public Long exec(Code code, Object... args) { } } - @Override - public void close() { - throw new IllegalStateException("You should close TarantoolClient instead."); - } - } - protected boolean isDead(TarantoolOp<?> future) { + protected boolean isDead(TarantoolRequest request) { if (this.thumbstone != null) { - fail(future, new CommunicationException("Connection is dead", thumbstone)); + fail(request, new CommunicationException("Connection is dead", thumbstone)); return true; } return false; } - protected static class TarantoolOp<V> extends CompletableFuture<V> { - - /** - * A task identifier used in {@link TarantoolClientImpl#futures}. - */ - private final long id; - - /** - * Tarantool binary protocol operation code. - */ - private final Code code; - - /** - * Arguments of operation. - */ - private final Object[] args; - - public TarantoolOp(long id, Code code, Object[] args) { - this.id = id; - this.code = code; - this.args = args; - } - - public long getId() { - return id; - } - - public Code getCode() { - return code; - } - - public Object[] getArgs() { - return args; - } - - /** - * Missed in jdk8 CompletableFuture operator to limit execution - * by time. - */ - public TarantoolOp<V> orTimeout(long timeout, TimeUnit unit) { - if (timeout < 0) { - throw new IllegalArgumentException("Timeout cannot be negative"); - } - if (unit == null) { - throw new IllegalArgumentException("Time unit cannot be null"); - } - if (timeout == 0 || isDone()) { - return this; - } - ScheduledFuture<?> abandonByTimeoutAction = TimeoutScheduler.EXECUTOR.schedule( - () -> { - if (!this.isDone()) { - this.completeExceptionally(new TimeoutException()); - } - }, - timeout, unit - ); - whenComplete( - (ignored, error) -> { - if (error == null && !abandonByTimeoutAction.isDone()) { - abandonByTimeoutAction.cancel(false); - } - } - ); - return this; - } - - /** - * Runs timeout operation as a delayed task. - */ - static class TimeoutScheduler { - - static final ScheduledThreadPoolExecutor EXECUTOR; - - static { - EXECUTOR = - new ScheduledThreadPoolExecutor(1, new TarantoolThreadDaemonFactory("tarantoolTimeout")); - EXECUTOR.setRemoveOnCancelPolicy(true); - } - } - - } - /** * A subclass may use this as a trigger to start retries. * This method is called when state becomes ALIVE. @@ -786,9 +884,12 @@ protected final class StateHelper { static final int UNINITIALIZED = 0; static final int READING = 1; static final int WRITING = 2; + static final int ALIVE = READING | WRITING; - static final int RECONNECT = 4; - static final int CLOSED = 8; + static final int SCHEMA_UPDATING = 1 << 2; + + static final int RECONNECT = 1 << 3; + static final int CLOSED = 1 << 4; private final AtomicInteger state; @@ -817,6 +918,10 @@ protected int getState() { return state.get(); } + boolean isStateSet(int mask) { + return (getState() & mask) == mask; + } + /** * Set CLOSED state, drop RECONNECT state. */ @@ -825,12 +930,12 @@ protected boolean close() { int currentState = getState(); /* CLOSED is the terminal state. */ - if ((currentState & CLOSED) == CLOSED) { + if (isStateSet(CLOSED)) { return false; } - /* Drop RECONNECT, set CLOSED. */ - if (compareAndSet(currentState, (currentState & ~RECONNECT) | CLOSED)) { + /* Clear all states and set CLOSED. */ + if (compareAndSet(currentState, CLOSED)) { return true; } } @@ -846,7 +951,7 @@ protected boolean acquire(int mask) { int currentState = getState(); /* CLOSED is the terminal state. */ - if ((currentState & CLOSED) == CLOSED) { + if ((isStateSet(CLOSED))) { return false; } @@ -856,8 +961,8 @@ protected boolean acquire(int mask) { } /* Cannot move from a state to the same state. */ - if ((currentState & mask) != 0) { - throw new IllegalStateException("State is already " + mask); + if (isStateSet(mask)) { + return false; } /* Set acquired state. */ @@ -881,7 +986,8 @@ protected boolean compareAndSet(int expect, int update) { return false; } - if (update == ALIVE) { + boolean wasAlreadyAlive = (expect & ALIVE) == ALIVE; + if (!wasAlreadyAlive && (update & ALIVE) == ALIVE) { CountDownLatch latch = nextAliveLatch.getAndSet(new CountDownLatch(1)); latch.countDown(); onReconnect(); @@ -916,13 +1022,13 @@ private CountDownLatch getStateLatch(int state) { return closedLatch; } if (state == ALIVE) { - if (getState() == CLOSED) { + if (isStateSet(CLOSED)) { throw new IllegalStateException("State is CLOSED."); } CountDownLatch latch = nextAliveLatch.get(); /* It may happen so that an error is detected but the state is still alive. Wait for the 'next' alive state in such cases. */ - return (getState() == ALIVE && thumbstone == null) ? null : latch; + return (isStateSet(ALIVE) && thumbstone == null) ? null : latch; } return null; } @@ -935,7 +1041,7 @@ private CountDownLatch getStateLatch(int state) { private void awaitReconnection() throws InterruptedException { connectorLock.lock(); try { - while (getState() != StateHelper.RECONNECT) { + while (!isStateSet(RECONNECT)) { reconnectRequired.await(); } } finally { @@ -961,12 +1067,11 @@ private void trySignalForReconnection() { } - protected class ComposableAsyncOps - extends AbstractTarantoolOps<Integer, List<?>, Object, CompletionStage<List<?>>> { + protected class ComposableAsyncOps extends BaseClientOps<CompletionStage<List<?>>> { @Override - public CompletionStage<List<?>> exec(Code code, Object... args) { - return (CompletionStage<List<?>>) TarantoolClientImpl.this.exec(code, args); + protected CompletionStage<List<?>> exec(TarantoolRequest request) { + return (CompletionStage<List<?>>) TarantoolClientImpl.this.exec(request); } @Override @@ -976,4 +1081,32 @@ public void close() { } + /** + * Used by internal services to ignore schema ID issues. + */ + protected class UnsafeSchemaOps extends BaseClientOps<TupleTwo<List<?>, Long>> { + + protected TupleTwo<List<?>, Long> exec(TarantoolRequest request) { + long syncId = TarantoolClientImpl.this.syncId.incrementAndGet(); + request.begin(syncId, operationTimeout); + List<?> result = (List<?>) syncGet(registerOperation(request, 0L).getResult()); + return TupleTwo.of(result, request.getCompletedSchemaId()); + } + + } + + protected abstract class BaseClientOps<R> extends AbstractTarantoolOps<R> { + + @Override + protected TarantoolSchemaMeta getSchemaMeta() { + return TarantoolClientImpl.this.getSchemaMeta(); + } + + @Override + public void close() { + throw new IllegalStateException("You should close TarantoolClient instead."); + } + + } + } diff --git a/src/main/java/org/tarantool/TarantoolClientOps.java b/src/main/java/org/tarantool/TarantoolClientOps.java index 69ab3a9e..880102fa 100644 --- a/src/main/java/org/tarantool/TarantoolClientOps.java +++ b/src/main/java/org/tarantool/TarantoolClientOps.java @@ -1,20 +1,35 @@ package org.tarantool; +import org.tarantool.dsl.TarantoolRequestConvertible; -public interface TarantoolClientOps<T,O,P,R> { - R select(T space, T index, O key, int offset, int limit, int iterator); +public interface TarantoolClientOps<O, P, R> { + R select(Integer space, Integer index, O key, int offset, int limit, int iterator); - R select(T space, T index, O key, int offset, int limit, Iterator iterator); + R select(String space, String index, O key, int offset, int limit, int iterator); - R insert(T space, O tuple); + R select(Integer space, Integer index, O key, int offset, int limit, Iterator iterator); - R replace(T space, O tuple); + R select(String space, String index, O key, int offset, int limit, Iterator iterator); - R update(T space, O key, P... tuple); + R insert(Integer space, O tuple); - R upsert(T space, O key, O defTuple, P... ops); + R insert(String space, O tuple); - R delete(T space, O key); + R replace(Integer space, O tuple); + + R replace(String space, O tuple); + + R update(Integer space, O key, P... tuple); + + R update(String space, O key, P... tuple); + + R upsert(Integer space, O key, O defTuple, P... ops); + + R upsert(String space, O key, O defTuple, P... ops); + + R delete(Integer space, O key); + + R delete(String space, O key); R call(String function, Object... args); @@ -22,5 +37,7 @@ public interface TarantoolClientOps<T,O,P,R> { void ping(); + R execute(TarantoolRequestConvertible requestSpec); + void close(); } diff --git a/src/main/java/org/tarantool/TarantoolClusterClient.java b/src/main/java/org/tarantool/TarantoolClusterClient.java index b0c4711b..8162473d 100644 --- a/src/main/java/org/tarantool/TarantoolClusterClient.java +++ b/src/main/java/org/tarantool/TarantoolClusterClient.java @@ -14,7 +14,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.StampedLock; @@ -35,14 +34,13 @@ public class TarantoolClusterClient extends TarantoolClientImpl { /** * Discovery activity. */ - private ScheduledExecutorService instancesDiscoveryExecutor; private Runnable instancesDiscovererTask; private StampedLock discoveryLock = new StampedLock(); /** * Collection of operations to be retried. */ - private ConcurrentHashMap<Long, TarantoolOp<?>> retries = new ConcurrentHashMap<>(); + private ConcurrentHashMap<Long, TarantoolRequest> retries = new ConcurrentHashMap<>(); /** * Constructs a new cluster client. @@ -70,14 +68,12 @@ public TarantoolClusterClient(TarantoolClusterClientConfig config, SocketChannel if (StringUtils.isNotBlank(config.clusterDiscoveryEntryFunction)) { this.instancesDiscovererTask = createDiscoveryTask(new TarantoolClusterStoredFunctionDiscoverer(config, this)); - this.instancesDiscoveryExecutor - = Executors.newSingleThreadScheduledExecutor(new TarantoolThreadDaemonFactory("tarantoolDiscoverer")); int delay = config.clusterDiscoveryDelayMillis > 0 ? config.clusterDiscoveryDelayMillis : TarantoolClusterClientConfig.DEFAULT_CLUSTER_DISCOVERY_DELAY_MILLIS; // todo: it's better to start a job later (out of ctor) - this.instancesDiscoveryExecutor.scheduleWithFixedDelay( + this.workExecutor.scheduleWithFixedDelay( this.instancesDiscovererTask, 0, delay, @@ -87,72 +83,49 @@ public TarantoolClusterClient(TarantoolClusterClientConfig config, SocketChannel } @Override - protected boolean isDead(TarantoolOp<?> future) { + protected boolean isDead(TarantoolRequest request) { if ((state.getState() & StateHelper.CLOSED) != 0) { - future.completeExceptionally(new CommunicationException("Connection is dead", thumbstone)); + request.getResult().completeExceptionally(new CommunicationException("Connection is dead", thumbstone)); return true; } Exception err = thumbstone; if (err != null) { - return checkFail(future, err); + return checkFail(request, err); } return false; } - @Override - protected TarantoolOp<?> doExec(long timeoutMillis, Code code, Object[] args) { - validateArgs(args); - long sid = syncId.incrementAndGet(); - TarantoolOp<?> future = makeNewOperation(timeoutMillis, sid, code, args); - return registerOperation(future); - } - /** * Registers a new async operation which will be resolved later. * Registration is discovery-aware in term of synchronization and * it may be blocked util the discovery finishes its work. * - * @param future operation to be performed + * @param request operation to be performed * * @return registered operation */ - private TarantoolOp<?> registerOperation(TarantoolOp<?> future) { + @Override + protected TarantoolRequest registerOperation(TarantoolRequest request, long schemaId) { long stamp = discoveryLock.readLock(); try { - if (isDead(future)) { - return future; - } - futures.put(future.getId(), future); - if (isDead(future)) { - futures.remove(future.getId()); - return future; - } - - try { - write(future.getCode(), future.getId(), null, future.getArgs()); - } catch (Exception e) { - futures.remove(future.getId()); - fail(future, e); - } - - return future; + return super.registerOperation(request, schemaId); } finally { discoveryLock.unlock(stamp); } } @Override - protected void fail(TarantoolOp<?> future, Exception e) { - checkFail(future, e); + protected void fail(TarantoolRequest request, Exception e) { + checkFail(request, e); } - protected boolean checkFail(TarantoolOp<?> future, Exception e) { + protected boolean checkFail(TarantoolRequest request, Exception e) { if (!isTransientError(e)) { - future.completeExceptionally(e); + request.getResult().completeExceptionally(e); return true; } else { assert retries != null; - retries.put(future.getId(), future); + retries.put(request.getId(), request); return false; } } @@ -161,17 +134,13 @@ protected boolean checkFail(TarantoolOp<?> future, Exception e) { protected void close(Exception e) { super.close(e); - if (instancesDiscoveryExecutor != null) { - instancesDiscoveryExecutor.shutdownNow(); - } - if (retries == null) { // May happen within constructor. return; } - for (TarantoolOp<?> op : retries.values()) { - op.completeExceptionally(e); + for (TarantoolRequest request : retries.values()) { + request.getResult().completeExceptionally(e); } } @@ -194,18 +163,18 @@ protected void onReconnect() { // First call is before the constructor finished. Skip it. return; } - Collection<TarantoolOp<?>> futuresToRetry = new ArrayList<>(retries.values()); + Collection<TarantoolRequest> futuresToRetry = new ArrayList<>(retries.values()); retries.clear(); - for (final TarantoolOp<?> future : futuresToRetry) { - if (!future.isDone()) { - executor.execute(() -> registerOperation(future)); + for (final TarantoolRequest request : futuresToRetry) { + if (!request.getResult().isDone()) { + executor.execute(() -> registerOperation(request, schemaMeta.getSchemaVersion())); } } } @Override - protected void complete(TarantoolPacket packet, TarantoolOp<?> future) { - super.complete(packet, future); + protected void complete(TarantoolPacket packet, TarantoolRequest request) { + super.complete(packet, request); RefreshableSocketProvider provider = getRefreshableSocketProvider(); if (provider != null) { renewConnectionIfRequired(provider.getAddresses()); @@ -278,7 +247,6 @@ public synchronized void run() { onInstancesRefreshed(lastInstances); } } catch (Exception ignored) { - ignored.getCause(); // no-op } } diff --git a/src/main/java/org/tarantool/TarantoolConnection.java b/src/main/java/org/tarantool/TarantoolConnection.java index 09883bc0..4db84872 100644 --- a/src/main/java/org/tarantool/TarantoolConnection.java +++ b/src/main/java/org/tarantool/TarantoolConnection.java @@ -2,6 +2,7 @@ import org.tarantool.protocol.ProtoUtils; import org.tarantool.protocol.TarantoolPacket; +import org.tarantool.schema.TarantoolSchemaMeta; import java.io.IOException; import java.io.InputStream; @@ -27,11 +28,17 @@ public TarantoolConnection(String username, String password, Socket socket) thro } @Override - protected List<?> exec(Code code, Object... args) { - TarantoolPacket responsePacket = writeAndRead(code, args); + protected List<?> exec(TarantoolRequest request) { + Object[] args = request.getArguments().toArray(); + TarantoolPacket responsePacket = writeAndRead(request.getCode(), args); return (List) responsePacket.getBody().get(Key.DATA.getId()); } + @Override + protected TarantoolSchemaMeta getSchemaMeta() { + return null; + } + protected TarantoolPacket writeAndRead(Code code, Object... args) { try { ByteBuffer packet = ProtoUtils.createPacket(initialRequestSize, msgPackLite, @@ -44,7 +51,7 @@ protected TarantoolPacket writeAndRead(Code code, Object... args) { Long c = responsePacket.getCode(); if (c != 0) { - throw serverError(c, responsePacket.getBody().get(Key.ERROR.getId())); + throw serverError(c, responsePacket.getError()); } return responsePacket; diff --git a/src/main/java/org/tarantool/TarantoolException.java b/src/main/java/org/tarantool/TarantoolException.java index e7d38e82..10afdfb8 100644 --- a/src/main/java/org/tarantool/TarantoolException.java +++ b/src/main/java/org/tarantool/TarantoolException.java @@ -1,5 +1,11 @@ package org.tarantool; +import static org.tarantool.protocol.ProtoConstants.ERR_LOADING; +import static org.tarantool.protocol.ProtoConstants.ERR_LOCAL_INSTANCE_ID_IS_READ_ONLY; +import static org.tarantool.protocol.ProtoConstants.ERR_READONLY; +import static org.tarantool.protocol.ProtoConstants.ERR_TIMEOUT; +import static org.tarantool.protocol.ProtoConstants.ERR_WRONG_SCHEMA_VERSION; + /** * A remote server error with error code and message. * @@ -7,11 +13,6 @@ * @version $Id: $ */ public class TarantoolException extends RuntimeException { - /* taken from src/box/errcode.h */ - public static final int ERR_READONLY = 7; - public static final int ERR_TIMEOUT = 78; - public static final int ERR_LOADING = 116; - public static final int ERR_LOCAL_INSTANCE_ID_IS_READ_ONLY = 128; private static final long serialVersionUID = 1L; long code; @@ -60,6 +61,7 @@ public boolean isTransient() { switch ((int) code) { case ERR_READONLY: case ERR_TIMEOUT: + case ERR_WRONG_SCHEMA_VERSION: case ERR_LOADING: case ERR_LOCAL_INSTANCE_ID_IS_READ_ONLY: return true; diff --git a/src/main/java/org/tarantool/TarantoolOp.java b/src/main/java/org/tarantool/TarantoolOp.java new file mode 100644 index 00000000..0c75f4b0 --- /dev/null +++ b/src/main/java/org/tarantool/TarantoolOp.java @@ -0,0 +1,57 @@ +package org.tarantool; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public class TarantoolOp<V> extends CompletableFuture<V> { + + /** + * Missed in jdk8 CompletableFuture operator to limit execution + * by time. + */ + public TarantoolOp<V> orTimeout(long timeout, TimeUnit unit) { + if (timeout < 0) { + throw new IllegalArgumentException("Timeout cannot be negative"); + } + if (unit == null) { + throw new IllegalArgumentException("Time unit cannot be null"); + } + if (timeout == 0 || isDone()) { + return this; + } + ScheduledFuture<?> abandonByTimeoutAction = TimeoutScheduler.EXECUTOR.schedule( + () -> { + if (!this.isDone()) { + this.completeExceptionally(new TimeoutException()); + } + }, + timeout, unit + ); + whenComplete( + (ignored, error) -> { + if (error == null && !abandonByTimeoutAction.isDone()) { + abandonByTimeoutAction.cancel(false); + } + } + ); + return this; + } + + /** + * Runs timeout operation as a delayed task. + */ + static class TimeoutScheduler { + + static final ScheduledThreadPoolExecutor EXECUTOR; + + static { + EXECUTOR = + new ScheduledThreadPoolExecutor(1, new TarantoolThreadDaemonFactory("tarantoolTimeout")); + EXECUTOR.setRemoveOnCancelPolicy(true); + } + } + +} diff --git a/src/main/java/org/tarantool/TarantoolRequest.java b/src/main/java/org/tarantool/TarantoolRequest.java new file mode 100644 index 00000000..71d31d0c --- /dev/null +++ b/src/main/java/org/tarantool/TarantoolRequest.java @@ -0,0 +1,174 @@ +package org.tarantool; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class TarantoolRequest implements Comparable<TarantoolRequest> { + + /** + * A task identifier used in {@link TarantoolClientImpl#futures}. + */ + private long id; + + /** + * Tarantool binary protocol operation code. + */ + private Code code; + + /** + * Schema ID when this future was registered + * successfully. + */ + private long startedSchemaId; + + /** + * Schema ID when this future was completed + * successfully. + */ + private long completedSchemaId; + + /** + * Arguments of operation. + */ + private List<TarantoolRequestArgument> arguments; + + /** + * Request timeout start just after initialization. + */ + private Duration timeout = Duration.ZERO; + + /** + * Asynchronous request result. + */ + private TarantoolOp<?> result = new TarantoolOp<>(); + + /** + * Internal flag indicating that this request + * is used to sync up the schema and if nothing + * changes then it will fail the dependent request. + */ + private boolean sync; + private TarantoolRequest syncDependent; + + public TarantoolRequest(Code code) { + this.code = code; + this.arguments = new ArrayList<>(); + } + + public TarantoolRequest(Code code, TarantoolRequestArgument... arguments) { + this.code = code; + this.arguments = Arrays.asList(arguments); + } + + /** + * Initializes a request and starts its timer. + * + * @param sid internal request id + * @param defaultTimeout default timeout + * + * @return issued operation + */ + void begin(long sid, Duration defaultTimeout) { + this.id = sid; + Objects.requireNonNull(defaultTimeout); + long time = timeout == null ? defaultTimeout.toMillis() : timeout.toMillis(); + result.orTimeout(time, TimeUnit.MILLISECONDS); + } + + void beginSync(long sid, Duration defaultTimeout, TarantoolRequest syncDependent) { + this.id = sid; + Objects.requireNonNull(defaultTimeout); + Objects.requireNonNull(syncDependent); + this.syncDependent = syncDependent; + this.sync = true; + long time = timeout == null ? defaultTimeout.toMillis() : timeout.toMillis(); + result.orTimeout(time, TimeUnit.MILLISECONDS); + } + + public long getId() { + return id; + } + + public long getStartedSchemaId() { + return startedSchemaId; + } + + public void setStartedSchemaId(long startedSchemaId) { + this.startedSchemaId = startedSchemaId; + } + + public long getCompletedSchemaId() { + return completedSchemaId; + } + + public void setCompletedSchemaId(long completedSchemaId) { + this.completedSchemaId = completedSchemaId; + } + + public TarantoolOp<?> getResult() { + return result; + } + + public Code getCode() { + return code; + } + + public void setCode(Code code) { + this.code = code; + } + + public Duration getTimeout() { + return timeout; + } + + public void setTimeout(Duration timeout) { + this.timeout = timeout; + } + + boolean isSync() { + return sync; + } + + TarantoolRequest getSyncDependent() { + return syncDependent; + } + + /** + * Serializability means this requests is capable being + * translated in a binary packet according to {@code iproto} + * protocol. + * + * @return {@literal true} if this request is serializable + */ + public boolean isSerializable() { + return arguments.stream().allMatch(TarantoolRequestArgument::isSerializable); + } + + public List<Object> getArguments() { + return arguments.stream().map(TarantoolRequestArgument::getValue).collect(Collectors.toList()); + } + + public void addArguments(TarantoolRequestArgument... arguments) { + this.arguments.addAll(Arrays.asList(arguments)); + } + + /** + * Checks request DDL-status. + * + * @return {@literal true} if request possibly is DDL + */ + public boolean canModifySchema() { + return code == Code.OLD_CALL || code == Code.CALL || code == Code.EVAL || code == Code.EXECUTE; + } + + @Override + public int compareTo(TarantoolRequest other) { + return Long.compareUnsigned(this.id, other.id); + } + +} diff --git a/src/main/java/org/tarantool/TarantoolRequestArgument.java b/src/main/java/org/tarantool/TarantoolRequestArgument.java new file mode 100644 index 00000000..6ff11c21 --- /dev/null +++ b/src/main/java/org/tarantool/TarantoolRequestArgument.java @@ -0,0 +1,23 @@ +package org.tarantool; + +/** + * Holds a request argument value. + */ +public interface TarantoolRequestArgument { + + /** + * Flag indicating that held value can be + * represented as bytes supported by iproto. + * + * @return {@literal true} if value is {@code iproto} serializable + */ + boolean isSerializable(); + + /** + * Gets a held value. + * + * @return wrapped value + */ + Object getValue(); + +} diff --git a/src/main/java/org/tarantool/cluster/TarantoolClusterStoredFunctionDiscoverer.java b/src/main/java/org/tarantool/cluster/TarantoolClusterStoredFunctionDiscoverer.java index da92cd3a..e905f77f 100644 --- a/src/main/java/org/tarantool/cluster/TarantoolClusterStoredFunctionDiscoverer.java +++ b/src/main/java/org/tarantool/cluster/TarantoolClusterStoredFunctionDiscoverer.java @@ -1,9 +1,10 @@ package org.tarantool.cluster; -import org.tarantool.TarantoolClient; +import org.tarantool.TarantoolClientImpl; import org.tarantool.TarantoolClientOps; import org.tarantool.TarantoolClusterClientConfig; import org.tarantool.util.StringUtils; +import org.tarantool.util.TupleTwo; import java.util.LinkedHashSet; import java.util.List; @@ -13,32 +14,35 @@ /** * A cluster nodes discoverer based on calling a predefined function * which returns list of nodes. - * + * <p> * The function has to have no arguments and return list of * the strings which follow <code>host[:port]</code> format + * <p> + * This class is not a part of public API. */ public class TarantoolClusterStoredFunctionDiscoverer implements TarantoolClusterDiscoverer { - private TarantoolClient client; + private TarantoolClientImpl client; private String entryFunction; - public TarantoolClusterStoredFunctionDiscoverer(TarantoolClusterClientConfig clientConfig, TarantoolClient client) { + public TarantoolClusterStoredFunctionDiscoverer(TarantoolClusterClientConfig clientConfig, + TarantoolClientImpl client) { this.client = client; this.entryFunction = clientConfig.clusterDiscoveryEntryFunction; } @Override public Set<String> getInstances() { - TarantoolClientOps<Integer, List<?>, Object, List<?>> syncOperations = client.syncOps(); + TarantoolClientOps<List<?>, Object, TupleTwo<List<?>, Long>> syncOperations = client.unsafeSchemaOps(); - List<?> list = syncOperations.call(entryFunction); + TupleTwo<List<?>, Long> result = syncOperations.call(entryFunction); // discoverer expects a single array result from the function now; // in order to protect this contract the discoverer does a strict // validation against the data returned; // this strict-mode allows us to extend the contract in a non-breaking // way for old clients just reserve an extra return value in // terms of Lua multi-result support.; - return checkAndFilterAddresses(list); + return checkAndFilterAddresses(result.getFirst()); } /** diff --git a/src/main/java/org/tarantool/dsl/AbstractRequestSpec.java b/src/main/java/org/tarantool/dsl/AbstractRequestSpec.java new file mode 100644 index 00000000..bbb68ce2 --- /dev/null +++ b/src/main/java/org/tarantool/dsl/AbstractRequestSpec.java @@ -0,0 +1,46 @@ +package org.tarantool.dsl; + +import org.tarantool.Code; +import org.tarantool.TarantoolRequest; +import org.tarantool.schema.TarantoolSchemaMeta; + +import java.time.Duration; + +public abstract class AbstractRequestSpec<B extends AbstractRequestSpec<B>> + implements TarantoolRequestConvertible { + + final Code code; + Duration duration = Duration.ZERO; + boolean useDefaultTimeout = true; + + AbstractRequestSpec(Code code) { + this.code = code; + } + + AbstractRequestSpec(Code code, Duration duration) { + this.code = code; + this.duration = duration; + } + + @SuppressWarnings("unchecked") + public B timeout(Duration duration) { + this.duration = duration; + this.useDefaultTimeout = false; + return (B) this; + } + + @SuppressWarnings("unchecked") + public B useDefaultTimeout() { + this.duration = Duration.ZERO; + this.useDefaultTimeout = true; + return (B) this; + } + + @Override + public TarantoolRequest toTarantoolRequest(TarantoolSchemaMeta schemaMeta) { + TarantoolRequest request = new TarantoolRequest(code); + request.setTimeout(useDefaultTimeout ? null : duration); + return request; + } + +} diff --git a/src/main/java/org/tarantool/dsl/CallRequestSpec.java b/src/main/java/org/tarantool/dsl/CallRequestSpec.java new file mode 100644 index 00000000..44912499 --- /dev/null +++ b/src/main/java/org/tarantool/dsl/CallRequestSpec.java @@ -0,0 +1,63 @@ +package org.tarantool.dsl; + +import static org.tarantool.RequestArguments.value; + +import org.tarantool.Code; +import org.tarantool.Key; +import org.tarantool.TarantoolRequest; +import org.tarantool.schema.TarantoolSchemaMeta; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +public class CallRequestSpec extends AbstractRequestSpec<CallRequestSpec> { + + private String functionName; + private List<Object> arguments = new ArrayList<>(); + private boolean useCall16 = false; + + CallRequestSpec(String functionName) { + super(Code.CALL); + this.functionName = Objects.requireNonNull(functionName); + } + + public CallRequestSpec function(String functionName) { + Objects.requireNonNull(functionName); + this.functionName = functionName; + return this; + } + + public CallRequestSpec arguments(Object... arguments) { + this.arguments.clear(); + Collections.addAll(this.arguments, arguments); + return this; + } + + public CallRequestSpec arguments(Collection<?> arguments) { + this.arguments.clear(); + this.arguments.addAll(arguments); + return this; + } + + public CallRequestSpec useCall16(boolean flag) { + this.useCall16 = flag; + return this; + } + + @Override + public TarantoolRequest toTarantoolRequest(TarantoolSchemaMeta schemaMeta) { + TarantoolRequest request = super.toTarantoolRequest(schemaMeta); + if (useCall16) { + request.setCode(Code.OLD_CALL); + } + request.addArguments( + value(Key.FUNCTION), value(functionName), + value(Key.TUPLE), value(arguments) + ); + return request; + } + +} diff --git a/src/main/java/org/tarantool/dsl/DeleteRequestSpec.java b/src/main/java/org/tarantool/dsl/DeleteRequestSpec.java new file mode 100644 index 00000000..051abdef --- /dev/null +++ b/src/main/java/org/tarantool/dsl/DeleteRequestSpec.java @@ -0,0 +1,66 @@ +package org.tarantool.dsl; + +import static org.tarantool.RequestArguments.cacheLookupValue; +import static org.tarantool.RequestArguments.value; + +import org.tarantool.Code; +import org.tarantool.Key; +import org.tarantool.TarantoolRequest; +import org.tarantool.schema.TarantoolSchemaMeta; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +public class DeleteRequestSpec extends SpaceRequestSpec<DeleteRequestSpec> { + + private List<Object> key; + + DeleteRequestSpec(int spaceId, List<?> key) { + super(Code.DELETE, spaceId); + this.key = new ArrayList<>(key); + } + + DeleteRequestSpec(int spaceId, Object... keyParts) { + super(Code.DELETE, spaceId); + this.key = Arrays.asList(keyParts); + } + + DeleteRequestSpec(String spaceName, List<?> key) { + super(Code.DELETE, spaceName); + this.key = new ArrayList<>(key); + } + + DeleteRequestSpec(String spaceName, Object... keyParts) { + super(Code.DELETE, spaceName); + this.key = Arrays.asList(keyParts); + } + + public DeleteRequestSpec primaryKey(Object... keyParts) { + this.key.clear(); + Collections.addAll(this.key, keyParts); + return this; + } + + public DeleteRequestSpec primaryKey(Collection<?> key) { + this.key.clear(); + this.key.addAll(key); + return this; + } + + @Override + public TarantoolRequest toTarantoolRequest(TarantoolSchemaMeta schemaMeta) { + TarantoolRequest request = super.toTarantoolRequest(schemaMeta); + request.addArguments( + value(Key.SPACE), + spaceId == null + ? cacheLookupValue(() -> schemaMeta.getSpace(spaceName).getId()) + : value(spaceId), + value(Key.KEY), value(key) + ); + return request; + } + +} diff --git a/src/main/java/org/tarantool/dsl/EvalRequestSpec.java b/src/main/java/org/tarantool/dsl/EvalRequestSpec.java new file mode 100644 index 00000000..4455f2b6 --- /dev/null +++ b/src/main/java/org/tarantool/dsl/EvalRequestSpec.java @@ -0,0 +1,54 @@ +package org.tarantool.dsl; + +import static org.tarantool.RequestArguments.value; + +import org.tarantool.Code; +import org.tarantool.Key; +import org.tarantool.TarantoolRequest; +import org.tarantool.schema.TarantoolSchemaMeta; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +public class EvalRequestSpec extends AbstractRequestSpec<EvalRequestSpec> { + + private String expression; + private List<Object> arguments = new ArrayList<>(); + + EvalRequestSpec(String expression) { + super(Code.EVAL); + this.expression = Objects.requireNonNull(expression); + } + + public EvalRequestSpec expression(String expression) { + Objects.requireNonNull(expression); + this.expression = expression; + return this; + } + + public EvalRequestSpec arguments(Object... arguments) { + this.arguments.clear(); + Collections.addAll(this.arguments, arguments); + return this; + } + + public EvalRequestSpec arguments(Collection<?> arguments) { + this.arguments.clear(); + this.arguments.addAll(arguments); + return this; + } + + @Override + public TarantoolRequest toTarantoolRequest(TarantoolSchemaMeta schemaMeta) { + TarantoolRequest request = super.toTarantoolRequest(schemaMeta); + request.addArguments( + value(Key.EXPRESSION), value(expression), + value(Key.TUPLE), value(arguments) + ); + return request; + } + +} diff --git a/src/main/java/org/tarantool/dsl/ExecuteRequestSpec.java b/src/main/java/org/tarantool/dsl/ExecuteRequestSpec.java new file mode 100644 index 00000000..043d0154 --- /dev/null +++ b/src/main/java/org/tarantool/dsl/ExecuteRequestSpec.java @@ -0,0 +1,91 @@ +package org.tarantool.dsl; + +import org.tarantool.Code; +import org.tarantool.Key; +import org.tarantool.RequestArguments; +import org.tarantool.TarantoolRequest; +import org.tarantool.schema.TarantoolSchemaMeta; +import org.tarantool.util.TupleTwo; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +public class ExecuteRequestSpec extends AbstractRequestSpec<ExecuteRequestSpec> { + + private String sqlText; + private List<Object> ordinalBindings = new ArrayList<>(); + private List<TupleTwo<String, Object>> namedBindings = new ArrayList<>(); + + ExecuteRequestSpec(String sqlText) { + super(Code.EXECUTE); + this.sqlText = Objects.requireNonNull(sqlText); + } + + public ExecuteRequestSpec sql(String text) { + Objects.requireNonNull(text); + this.sqlText = text; + return this; + } + + public ExecuteRequestSpec ordinalParameters(Object... bindings) { + this.ordinalBindings.clear(); + Collections.addAll(this.ordinalBindings, bindings); + this.namedBindings.clear(); + return this; + } + + public ExecuteRequestSpec ordinalParameters(Collection<?> bindings) { + this.ordinalBindings.clear(); + this.ordinalBindings.addAll(bindings); + this.namedBindings.clear(); + return this; + } + + public ExecuteRequestSpec namedParameters(Map<String, ?> bindings) { + this.namedBindings.clear(); + this.namedBindings.addAll( + bindings.entrySet().stream() + .map(e -> TupleTwo.<String, Object>of(e.getKey(), e.getValue())) + .collect(Collectors.toList()) + ); + this.ordinalBindings.clear(); + return this; + } + + public ExecuteRequestSpec namedParameters(TupleTwo<String, ?>[] bindings) { + this.namedBindings.clear(); + for (TupleTwo<String, ?> binding : bindings) { + this.namedBindings.add(TupleTwo.of(binding.getFirst(), binding.getSecond())); + } + this.ordinalBindings.clear(); + return this; + } + + @Override + public TarantoolRequest toTarantoolRequest(TarantoolSchemaMeta schemaMeta) { + TarantoolRequest request = super.toTarantoolRequest(schemaMeta); + request.addArguments( + RequestArguments.value(Key.SQL_TEXT), + RequestArguments.value(sqlText) + ); + if (!ordinalBindings.isEmpty()) { + request.addArguments( + RequestArguments.value(Key.SQL_BIND), + RequestArguments.value(ordinalBindings) + ); + } + if (!namedBindings.isEmpty()) { + request.addArguments( + RequestArguments.value(Key.SQL_BIND), + RequestArguments.value(namedBindings) + ); + } + return request; + } + +} diff --git a/src/main/java/org/tarantool/dsl/InsertOrReplaceRequestSpec.java b/src/main/java/org/tarantool/dsl/InsertOrReplaceRequestSpec.java new file mode 100644 index 00000000..35cb59c3 --- /dev/null +++ b/src/main/java/org/tarantool/dsl/InsertOrReplaceRequestSpec.java @@ -0,0 +1,81 @@ +package org.tarantool.dsl; + +import static org.tarantool.RequestArguments.cacheLookupValue; +import static org.tarantool.RequestArguments.value; + +import org.tarantool.Code; +import org.tarantool.Key; +import org.tarantool.TarantoolRequest; +import org.tarantool.schema.TarantoolSchemaMeta; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +public class InsertOrReplaceRequestSpec extends SpaceRequestSpec<InsertOrReplaceRequestSpec> { + + public enum Mode { + INSERT(Code.INSERT), + REPLACE(Code.REPLACE); + + final Code code; + + Mode(Code code) { + this.code = code; + } + + public Code getCode() { + return code; + } + } + + private List<Object> tuple; + + InsertOrReplaceRequestSpec(Mode mode, int spaceId, List<?> tuple) { + super(mode.getCode(), spaceId); + this.tuple = new ArrayList<>(tuple); + } + + InsertOrReplaceRequestSpec(Mode mode, String spaceName, List<?> tuple) { + super(mode.getCode(), spaceName); + this.tuple = new ArrayList<>(tuple); + } + + InsertOrReplaceRequestSpec(Mode mode, int spaceId, Object... tupleItems) { + super(mode.getCode(), spaceId); + this.tuple = Arrays.asList(tupleItems); + } + + InsertOrReplaceRequestSpec(Mode mode, String spaceName, Object... tupleItems) { + super(mode.getCode(), spaceName); + this.tuple = Arrays.asList(tupleItems); + } + + public InsertOrReplaceRequestSpec tuple(Object... tupleItems) { + this.tuple.clear(); + Collections.addAll(this.tuple, tupleItems); + return this; + } + + public InsertOrReplaceRequestSpec tuple(Collection<?> tuple) { + this.tuple.clear(); + this.tuple.addAll(tuple); + return this; + } + + @Override + public TarantoolRequest toTarantoolRequest(TarantoolSchemaMeta schemaMeta) { + TarantoolRequest request = super.toTarantoolRequest(schemaMeta); + request.addArguments( + value(Key.SPACE), + spaceId == null + ? cacheLookupValue(() -> schemaMeta.getSpace(spaceName).getId()) + : value(spaceId), + value(Key.TUPLE), value(tuple) + ); + return request; + } + +} diff --git a/src/main/java/org/tarantool/dsl/Operation.java b/src/main/java/org/tarantool/dsl/Operation.java new file mode 100644 index 00000000..aae09a6f --- /dev/null +++ b/src/main/java/org/tarantool/dsl/Operation.java @@ -0,0 +1,65 @@ +package org.tarantool.dsl; + +import java.util.Arrays; +import java.util.List; + +public class Operation { + + private final Operator operator; + private final List<Object> operands; + + public Operation(Operator operator, Object... operands) { + this.operator = operator; + this.operands = Arrays.asList(operands); + } + + public Operator getOperator() { + return operator; + } + + public Object[] toArray() { + Object[] array = new Object[operands.size() + 1]; + array[0] = operator.getOpCode(); + for (int i = 1; i < array.length; i++) { + array[i] = operands.get(i - 1); + } + return array; + } + + /** + * It's used to perform a transformation between raw type + * and type safe DSL Operation class. This is required + * because of being compatible with old operations interface + * and a new DSL approach. + * + * This client expects an operation in format of simple + * array or list like {opCode, args...}. For instance, + * addition 3 to second field will be {"+", 2, 3} + * + * @param operation raw operation + * + * @return type safe operation + */ + public static Operation fromArray(Object operation) { + try { + if (operation instanceof Object[]) { + Object[] opArray = (Object[]) operation; + String code = opArray[0].toString(); + Object[] args = new Object[opArray.length - 1]; + System.arraycopy(opArray, 1, args, 0, args.length); + return new Operation(Operator.byOpCode(code), args); + } + List<?> opList = (List<?>) operation; + String code = opList.get(0).toString(); + Object[] args = opList.subList(1, opList.size()).toArray(); + return new Operation(Operator.byOpCode(code), args); + } catch (Exception cause) { + throw new IllegalArgumentException( + "Operation is invalid. Use an array or list as {\"opCode\", args...}. " + + "Or use request DSL to build type safe operation.", + cause + ); + } + } + +} diff --git a/src/main/java/org/tarantool/dsl/Operations.java b/src/main/java/org/tarantool/dsl/Operations.java new file mode 100644 index 00000000..429dd6d8 --- /dev/null +++ b/src/main/java/org/tarantool/dsl/Operations.java @@ -0,0 +1,48 @@ +package org.tarantool.dsl; + +import java.util.Objects; + +public class Operations { + + public static Operation add(int fieldNumber, long value) { + return new Operation(Operator.ADDITION, fieldNumber, value); + } + + public static Operation subtract(int fieldNumber, long value) { + return new Operation(Operator.SUBTRACTION, fieldNumber, value); + } + + public static Operation bitwiseAnd(int fieldNumber, long value) { + return new Operation(Operator.BITWISE_AND, fieldNumber, value); + } + + public static Operation bitwiseOr(int fieldNumber, long value) { + return new Operation(Operator.BITWISE_OR, fieldNumber, value); + } + + public static Operation bitwiseXor(int fieldNumber, long value) { + return new Operation(Operator.BITWISE_XOR, fieldNumber, value); + } + + public static Operation splice(int fieldNumber, int position, int offset, String substitution) { + return new Operation(Operator.SPLICE, fieldNumber, position, offset, substitution); + } + + public static Operation insert(int fieldNumber, Object value) { + return new Operation(Operator.INSERT, fieldNumber, value); + } + + public static Operation delete(int fromField, int length) { + return new Operation(Operator.DELETE, fromField, length); + } + + public static Operation assign(int fieldNumber, Object value) { + return new Operation(Operator.ASSIGN, fieldNumber, value); + } + + private static Operation createOperation(Operator operator, int fieldNumber, Object value) { + Objects.requireNonNull(value); + return new Operation(operator, fieldNumber, value); + } + +} diff --git a/src/main/java/org/tarantool/dsl/Operator.java b/src/main/java/org/tarantool/dsl/Operator.java new file mode 100644 index 00000000..ae69e0c9 --- /dev/null +++ b/src/main/java/org/tarantool/dsl/Operator.java @@ -0,0 +1,33 @@ +package org.tarantool.dsl; + +import java.util.stream.Stream; + +public enum Operator { + ADDITION("+"), + SUBTRACTION("-"), + BITWISE_AND("&"), + BITWISE_OR("|"), + BITWISE_XOR("^"), + SPLICE(":"), + INSERT("!"), + DELETE("#"), + ASSIGN("="); + + private final String opCode; + + Operator(String opCode) { + this.opCode = opCode; + } + + public String getOpCode() { + return opCode; + } + + public static Operator byOpCode(String opCode) { + return Stream.of(Operator.values()) + .filter(s -> s.getOpCode().equals(opCode)) + .findFirst() + .orElseThrow(IllegalArgumentException::new); + } + +} diff --git a/src/main/java/org/tarantool/dsl/PingRequestSpec.java b/src/main/java/org/tarantool/dsl/PingRequestSpec.java new file mode 100644 index 00000000..3449edad --- /dev/null +++ b/src/main/java/org/tarantool/dsl/PingRequestSpec.java @@ -0,0 +1,11 @@ +package org.tarantool.dsl; + +import org.tarantool.Code; + +public class PingRequestSpec extends AbstractRequestSpec<PingRequestSpec> { + + PingRequestSpec() { + super(Code.PING); + } + +} diff --git a/src/main/java/org/tarantool/dsl/Requests.java b/src/main/java/org/tarantool/dsl/Requests.java new file mode 100644 index 00000000..2b0ca763 --- /dev/null +++ b/src/main/java/org/tarantool/dsl/Requests.java @@ -0,0 +1,109 @@ +package org.tarantool.dsl; + +import org.tarantool.dsl.InsertOrReplaceRequestSpec.Mode; + +import java.util.List; + +/** + * Entry point to build requests + * using DSL approach a.k.a Request DSL. + */ +public class Requests { + + public static SelectRequestSpec selectRequest(int space, int index) { + return new SelectRequestSpec(space, index); + } + + public static SelectRequestSpec selectRequest(String space, int index) { + return new SelectRequestSpec(space, index); + } + + public static SelectRequestSpec selectRequest(int space, String index) { + return new SelectRequestSpec(space, index); + } + + public static SelectRequestSpec selectRequest(String space, String index) { + return new SelectRequestSpec(space, index); + } + + public static InsertOrReplaceRequestSpec insertRequest(int space, List<?> tuple) { + return new InsertOrReplaceRequestSpec(Mode.INSERT, space, tuple); + } + + public static InsertOrReplaceRequestSpec insertRequest(int space, Object... tupleItems) { + return new InsertOrReplaceRequestSpec(Mode.INSERT, space, tupleItems); + } + + public static InsertOrReplaceRequestSpec insertRequest(String space, List<?> tuple) { + return new InsertOrReplaceRequestSpec(Mode.INSERT, space, tuple); + } + + public static InsertOrReplaceRequestSpec insertRequest(String space, Object... tupleItems) { + return new InsertOrReplaceRequestSpec(Mode.INSERT, space, tupleItems); + } + + public static InsertOrReplaceRequestSpec replaceRequest(int space, List<?> tuple) { + return new InsertOrReplaceRequestSpec(Mode.REPLACE, space, tuple); + } + + public static InsertOrReplaceRequestSpec replaceRequest(int space, Object... tupleItems) { + return new InsertOrReplaceRequestSpec(Mode.REPLACE, space, tupleItems); + } + + public static InsertOrReplaceRequestSpec replaceRequest(String space, List<?> tuple) { + return new InsertOrReplaceRequestSpec(Mode.REPLACE, space, tuple); + } + + public static InsertOrReplaceRequestSpec replaceRequest(String space, Object... tupleItems) { + return new InsertOrReplaceRequestSpec(Mode.REPLACE, space, tupleItems); + } + + public static UpdateRequestSpec updateRequest(int space, List<?> key, Operation... operations) { + return new UpdateRequestSpec(space, key, operations); + } + + public static UpdateRequestSpec updateRequest(String space, List<?> key, Operation... operations) { + return new UpdateRequestSpec(space, key, operations); + } + + public static UpsertRequestSpec upsertRequest(int space, List<?> key, List<?> tuple, Operation... operations) { + return new UpsertRequestSpec(space, key, tuple, operations); + } + + public static UpsertRequestSpec upsertRequest(String space, List<?> key, List<?> tuple, Operation... operations) { + return new UpsertRequestSpec(space, key, tuple, operations); + } + + public static DeleteRequestSpec deleteRequest(int space, List<?> key) { + return new DeleteRequestSpec(space, key); + } + + public static DeleteRequestSpec deleteRequest(int space, Object... keyParts) { + return new DeleteRequestSpec(space, keyParts); + } + + public static DeleteRequestSpec deleteRequest(String space, List<?> key) { + return new DeleteRequestSpec(space, key); + } + + public static DeleteRequestSpec deleteRequest(String space, Object... keyParts) { + return new DeleteRequestSpec(space, keyParts); + } + + public static CallRequestSpec callRequest(String function) { + return new CallRequestSpec(function); + } + + public static EvalRequestSpec evalRequest(String expression) { + return new EvalRequestSpec(expression); + } + + public static PingRequestSpec pingRequest() { + return new PingRequestSpec(); + } + + public static ExecuteRequestSpec executeRequest(String sql) { + return new ExecuteRequestSpec(sql); + } + +} diff --git a/src/main/java/org/tarantool/dsl/SelectRequestSpec.java b/src/main/java/org/tarantool/dsl/SelectRequestSpec.java new file mode 100644 index 00000000..1d3df5e2 --- /dev/null +++ b/src/main/java/org/tarantool/dsl/SelectRequestSpec.java @@ -0,0 +1,111 @@ +package org.tarantool.dsl; + +import static org.tarantool.RequestArguments.cacheLookupValue; +import static org.tarantool.RequestArguments.value; + +import org.tarantool.Code; +import org.tarantool.Iterator; +import org.tarantool.Key; +import org.tarantool.TarantoolRequest; +import org.tarantool.schema.TarantoolSchemaMeta; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +public class SelectRequestSpec extends SpaceRequestSpec<SelectRequestSpec> { + + private Integer indexId; + private String indexName; + private List<Object> key = new ArrayList<>(); + private Iterator iterator = Iterator.ALL; + private int offset = 0; + private int limit = Integer.MAX_VALUE; + + public SelectRequestSpec(int spaceId, int indexId) { + super(Code.SELECT, spaceId); + this.indexId = indexId; + } + + public SelectRequestSpec(int spaceId, String indexName) { + super(Code.SELECT, spaceId); + this.indexName = Objects.requireNonNull(indexName); + } + + public SelectRequestSpec(String spaceName, int indexId) { + super(Code.SELECT, spaceName); + this.indexId = indexId; + } + + public SelectRequestSpec(String spaceName, String indexName) { + super(Code.SELECT, spaceName); + this.indexName = Objects.requireNonNull(indexName); + } + + public SelectRequestSpec index(int indexId) { + this.indexId = indexId; + this.indexName = null; + return this; + } + + public SelectRequestSpec index(String indexName) { + this.indexName = Objects.requireNonNull(indexName); + this.indexId = null; + return this; + } + + public SelectRequestSpec key(Object... keyParts) { + this.key.clear(); + Collections.addAll(this.key, keyParts); + return this; + } + + public SelectRequestSpec key(Collection<?> key) { + this.key.clear(); + this.key.addAll(key); + return this; + } + + public SelectRequestSpec iterator(Iterator iterator) { + this.iterator = iterator; + return this; + } + + public SelectRequestSpec iterator(int iterator) { + this.iterator = Iterator.valueOf(iterator); + return this; + } + + public SelectRequestSpec offset(int offset) { + this.offset = offset; + return this; + } + + public SelectRequestSpec limit(int limit) { + this.limit = limit; + return this; + } + + @Override + public TarantoolRequest toTarantoolRequest(TarantoolSchemaMeta schemaMeta) { + TarantoolRequest request = super.toTarantoolRequest(schemaMeta); + request.addArguments( + value(Key.SPACE), + spaceId == null + ? cacheLookupValue(() -> schemaMeta.getSpace(spaceName).getId()) + : value(spaceId), + value(Key.INDEX), + indexId == null + ? cacheLookupValue(() -> schemaMeta.getSpaceIndex(spaceName, indexName).getId()) + : value(indexId), + value(Key.KEY), value(key), + value(Key.ITERATOR), value(iterator.getValue()), + value(Key.LIMIT), value(limit), + value(Key.OFFSET), value(offset) + ); + return request; + } + +} diff --git a/src/main/java/org/tarantool/dsl/SpaceRequestSpec.java b/src/main/java/org/tarantool/dsl/SpaceRequestSpec.java new file mode 100644 index 00000000..fb03152d --- /dev/null +++ b/src/main/java/org/tarantool/dsl/SpaceRequestSpec.java @@ -0,0 +1,46 @@ +package org.tarantool.dsl; + +import org.tarantool.Code; + +import java.util.Objects; + +/** + * Supports space related DSL builders. + * + * @param <B> current build type + */ +public abstract class SpaceRequestSpec<B extends SpaceRequestSpec<B>> + extends AbstractRequestSpec<B> { + + Integer spaceId; + String spaceName; + + public SpaceRequestSpec(Code code) { + super(code); + } + + public SpaceRequestSpec(Code code, int spaceId) { + this(code); + this.spaceId = spaceId; + } + + public SpaceRequestSpec(Code code, String spaceName) { + this(code); + this.spaceName = Objects.requireNonNull(spaceName); + } + + @SuppressWarnings("unchecked") + public B space(int spaceId) { + this.spaceId = spaceId; + this.spaceName = null; + return (B) this; + } + + @SuppressWarnings("unchecked") + public B space(String spaceName) { + this.spaceName = Objects.requireNonNull(spaceName); + this.spaceId = null; + return (B) this; + } + +} diff --git a/src/main/java/org/tarantool/dsl/TarantoolRequestConvertible.java b/src/main/java/org/tarantool/dsl/TarantoolRequestConvertible.java new file mode 100644 index 00000000..d0ab37b2 --- /dev/null +++ b/src/main/java/org/tarantool/dsl/TarantoolRequestConvertible.java @@ -0,0 +1,21 @@ +package org.tarantool.dsl; + +import org.tarantool.TarantoolRequest; +import org.tarantool.schema.TarantoolSchemaMeta; + +/** + * Used to convert DSL builder to appropriate + * Tarantool requests. + * + * This interface is not a part of public API. + */ +public interface TarantoolRequestConvertible { + + /** + * Converts the target to {@link TarantoolRequest}. + * + * @return converted request + */ + TarantoolRequest toTarantoolRequest(TarantoolSchemaMeta schemaMeta); + +} diff --git a/src/main/java/org/tarantool/dsl/UpdateRequestSpec.java b/src/main/java/org/tarantool/dsl/UpdateRequestSpec.java new file mode 100644 index 00000000..eda8ebba --- /dev/null +++ b/src/main/java/org/tarantool/dsl/UpdateRequestSpec.java @@ -0,0 +1,73 @@ +package org.tarantool.dsl; + +import static org.tarantool.RequestArguments.cacheLookupValue; +import static org.tarantool.RequestArguments.value; + +import org.tarantool.Code; +import org.tarantool.Key; +import org.tarantool.TarantoolRequest; +import org.tarantool.schema.TarantoolSchemaMeta; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public class UpdateRequestSpec extends SpaceRequestSpec<UpdateRequestSpec> { + + private List<Object> key; + private List<Operation> operations; + + public UpdateRequestSpec(int spaceId, List<?> key, Operation... operations) { + super(Code.UPDATE, spaceId); + this.key = new ArrayList<>(key); + this.operations = Arrays.asList(operations); + } + + public UpdateRequestSpec(String spaceName, List<?> key, Operation... operations) { + super(Code.UPDATE, spaceName); + this.key = new ArrayList<>(key); + this.operations = Arrays.asList(operations); + } + + public UpdateRequestSpec primaryKey(Object... keyParts) { + this.key.clear(); + Collections.addAll(this.key, keyParts); + return this; + } + + public UpdateRequestSpec primaryKey(Collection<?> key) { + this.key.clear(); + this.key.addAll(key); + return this; + } + + public UpdateRequestSpec operations(Operation... operations) { + this.operations.clear(); + Collections.addAll(this.operations, operations); + return this; + } + + public UpdateRequestSpec operations(Collection<? extends Operation> operations) { + this.operations.clear(); + this.operations.addAll(operations); + return this; + } + + @Override + public TarantoolRequest toTarantoolRequest(TarantoolSchemaMeta schemaMeta) { + TarantoolRequest request = super.toTarantoolRequest(schemaMeta); + request.addArguments( + value(Key.SPACE), + spaceId == null + ? cacheLookupValue(() -> schemaMeta.getSpace(spaceName).getId()) + : value(spaceId), + value(Key.KEY), value(key), + value(Key.TUPLE), value(operations.stream().map(Operation::toArray).collect(Collectors.toList())) + ); + return request; + } + +} diff --git a/src/main/java/org/tarantool/dsl/UpsertRequestSpec.java b/src/main/java/org/tarantool/dsl/UpsertRequestSpec.java new file mode 100644 index 00000000..c53fb5a4 --- /dev/null +++ b/src/main/java/org/tarantool/dsl/UpsertRequestSpec.java @@ -0,0 +1,88 @@ +package org.tarantool.dsl; + +import static org.tarantool.RequestArguments.cacheLookupValue; +import static org.tarantool.RequestArguments.value; + +import org.tarantool.Code; +import org.tarantool.Key; +import org.tarantool.TarantoolRequest; +import org.tarantool.schema.TarantoolSchemaMeta; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public class UpsertRequestSpec extends SpaceRequestSpec<UpsertRequestSpec> { + + private List<Object> key; + private List<Object> tuple; + private List<Operation> operations; + + public UpsertRequestSpec(int spaceId, List<?> key, List<?> tuple, Operation... operations) { + super(Code.UPSERT, spaceId); + this.key = new ArrayList<>(key); + this.tuple = new ArrayList<>(tuple); + this.operations = Arrays.asList(operations); + } + + public UpsertRequestSpec(String spaceName, List<?> key, List<?> tuple, Operation... operations) { + super(Code.UPSERT, spaceName); + this.key = new ArrayList<>(key); + this.tuple = new ArrayList<>(tuple); + this.operations = Arrays.asList(operations); + } + + public UpsertRequestSpec primaryKey(Object... keyParts) { + this.key.clear(); + Collections.addAll(this.key, keyParts); + return this; + } + + public UpsertRequestSpec primaryKey(Collection<?> key) { + this.key.clear(); + this.key.addAll(key); + return this; + } + + public UpsertRequestSpec tuple(Collection<?> tuple) { + this.tuple.clear(); + this.tuple.addAll(tuple); + return this; + } + + public UpsertRequestSpec tuple(Object... tupleItems) { + this.tuple.clear(); + Collections.addAll(this.tuple, tupleItems); + return this; + } + + public UpsertRequestSpec operations(Collection<? extends Operation> operations) { + this.operations.clear(); + this.operations.addAll(operations); + return this; + } + + public UpsertRequestSpec operations(Operation... operations) { + this.operations.clear(); + Collections.addAll(this.operations, operations); + return this; + } + + @Override + public TarantoolRequest toTarantoolRequest(TarantoolSchemaMeta schemaMeta) { + TarantoolRequest request = super.toTarantoolRequest(schemaMeta); + request.addArguments( + value(Key.SPACE), + spaceId == null + ? cacheLookupValue(() -> schemaMeta.getSpace(spaceName).getId()) + : value(spaceId), + value(Key.KEY), value(key), + value(Key.TUPLE), value(tuple), + value(Key.UPSERT_OPS), value(operations.stream().map(Operation::toArray).collect(Collectors.toList())) + ); + return request; + } +} diff --git a/src/main/java/org/tarantool/jdbc/SQLConnection.java b/src/main/java/org/tarantool/jdbc/SQLConnection.java index 15f3258a..8b08656f 100644 --- a/src/main/java/org/tarantool/jdbc/SQLConnection.java +++ b/src/main/java/org/tarantool/jdbc/SQLConnection.java @@ -1,12 +1,14 @@ package org.tarantool.jdbc; -import org.tarantool.Code; +import static org.tarantool.dsl.Requests.executeRequest; + import org.tarantool.CommunicationException; -import org.tarantool.Key; import org.tarantool.SocketChannelProvider; import org.tarantool.SqlProtoUtils; import org.tarantool.TarantoolClientConfig; import org.tarantool.TarantoolClientImpl; +import org.tarantool.TarantoolOp; +import org.tarantool.TarantoolRequest; import org.tarantool.protocol.TarantoolPacket; import org.tarantool.util.JdbcConstants; import org.tarantool.util.SQLStates; @@ -32,6 +34,8 @@ import java.sql.Savepoint; import java.sql.Statement; import java.sql.Struct; +import java.time.Duration; +import java.time.temporal.ChronoUnit; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -538,8 +542,8 @@ public SQLBatchResultHolder executeBatch(long timeout, List<SQLQueryHolder> quer checkNotClosed(); SQLTarantoolClientImpl.SQLRawOps sqlOps = client.sqlRawOps(); SQLBatchResultHolder batchResult = useNetworkTimeout(timeout) - ? sqlOps.executeBatch(queries) - : sqlOps.executeBatch(timeout, queries); + ? sqlOps.executeBatch(queries) + : sqlOps.executeBatch(timeout, queries); return batchResult; } @@ -734,12 +738,14 @@ private static String formatError(SQLQueryHolder query) { static class SQLTarantoolClientImpl extends TarantoolClientImpl { private Future<?> executeQuery(SQLQueryHolder queryHolder) { - return exec(Code.EXECUTE, Key.SQL_TEXT, queryHolder.getQuery(), Key.SQL_BIND, queryHolder.getParams()); + return execute(executeRequest(queryHolder.getQuery()).ordinalParameters(queryHolder.getParams())); } private Future<?> executeQuery(SQLQueryHolder queryHolder, long timeoutMillis) { - return exec( - timeoutMillis, Code.EXECUTE, Key.SQL_TEXT, queryHolder.getQuery(), Key.SQL_BIND, queryHolder.getParams() + return execute( + executeRequest(queryHolder.getQuery()) + .ordinalParameters(queryHolder.getParams()) + .timeout(Duration.of(timeoutMillis, ChronoUnit.MILLIS)) ); } @@ -809,12 +815,12 @@ SQLRawOps sqlRawOps() { } @Override - protected void completeSql(TarantoolOp<?> future, TarantoolPacket pack) { + protected void completeSql(TarantoolRequest request, TarantoolPacket pack) { Long rowCount = SqlProtoUtils.getSQLRowCount(pack); SQLResultHolder result = (rowCount == null) ? SQLResultHolder.ofQuery(SqlProtoUtils.getSQLMetadata(pack), SqlProtoUtils.getSQLData(pack)) : SQLResultHolder.ofUpdate(rowCount.intValue(), SqlProtoUtils.getSQLAutoIncrementIds(pack)); - ((TarantoolOp) future).complete(result); + ((TarantoolOp) request.getResult()).complete(result); } interface SQLRawOps { diff --git a/src/main/java/org/tarantool/protocol/ProtoConstants.java b/src/main/java/org/tarantool/protocol/ProtoConstants.java new file mode 100644 index 00000000..daee4dd7 --- /dev/null +++ b/src/main/java/org/tarantool/protocol/ProtoConstants.java @@ -0,0 +1,19 @@ +package org.tarantool.protocol; + +public class ProtoConstants { + + private ProtoConstants() { + } + + public static final long ERROR_TYPE_MARKER = 0x8000; + + public static final long SUCCESS = 0x0; + + /* taken from src/box/errcode.h */ + public static final int ERR_READONLY = 7; + public static final int ERR_TIMEOUT = 78; + public static final int ERR_WRONG_SCHEMA_VERSION = 109; + public static final int ERR_LOADING = 116; + public static final int ERR_LOCAL_INSTANCE_ID_IS_READ_ONLY = 128; + +} diff --git a/src/main/java/org/tarantool/protocol/ProtoUtils.java b/src/main/java/org/tarantool/protocol/ProtoUtils.java index ec3283b7..7fcd21b0 100644 --- a/src/main/java/org/tarantool/protocol/ProtoUtils.java +++ b/src/main/java/org/tarantool/protocol/ProtoUtils.java @@ -209,7 +209,7 @@ private static void assertCorrectWelcome(String firstLine, SocketAddress remoteA private static void assertNoErrCode(TarantoolPacket authResponse) { Long code = (Long) authResponse.getHeaders().get(Key.CODE.getId()); if (code != 0) { - Object error = authResponse.getBody().get(Key.ERROR.getId()); + Object error = authResponse.getError(); String errorMsg = error instanceof String ? (String) error : new String((byte[]) error); throw new TarantoolException(code, errorMsg); } @@ -301,7 +301,22 @@ public static ByteBuffer createPacket(int initialRequestSize, return buffer; } + /** + * Extracts an error code. + * + * @param code in 0x8XXX format + * + * @return actual error code (which is a XXX part) + */ + public static long extractErrorCode(long code) { + if ((code & ProtoConstants.ERROR_TYPE_MARKER) == 0) { + throw new IllegalArgumentException(String.format("Code %h does not follow 0x8XXX format", code)); + } + return (~ProtoConstants.ERROR_TYPE_MARKER & code); + } + private static class ByteArrayOutputStream extends java.io.ByteArrayOutputStream { + public ByteArrayOutputStream(int size) { super(size); } @@ -309,6 +324,7 @@ public ByteArrayOutputStream(int size) { ByteBuffer toByteBuffer() { return ByteBuffer.wrap(buf, 0, count); } + } } diff --git a/src/main/java/org/tarantool/protocol/TarantoolPacket.java b/src/main/java/org/tarantool/protocol/TarantoolPacket.java index ca131177..ca661ed9 100644 --- a/src/main/java/org/tarantool/protocol/TarantoolPacket.java +++ b/src/main/java/org/tarantool/protocol/TarantoolPacket.java @@ -29,8 +29,9 @@ public Long getCode() { potenticalCode != null ? potenticalCode.getClass().toString() : "null" ); } + Long code = (Long) potenticalCode; - return (Long) potenticalCode; + return code == 0 ? code : ProtoUtils.extractErrorCode(code); } public Long getSync() { @@ -48,4 +49,16 @@ public Map<Integer, Object> getBody() { public boolean hasBody() { return body != null && body.size() > 0; } + + public long getSchemaId() { + return (Long) headers.get(Key.SCHEMA_ID.getId()); + } + + public Object getData() { + return hasBody() ? body.get(Key.DATA.getId()) : null; + } + + public Object getError() { + return hasBody() ? body.get(Key.ERROR.getId()) : null; + } } diff --git a/src/main/java/org/tarantool/schema/TarantoolIndexMeta.java b/src/main/java/org/tarantool/schema/TarantoolIndexMeta.java new file mode 100644 index 00000000..bcdd8ab1 --- /dev/null +++ b/src/main/java/org/tarantool/schema/TarantoolIndexMeta.java @@ -0,0 +1,128 @@ +package org.tarantool.schema; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Keeps a space index metadata. + */ +public class TarantoolIndexMeta { + + public static final int VINDEX_IID_FIELD_NUMBER = 1; + public static final int VINDEX_NAME_FIELD_NUMBER = 2; + public static final int VINDEX_TYPE_FIELD_NUMBER = 3; + public static final int VINDEX_OPTIONS_FIELD_NUMBER = 4; + public static final int VINDEX_PARTS_FIELD_NUMBER = 5; + + public static final int VINDEX_PART_FIELD = 0; + public static final int VINDEX_PART_TYPE = 1; + + private final int id; + private final String name; + private final String type; + private final IndexOptions options; + private final List<IndexPart> parts; + + public TarantoolIndexMeta(int id, + String name, + String type, + IndexOptions options, + List<IndexPart> parts) { + this.id = id; + this.name = name; + this.type = type; + this.options = options; + this.parts = parts; + } + + public static TarantoolIndexMeta fromTuple(List<?> tuple) { + Map<String, Object> optionsMap = (Map<String, Object>) tuple.get(VINDEX_OPTIONS_FIELD_NUMBER); + + List<IndexPart> parts = Collections.emptyList(); + List<?> partsTuple = (List<?>) tuple.get(VINDEX_PARTS_FIELD_NUMBER); + if (!partsTuple.isEmpty()) { + // in case of raw index each part is an array + // in case of sql generated index each part is a map + if (partsTuple.get(0) instanceof List) { + parts = ((List<List<?>>) partsTuple) + .stream() + .map(part -> new IndexPart( + (Integer) part.get(VINDEX_PART_FIELD), + (String) part.get(VINDEX_PART_TYPE) + ) + ) + .collect(Collectors.toList()); + } else if (partsTuple.get(0) instanceof Map) { + parts = ((List<Map<String, Object>>) partsTuple) + .stream() + .map(part -> new IndexPart((Integer) part.get("field"), (String) part.get("type"))) + .collect(Collectors.toList()); + } + } + + return new TarantoolIndexMeta( + (Integer) tuple.get(VINDEX_IID_FIELD_NUMBER), + (String) tuple.get(VINDEX_NAME_FIELD_NUMBER), + (String) tuple.get(VINDEX_TYPE_FIELD_NUMBER), + new IndexOptions((Boolean) optionsMap.get("unique")), + parts + ); + } + + public int getId() { + return id; + } + + public String getName() { + return name; + } + + public String getType() { + return type; + } + + public IndexOptions getOptions() { + return options; + } + + public List<IndexPart> getParts() { + return parts; + } + + public static class IndexOptions { + + private final boolean unique; + + public IndexOptions(boolean unique) { + this.unique = unique; + } + + public boolean isUnique() { + return unique; + } + + } + + public static class IndexPart { + + private final int fieldNumber; + private final String type; + + public IndexPart(int fieldNumber, String type) { + this.fieldNumber = fieldNumber; + this.type = type; + } + + public int getFieldNumber() { + return fieldNumber; + } + + public String getType() { + return type; + } + + } + +} diff --git a/src/main/java/org/tarantool/schema/TarantoolIndexNotFoundException.java b/src/main/java/org/tarantool/schema/TarantoolIndexNotFoundException.java new file mode 100644 index 00000000..49406fae --- /dev/null +++ b/src/main/java/org/tarantool/schema/TarantoolIndexNotFoundException.java @@ -0,0 +1,16 @@ +package org.tarantool.schema; + +public class TarantoolIndexNotFoundException extends TarantoolSchemaException { + + private final String indexName; + + public TarantoolIndexNotFoundException(String targetSpace, String indexName) { + super(targetSpace); + this.indexName = indexName; + } + + public String getIndexName() { + return indexName; + } + +} diff --git a/src/main/java/org/tarantool/schema/TarantoolMetaSpacesCache.java b/src/main/java/org/tarantool/schema/TarantoolMetaSpacesCache.java new file mode 100644 index 00000000..d769baed --- /dev/null +++ b/src/main/java/org/tarantool/schema/TarantoolMetaSpacesCache.java @@ -0,0 +1,116 @@ +package org.tarantool.schema; + +import static org.tarantool.dsl.Requests.selectRequest; + +import org.tarantool.TarantoolClientImpl; +import org.tarantool.TarantoolClientOps; +import org.tarantool.util.TupleTwo; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * In-memory schema cache. + * <p> + * Caches meta spaces {@code _vspace} and {@code _vindex}. + * <p> + * This class is not a part of public API. + */ +public class TarantoolMetaSpacesCache implements TarantoolSchemaMeta { + + private static final int VSPACE_ID = 281; + private static final int VSPACE_ID_INDEX_ID = 0; + + private static final int VINDEX_ID = 289; + private static final int VINDEX_ID_INDEX_ID = 0; + + /** + * Describes the theoretical maximum tuple size + * which is (2^31 - 1) (box.schema.SPACE_MAX) + */ + private static final int MAX_TUPLES = 2_147_483_647; + + private TarantoolClientImpl client; + + private volatile Map<String, TarantoolSpaceMeta> cachedSpaces = new ConcurrentHashMap<>(); + private volatile long schemaVersion; + + public TarantoolMetaSpacesCache(TarantoolClientImpl client) { + this.client = client; + } + + @Override + public TarantoolSpaceMeta getSpace(String spaceName) { + TarantoolSpaceMeta space = cachedSpaces.get(spaceName); + if (space == null) { + throw new TarantoolSpaceNotFoundException(spaceName); + } + return space; + } + + @Override + public TarantoolIndexMeta getSpaceIndex(String spaceName, String indexName) { + TarantoolIndexMeta index = getSpace(spaceName).getIndex(indexName); + if (index == null) { + throw new TarantoolIndexNotFoundException(spaceName, indexName); + } + return index; + } + + @Override + public long getSchemaVersion() { + return schemaVersion; + } + + @Override + public synchronized long refresh() { + TupleTwo<List<TarantoolSpaceMeta>, Long> result = fetchSpaces(); + cachedSpaces = result.getFirst() + .stream() + .collect( + Collectors.toConcurrentMap( + TarantoolSpaceMeta::getName, + Function.identity(), + (oldValue, newValue) -> newValue, + ConcurrentHashMap::new + ) + ); + return schemaVersion = result.getSecond(); + } + + private TupleTwo<List<TarantoolSpaceMeta>, Long> fetchSpaces() { + TarantoolClientOps<List<?>, Object, TupleTwo<List<?>, Long>> clientOps = client.unsafeSchemaOps(); + + long firstRequestSchema = -1; + long secondRequestSchema = 0; + List<?> spaces = null; + List<?> indexes = null; + while (firstRequestSchema != secondRequestSchema) { + TupleTwo<List<?>, Long> spacesResult = clientOps.execute(selectRequest(VSPACE_ID, VSPACE_ID_INDEX_ID)); + TupleTwo<List<?>, Long> indexesResult = clientOps.execute(selectRequest(VINDEX_ID, VINDEX_ID_INDEX_ID)); + spaces = spacesResult.getFirst(); + indexes = indexesResult.getFirst(); + firstRequestSchema = spacesResult.getSecond(); + secondRequestSchema = indexesResult.getSecond(); + } + + Map<Integer, List<List<?>>> indexesBySpace = indexes.stream() + .map(tuple -> (List<?>) tuple) + .collect(Collectors.groupingBy(tuple -> (Integer) tuple.get(0))); + + List<TarantoolSpaceMeta> cachedMeta = spaces.stream() + .map(tuple -> (List<?>) tuple) + .map(tuple -> TarantoolSpaceMeta.fromTuple( + tuple, + indexesBySpace.getOrDefault((Integer) tuple.get(0), Collections.emptyList())) + ) + .collect(Collectors.toList()); + + return TupleTwo.of(cachedMeta, firstRequestSchema); + } + +} diff --git a/src/main/java/org/tarantool/schema/TarantoolSchemaException.java b/src/main/java/org/tarantool/schema/TarantoolSchemaException.java new file mode 100644 index 00000000..877f6c6b --- /dev/null +++ b/src/main/java/org/tarantool/schema/TarantoolSchemaException.java @@ -0,0 +1,15 @@ +package org.tarantool.schema; + +public class TarantoolSchemaException extends RuntimeException { + + private final String schemaName; + + public TarantoolSchemaException(String schemaName) { + this.schemaName = schemaName; + } + + public String getSchemaName() { + return schemaName; + } + +} diff --git a/src/main/java/org/tarantool/schema/TarantoolSchemaMeta.java b/src/main/java/org/tarantool/schema/TarantoolSchemaMeta.java new file mode 100644 index 00000000..b4f7540e --- /dev/null +++ b/src/main/java/org/tarantool/schema/TarantoolSchemaMeta.java @@ -0,0 +1,41 @@ +package org.tarantool.schema; + +/** + * Provides Tarantool instance schema info. + */ +public interface TarantoolSchemaMeta { + + /** + * Finds a space by name if any. + * + * @param spaceName name of target space + * + * @return found space + */ + TarantoolSpaceMeta getSpace(String spaceName); + + /** + * Finds a space index by name if any. + * + * @param spaceName name of target space + * @param indexName name of target index + * + * @return found index meta + */ + TarantoolIndexMeta getSpaceIndex(String spaceName, String indexName); + + /** + * Gets current schema version that is cached. + * + * @return current version + */ + long getSchemaVersion(); + + /** + * Fetches schema metadata. + * + * @return fetched schema metadata version + */ + long refresh(); + +} diff --git a/src/main/java/org/tarantool/schema/TarantoolSpaceMeta.java b/src/main/java/org/tarantool/schema/TarantoolSpaceMeta.java new file mode 100644 index 00000000..cbbeb372 --- /dev/null +++ b/src/main/java/org/tarantool/schema/TarantoolSpaceMeta.java @@ -0,0 +1,99 @@ +package org.tarantool.schema; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Keeps a space metadata. + */ +public class TarantoolSpaceMeta { + + public static final int VSPACE_ID_FILED_NUMBER = 0; + public static final int VSPACE_NAME_FILED_NUMBER = 2; + public static final int VSPACE_ENGINE_FILED_NUMBER = 3; + public static final int VSPACE_FORMAT_FIELD_NUMBER = 6; + + private final int id; + private final String name; + private final String engine; + private final List<SpaceField> format; + private final Map<String, TarantoolIndexMeta> indexes; + + public static TarantoolSpaceMeta fromTuple(List<?> spaceTuple, List<List<?>> indexTuples) { + List<SpaceField> fields = ((List<Map<String, ?>>) spaceTuple.get(VSPACE_FORMAT_FIELD_NUMBER)).stream() + .map(field -> new SpaceField(field.get("name").toString(), field.get("type").toString())) + .collect(Collectors.toList()); + + Map<String, TarantoolIndexMeta> indexesMap = indexTuples.stream() + .map(TarantoolIndexMeta::fromTuple) + .collect(Collectors.toMap(TarantoolIndexMeta::getName, Function.identity())); + + return new TarantoolSpaceMeta( + (Integer) spaceTuple.get(VSPACE_ID_FILED_NUMBER), + spaceTuple.get(VSPACE_NAME_FILED_NUMBER).toString(), + spaceTuple.get(VSPACE_ENGINE_FILED_NUMBER).toString(), + Collections.unmodifiableList(fields), + Collections.unmodifiableMap(indexesMap) + ); + } + + public TarantoolSpaceMeta(int id, + String name, + String engine, + List<SpaceField> format, + Map<String, TarantoolIndexMeta> indexes) { + this.id = id; + this.name = name; + this.engine = engine; + this.format = format; + this.indexes = indexes; + } + + public int getId() { + return id; + } + + public String getName() { + return name; + } + + public String getEngine() { + return engine; + } + + public List<SpaceField> getFormat() { + return format; + } + + public Map<String, TarantoolIndexMeta> getIndexes() { + return indexes; + } + + public TarantoolIndexMeta getIndex(String indexName) { + return indexes.get(indexName); + } + + public static class SpaceField { + + private final String name; + private final String type; + + public SpaceField(String name, String type) { + this.name = name; + this.type = type; + } + + public String getName() { + return name; + } + + public String getType() { + return type; + } + + } + +} diff --git a/src/main/java/org/tarantool/schema/TarantoolSpaceNotFoundException.java b/src/main/java/org/tarantool/schema/TarantoolSpaceNotFoundException.java new file mode 100644 index 00000000..28498e6b --- /dev/null +++ b/src/main/java/org/tarantool/schema/TarantoolSpaceNotFoundException.java @@ -0,0 +1,9 @@ +package org.tarantool.schema; + +public class TarantoolSpaceNotFoundException extends TarantoolSchemaException { + + public TarantoolSpaceNotFoundException(String spaceName) { + super(spaceName); + } + +} diff --git a/src/main/java/org/tarantool/util/TupleTwo.java b/src/main/java/org/tarantool/util/TupleTwo.java index b124e287..9bad7059 100644 --- a/src/main/java/org/tarantool/util/TupleTwo.java +++ b/src/main/java/org/tarantool/util/TupleTwo.java @@ -8,12 +8,12 @@ public class TupleTwo<T, U> { private final T first; private final U second; - private TupleTwo(T first, U second) { + TupleTwo(T first, U second) { this.first = first; this.second = second; } - public static <T, U> TupleTwo of(T first, U second) { + public static <T, U> TupleTwo<T, U> of(T first, U second) { return new TupleTwo<>(first, second); } diff --git a/src/test/java/org/tarantool/ClientAsyncOperationsIT.java b/src/test/java/org/tarantool/ClientAsyncOperationsIT.java index 087a4760..054967a6 100644 --- a/src/test/java/org/tarantool/ClientAsyncOperationsIT.java +++ b/src/test/java/org/tarantool/ClientAsyncOperationsIT.java @@ -7,6 +7,10 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.tarantool.TestAssertions.checkRawTupleResult; +import org.tarantool.dsl.TarantoolRequestConvertible; +import org.tarantool.schema.TarantoolIndexNotFoundException; +import org.tarantool.schema.TarantoolSpaceNotFoundException; + import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -27,7 +31,7 @@ /** * Class with test cases for asynchronous operations - * + * <p> * NOTE: Parametrized tests can be simplified after * https://github.com/junit-team/junit5/issues/878 */ @@ -120,7 +124,7 @@ void testAsyncError(AsyncOpsProvider provider) { @MethodSource("getAsyncOps") void testOperations(AsyncOpsProvider provider) throws ExecutionException, InterruptedException, TimeoutException { - TarantoolClientOps<Integer, List<?>, Object, Future<List<?>>> ops = provider.getAsyncOps(); + TarantoolClientOps<List<?>, Object, Future<List<?>>> ops = provider.getAsyncOps(); List<Future<List<?>>> futures = new ArrayList<>(); @@ -145,7 +149,7 @@ void testOperations(AsyncOpsProvider provider) // Check the effects. checkRawTupleResult(consoleSelect(10), Arrays.asList(10, "ten")); checkRawTupleResult(consoleSelect(20), Arrays.asList(20, "twenty")); - assertEquals(consoleSelect(30), Collections.emptyList()); + assertEquals(Collections.emptyList(), consoleSelect(30)); provider.close(); } @@ -185,16 +189,225 @@ void testCall(AsyncOpsProvider provider) throws ExecutionException, InterruptedE provider.close(); } + @ParameterizedTest + @MethodSource("getAsyncOps") + void testStringSelect(AsyncOpsProvider provider) throws ExecutionException, InterruptedException, TimeoutException { + testHelper.executeLua("box.space.basic_test:insert{1, 'one'}"); + Future<List<?>> result = provider.getAsyncOps() + .select("basic_test", "pk", Collections.singletonList(1), 0, 1, Iterator.EQ); + + assertEquals( + Collections.singletonList(Arrays.asList(1, "one")), + result.get(TIMEOUT, TimeUnit.MILLISECONDS) + ); + + provider.close(); + } + + @ParameterizedTest + @MethodSource("getAsyncOps") + void testStringInsert(AsyncOpsProvider provider) throws ExecutionException, InterruptedException, TimeoutException { + Future<List<?>> resultOne = provider.getAsyncOps() + .insert("basic_test", Arrays.asList(1, "one")); + + Future<List<?>> resultTen = provider.getAsyncOps() + .insert("basic_test", Arrays.asList(10, "ten")); + + resultOne.get(TIMEOUT, TimeUnit.MILLISECONDS); + resultTen.get(TIMEOUT, TimeUnit.MILLISECONDS); + + checkRawTupleResult(consoleSelect(1), Arrays.asList(1, "one")); + checkRawTupleResult(consoleSelect(10), Arrays.asList(10, "ten")); + + provider.close(); + } + + @ParameterizedTest + @MethodSource("getAsyncOps") + void testStringReplace(AsyncOpsProvider provider) + throws ExecutionException, InterruptedException, TimeoutException { + testHelper.executeLua("box.space.basic_test:insert{1, '1'}"); + testHelper.executeLua("box.space.basic_test:insert{10, '10'}"); + + Future<List<?>> resultOne = provider.getAsyncOps() + .replace("basic_test", Arrays.asList(1, "one")); + + Future<List<?>> resultTen = provider.getAsyncOps() + .replace("basic_test", Arrays.asList(10, "ten")); + + resultOne.get(TIMEOUT, TimeUnit.MILLISECONDS); + resultTen.get(TIMEOUT, TimeUnit.MILLISECONDS); + + checkRawTupleResult(consoleSelect(1), Arrays.asList(1, "one")); + checkRawTupleResult(consoleSelect(10), Arrays.asList(10, "ten")); + + provider.close(); + } + + @ParameterizedTest + @MethodSource("getAsyncOps") + void testStringDelete(AsyncOpsProvider provider) throws ExecutionException, InterruptedException, TimeoutException { + testHelper.executeLua("box.space.basic_test:insert{1, '1'}"); + testHelper.executeLua("box.space.basic_test:insert{10, '10'}"); + testHelper.executeLua("box.space.basic_test:insert{20, '20'}"); + + Future<List<?>> resultOne = provider.getAsyncOps() + .delete("basic_test", Collections.singletonList(1)); + + Future<List<?>> resultTen = provider.getAsyncOps() + .delete("basic_test", Collections.singletonList(20)); + + resultOne.get(TIMEOUT, TimeUnit.MILLISECONDS); + resultTen.get(TIMEOUT, TimeUnit.MILLISECONDS); + + assertEquals(Collections.emptyList(), consoleSelect(1)); + checkRawTupleResult(consoleSelect(10), Arrays.asList(10, "10")); + assertEquals(Collections.emptyList(), consoleSelect(20)); + + provider.close(); + } + + @ParameterizedTest + @MethodSource("getAsyncOps") + void testStringUpdate(AsyncOpsProvider provider) throws ExecutionException, InterruptedException, TimeoutException { + testHelper.executeLua("box.space.basic_test:insert{1, '1'}"); + testHelper.executeLua("box.space.basic_test:insert{10, '10'}"); + + Future<List<?>> resultOne = provider.getAsyncOps() + .update("basic_test", Collections.singletonList(1), Arrays.asList("=", 1, "one")); + + Future<List<?>> resultTwo = provider.getAsyncOps() + .update("basic_test", Collections.singletonList(2), Arrays.asList("=", 1, "two")); + + Future<List<?>> resultTen = provider.getAsyncOps() + .update("basic_test", Collections.singletonList(10), Arrays.asList("=", 1, "ten")); + + resultOne.get(TIMEOUT, TimeUnit.MILLISECONDS); + resultTwo.get(TIMEOUT, TimeUnit.MILLISECONDS); + resultTen.get(TIMEOUT, TimeUnit.MILLISECONDS); + + checkRawTupleResult(consoleSelect(1), Arrays.asList(1, "one")); + assertEquals(Collections.emptyList(), consoleSelect(2)); + checkRawTupleResult(consoleSelect(10), Arrays.asList(10, "ten")); + + provider.close(); + } + + @ParameterizedTest + @MethodSource("getAsyncOps") + void testStringUpsert(AsyncOpsProvider provider) throws ExecutionException, InterruptedException, TimeoutException { + testHelper.executeLua("box.space.basic_test:insert{1, '1'}"); + testHelper.executeLua("box.space.basic_test:insert{10, '10'}"); + + Future<List<?>> resultOne = provider.getAsyncOps() + .upsert("basic_test", Collections.singletonList(1), Arrays.asList(1, "001"), Arrays.asList("=", 1, "one")); + + Future<List<?>> resultTwo = provider.getAsyncOps() + .upsert("basic_test", Collections.singletonList(2), Arrays.asList(2, "002"), Arrays.asList("=", 1, "two")); + + Future<List<?>> resultTen = provider.getAsyncOps() + .upsert("basic_test", Collections.singletonList(10), Arrays.asList(10, "010"), + Arrays.asList("=", 1, "ten")); + + resultOne.get(TIMEOUT, TimeUnit.MILLISECONDS); + resultTwo.get(TIMEOUT, TimeUnit.MILLISECONDS); + resultTen.get(TIMEOUT, TimeUnit.MILLISECONDS); + + checkRawTupleResult(consoleSelect(1), Arrays.asList(1, "one")); + checkRawTupleResult(consoleSelect(2), Arrays.asList(2, "002")); + checkRawTupleResult(consoleSelect(10), Arrays.asList(10, "ten")); + + provider.close(); + } + + @ParameterizedTest + @MethodSource("getAsyncOps") + void testStringMultipleIndirectChanges(AsyncOpsProvider provider) + throws ExecutionException, InterruptedException, TimeoutException { + testHelper.executeLua("box.space.basic_test:insert{1, 'one'}"); + Future<List<?>> result = provider.getAsyncOps() + .select("basic_test", "pk", Collections.singletonList(1), 0, 1, Iterator.EQ); + + assertEquals( + Collections.singletonList(Arrays.asList(1, "one")), + result.get(TIMEOUT, TimeUnit.MILLISECONDS) + ); + + testHelper.executeLua("box.space.basic_test and box.space.basic_test:drop()"); + testHelper.executeLua( + "box.schema.space.create('basic_test', { format = " + + "{{name = 'id', type = 'integer'}," + + " {name = 'val', type = 'string'} } })", + + "box.space.basic_test:create_index('pk', { type = 'TREE', parts = {'id'} } )" + ); + testHelper.executeLua("box.space.basic_test:insert{2, 'two'}"); + + result = provider.getAsyncOps() + .select("basic_test", "pk", Collections.singletonList(2), 0, 1, Iterator.EQ); + + assertEquals( + Collections.singletonList(Arrays.asList(2, "two")), + result.get(TIMEOUT, TimeUnit.MILLISECONDS) + ); + + testHelper.executeLua("box.space.basic_test and box.space.basic_test:drop()"); + testHelper.executeLua( + "box.schema.space.create('basic_test', { format = " + + "{{name = 'id', type = 'integer'}," + + " {name = 'val', type = 'string'} } })", + + "box.space.basic_test:create_index('pk', { type = 'TREE', parts = {'id'} } )" + ); + testHelper.executeLua("box.space.basic_test:insert{3, 'three'}"); + + result = provider.getAsyncOps() + .select("basic_test", "pk", Collections.singletonList(3), 0, 1, Iterator.EQ); + + assertEquals( + Collections.singletonList(Arrays.asList(3, "three")), + result.get(TIMEOUT, TimeUnit.MILLISECONDS) + ); + + provider.close(); + } + + @ParameterizedTest + @MethodSource("getAsyncOps") + void testUnknownSpace(AsyncOpsProvider provider) throws ExecutionException, InterruptedException, TimeoutException { + Future<List<?>> resultOne = provider.getAsyncOps() + .update("basic_test_unknown", Collections.singletonList(1), Arrays.asList("=", 1, "one")); + + Exception exception = assertThrows(Exception.class, () -> resultOne.get(TIMEOUT, TimeUnit.MILLISECONDS)); + assertTrue(exception.getCause() instanceof TarantoolSpaceNotFoundException); + + provider.close(); + } + + @ParameterizedTest + @MethodSource("getAsyncOps") + void testUnknownSpaceIndex(AsyncOpsProvider provider) { + Future<List<?>> resultOne = provider.getAsyncOps() + .select("basic_test", "pk_unknown", Collections.singletonList(3), 0, 1, Iterator.EQ); + + Exception exception = assertThrows(Exception.class, () -> resultOne.get(TIMEOUT, TimeUnit.MILLISECONDS)); + assertTrue(exception.getCause() instanceof TarantoolIndexNotFoundException); + + provider.close(); + } + private List<?> consoleSelect(Object key) { return testHelper.evaluate(TestUtils.toLuaSelect("basic_test", key)); } private interface AsyncOpsProvider { - TarantoolClientOps<Integer, List<?>, Object, Future<List<?>>> getAsyncOps(); + + TarantoolClientOps<List<?>, Object, Future<List<?>>> getAsyncOps(); TarantoolClient getClient(); void close(); + } private static class ClientAsyncOpsProvider implements AsyncOpsProvider { @@ -202,7 +415,7 @@ private static class ClientAsyncOpsProvider implements AsyncOpsProvider { TarantoolClient client = TestUtils.makeTestClient(TestUtils.makeDefaultClientConfig(), 2000); @Override - public TarantoolClientOps<Integer, List<?>, Object, Future<List<?>>> getAsyncOps() { + public TarantoolClientOps<List<?>, Object, Future<List<?>>> getAsyncOps() { return client.asyncOps(); } @@ -221,11 +434,11 @@ public void close() { private static class ComposableAsyncOpsProvider implements AsyncOpsProvider { TarantoolClient client = TestUtils.makeTestClient(TestUtils.makeDefaultClientConfig(), 2000); - TarantoolClientOps<Integer, List<?>, Object, Future<List<?>>> composableOps = + TarantoolClientOps<List<?>, Object, Future<List<?>>> composableOps = new Composable2FutureClientOpsAdapter(client.composableAsyncOps()); @Override - public TarantoolClientOps<Integer, List<?>, Object, Future<List<?>>> getAsyncOps() { + public TarantoolClientOps<List<?>, Object, Future<List<?>>> getAsyncOps() { return composableOps; } @@ -236,18 +449,18 @@ public TarantoolClient getClient() { @Override public void close() { - composableOps.close(); + client.close(); } } private static class Composable2FutureClientOpsAdapter - implements TarantoolClientOps<Integer, List<?>, Object, Future<List<?>>> { + implements TarantoolClientOps<List<?>, Object, Future<List<?>>> { - private final TarantoolClientOps<Integer, List<?>, Object, CompletionStage<List<?>>> originOps; + private final TarantoolClientOps<List<?>, Object, CompletionStage<List<?>>> originOps; private Composable2FutureClientOpsAdapter( - TarantoolClientOps<Integer, List<?>, Object, CompletionStage<List<?>>> originOps) { + TarantoolClientOps<List<?>, Object, CompletionStage<List<?>>> originOps) { this.originOps = originOps; } @@ -257,6 +470,11 @@ public Future<List<?>> select(Integer space, Integer index, List<?> key, int off return originOps.select(space, index, key, offset, limit, iterator).toCompletableFuture(); } + @Override + public Future<List<?>> select(String space, String index, List<?> key, int offset, int limit, int iterator) { + return originOps.select(space, index, key, offset, limit, iterator).toCompletableFuture(); + } + @Override public Future<List<?>> select(Integer space, Integer index, @@ -267,31 +485,66 @@ public Future<List<?>> select(Integer space, return originOps.select(space, index, key, offset, limit, iterator).toCompletableFuture(); } + @Override + public Future<List<?>> select(String space, + String index, + List<?> key, + int offset, + int limit, + Iterator iterator) { + return originOps.select(space, index, key, offset, limit, iterator).toCompletableFuture(); + } + @Override public Future<List<?>> insert(Integer space, List<?> tuple) { return originOps.insert(space, tuple).toCompletableFuture(); } + @Override + public Future<List<?>> insert(String space, List<?> tuple) { + return originOps.insert(space, tuple).toCompletableFuture(); + } + @Override public Future<List<?>> replace(Integer space, List<?> tuple) { return originOps.replace(space, tuple).toCompletableFuture(); } + @Override + public Future<List<?>> replace(String space, List<?> tuple) { + return originOps.replace(space, tuple).toCompletableFuture(); + } + @Override public Future<List<?>> update(Integer space, List<?> key, Object... tuple) { return originOps.update(space, key, tuple).toCompletableFuture(); } + @Override + public Future<List<?>> update(String space, List<?> key, Object... tuple) { + return originOps.update(space, key, tuple).toCompletableFuture(); + } + @Override public Future<List<?>> upsert(Integer space, List<?> key, List<?> defTuple, Object... ops) { return originOps.upsert(space, key, defTuple, ops).toCompletableFuture(); } + @Override + public Future<List<?>> upsert(String space, List<?> key, List<?> defTuple, Object... ops) { + return originOps.upsert(space, key, defTuple, ops).toCompletableFuture(); + } + @Override public Future<List<?>> delete(Integer space, List<?> key) { return originOps.delete(space, key).toCompletableFuture(); } + @Override + public Future<List<?>> delete(String space, List<?> key) { + return originOps.delete(space, key).toCompletableFuture(); + } + @Override public Future<List<?>> call(String function, Object... args) { return originOps.call(function, args).toCompletableFuture(); @@ -307,10 +560,16 @@ public void ping() { originOps.ping(); } + @Override + public Future<List<?>> execute(TarantoolRequestConvertible requestSpec) { + return originOps.execute(requestSpec).toCompletableFuture(); + } + @Override public void close() { originOps.close(); } + } } diff --git a/src/test/java/org/tarantool/ClientOperationsIT.java b/src/test/java/org/tarantool/ClientOperationsIT.java index 1f7e1826..3a2f004d 100644 --- a/src/test/java/org/tarantool/ClientOperationsIT.java +++ b/src/test/java/org/tarantool/ClientOperationsIT.java @@ -2,6 +2,12 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.tarantool.TestAssertions.checkRawTupleResult; +import static org.tarantool.dsl.Requests.selectRequest; + +import org.tarantool.schema.TarantoolIndexNotFoundException; +import org.tarantool.schema.TarantoolSpaceNotFoundException; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; @@ -9,6 +15,10 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + /** * Tests for synchronous operations of {@link TarantoolClientImpl} class. * @@ -34,11 +44,18 @@ public static void tearDownEnv() { @BeforeEach public void setUp() { + testHelper.executeLua( + "box.schema.space.create('basic_test', { format = " + + "{{name = 'id', type = 'integer'}," + + " {name = 'val', type = 'string'} } })", + "box.space.basic_test:create_index('pk', { type = 'TREE', parts = {'id'} } )" + ); client = TestUtils.makeTestClient(TestUtils.makeDefaultClientConfig(), 2000); } @AfterEach public void tearDown() { + testHelper.executeLua("box.space.basic_test and box.space.basic_test:drop()"); client.close(); } @@ -48,4 +65,170 @@ public void testClose() { assertEquals(e.getMessage(), "You should close TarantoolClient instead."); } + @Test + void testStringSelect() { + testHelper.executeLua("box.space.basic_test:insert{1, 'one'}"); + List<?> result = client.syncOps() + .select("basic_test", "pk", Collections.singletonList(1), 0, 1, Iterator.EQ); + + assertEquals(Collections.singletonList(Arrays.asList(1, "one")), result); + } + + @Test + void testStringInsert() { + client.syncOps().insert("basic_test", Arrays.asList(1, "one")); + client.syncOps().insert("basic_test", Arrays.asList(10, "ten")); + + checkRawTupleResult(consoleSelect(1), Arrays.asList(1, "one")); + checkRawTupleResult(consoleSelect(10), Arrays.asList(10, "ten")); + } + + @Test + void testStringReplace() { + testHelper.executeLua("box.space.basic_test:insert{1, '1'}"); + testHelper.executeLua("box.space.basic_test:insert{10, '10'}"); + + client.syncOps().replace("basic_test", Arrays.asList(1, "one")); + client.syncOps().replace("basic_test", Arrays.asList(10, "ten")); + + checkRawTupleResult(consoleSelect(1), Arrays.asList(1, "one")); + checkRawTupleResult(consoleSelect(10), Arrays.asList(10, "ten")); + } + + @Test + void testStringDelete() { + testHelper.executeLua("box.space.basic_test:insert{1, '1'}"); + testHelper.executeLua("box.space.basic_test:insert{10, '10'}"); + testHelper.executeLua("box.space.basic_test:insert{20, '20'}"); + + client.syncOps().delete("basic_test", Collections.singletonList(1)); + client.syncOps().delete("basic_test", Collections.singletonList(20)); + + assertEquals(Collections.emptyList(), consoleSelect(1)); + checkRawTupleResult(consoleSelect(10), Arrays.asList(10, "10")); + assertEquals(Collections.emptyList(), consoleSelect(20)); + } + + @Test + void testStringUpdate() { + testHelper.executeLua("box.space.basic_test:insert{1, '1'}"); + testHelper.executeLua("box.space.basic_test:insert{10, '10'}"); + + TarantoolClientOps<List<?>, Object, List<?>> clientOps = client.syncOps(); + clientOps.update("basic_test", Collections.singletonList(1), Arrays.asList("=", 1, "one")); + clientOps.update("basic_test", Collections.singletonList(2), Arrays.asList("=", 1, "two")); + clientOps.update("basic_test", Collections.singletonList(10), Arrays.asList("=", 1, "ten")); + + checkRawTupleResult(consoleSelect(1), Arrays.asList(1, "one")); + assertEquals(Collections.emptyList(), consoleSelect(2)); + checkRawTupleResult(consoleSelect(10), Arrays.asList(10, "ten")); + } + + @Test + void testStringUpsert() { + testHelper.executeLua("box.space.basic_test:insert{1, '1'}"); + testHelper.executeLua("box.space.basic_test:insert{10, '10'}"); + + TarantoolClientOps<List<?>, Object, List<?>> ops = client.syncOps(); + ops.upsert( + "basic_test", Collections.singletonList(1), + Arrays.asList(1, "001"), Arrays.asList("=", 1, "one") + ); + ops.upsert( + "basic_test", Collections.singletonList(2), + Arrays.asList(2, "002"), Arrays.asList("=", 1, "two") + ); + ops.upsert( + "basic_test", Collections.singletonList(10), + Arrays.asList(10, "010"), Arrays.asList("=", 1, "ten") + ); + + checkRawTupleResult(consoleSelect(1), Arrays.asList(1, "one")); + checkRawTupleResult(consoleSelect(2), Arrays.asList(2, "002")); + checkRawTupleResult(consoleSelect(10), Arrays.asList(10, "ten")); + } + + @Test + void testStringMultipleIndirectChanges() { + testHelper.executeLua("box.space.basic_test:insert{1, 'one'}"); + List<?> result = client.syncOps().select("basic_test", "pk", Collections.singletonList(1), 0, 1, Iterator.EQ); + assertEquals(Collections.singletonList(Arrays.asList(1, "one")), result); + + testHelper.executeLua("box.space.basic_test and box.space.basic_test:drop()"); + testHelper.executeLua( + "box.schema.space.create('basic_test', { format = " + + "{{name = 'id', type = 'integer'}," + + " {name = 'val', type = 'string'} } })", + + "box.space.basic_test:create_index('pk', { type = 'TREE', parts = {'id'} } )" + ); + testHelper.executeLua("box.space.basic_test:insert{2, 'two'}"); + result = client.syncOps().select("basic_test", "pk", Collections.singletonList(2), 0, 1, Iterator.EQ); + assertEquals(Collections.singletonList(Arrays.asList(2, "two")), result); + + testHelper.executeLua("box.space.basic_test and box.space.basic_test:drop()"); + testHelper.executeLua( + "box.schema.space.create('basic_test', { format = " + + "{{name = 'id', type = 'integer'}," + + " {name = 'val', type = 'string'} } })", + + "box.space.basic_test:create_index('pk', { type = 'TREE', parts = {'id'} } )" + ); + testHelper.executeLua("box.space.basic_test:insert{3, 'three'}"); + result = client.syncOps().select("basic_test", "pk", Collections.singletonList(3), 0, 1, Iterator.EQ); + assertEquals(Collections.singletonList(Arrays.asList(3, "three")), result); + } + + @Test + void testUnknownSpace() { + TarantoolClientOps<List<?>, Object, List<?>> clientOps = client.syncOps(); + Exception error = assertThrows( + Exception.class, + () -> clientOps.select("base_test_unknown", "pk", Collections.singletonList(12), 0, 1, Iterator.EQ) + ); + + assertTrue(error.getCause() instanceof TarantoolSpaceNotFoundException); + } + + @Test + void testUnknownSpaceIndex() { + TarantoolClientOps<List<?>, Object, List<?>> clientOps = client.syncOps(); + Exception error = assertThrows( + Exception.class, + () -> clientOps.select("basic_test", "pk_unknown", Collections.singletonList(12), 0, 1, Iterator.EQ) + ); + + assertTrue(error.getCause() instanceof TarantoolIndexNotFoundException); + } + + @Test + void testCreateSpaceAfterFailedRequest() { + TarantoolClientOps<List<?>, Object, List<?>> clientOps = client.syncOps(); + Exception error = assertThrows( + Exception.class, + () -> clientOps.execute(selectRequest("base_test_unknown", "pk")) + ); + assertTrue(error.getCause() instanceof TarantoolSpaceNotFoundException); + + testHelper.executeLua( + "box.schema.space.create('base_test_unknown', { format = { { name = 'id', type = 'integer' } } })", + "box.space.base_test_unknown:create_index('pk', { type = 'TREE', parts = {'id'} } )", + "box.space.base_test_unknown:insert{ 5 }" + ); + List<?> result = clientOps.execute(selectRequest("base_test_unknown", "pk")); + assertEquals(Arrays.asList(5), result.get(0)); + + error = assertThrows( + Exception.class, + () -> clientOps.execute(selectRequest("base_test_unknown1", "pk")) + ); + assertTrue(error.getCause() instanceof TarantoolSpaceNotFoundException); + + testHelper.executeLua("box.space.base_test_unknown:drop()"); + } + + private List<?> consoleSelect(Object key) { + return testHelper.evaluate(TestUtils.toLuaSelect("basic_test", key)); + } + } diff --git a/src/test/java/org/tarantool/ClientReconnectClusterIT.java b/src/test/java/org/tarantool/ClientReconnectClusterIT.java index c0228874..90c884e0 100644 --- a/src/test/java/org/tarantool/ClientReconnectClusterIT.java +++ b/src/test/java/org/tarantool/ClientReconnectClusterIT.java @@ -20,7 +20,7 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.Phaser; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -120,7 +120,7 @@ void testUpdateExtendedNodeList() { String service1Address = "localhost:" + PORTS[0]; String service2Address = "127.0.0.1:" + PORTS[1]; - CyclicBarrier barrier = new CyclicBarrier(2); + Phaser phaser = new Phaser(1); String infoFunctionName = "getAddresses"; String infoFunctionScript = @@ -131,7 +131,7 @@ void testUpdateExtendedNodeList() { final TarantoolClusterClient client = makeClientWithDiscoveryFeature( infoFunctionName, 0, - (ignored) -> tryAwait(barrier), + (ignored) -> phaser.arrive(), service1Address ); @@ -139,7 +139,7 @@ void testUpdateExtendedNodeList() { final int spaceId = ids[0]; final int pkId = ids[1]; - tryAwait(barrier); // client = { srv1 }; wait for { srv1, srv2 } + tryAwait(phaser, 0); // client = { srv1 }; wait for { srv1, srv2 } expectConnected(client, spaceId, pkId); @@ -165,7 +165,7 @@ void testUpdateNarrowNodeList() { String service1Address = "localhost:" + PORTS[0]; String service2Address = "127.0.0.1:" + PORTS[1]; - CyclicBarrier barrier = new CyclicBarrier(2); + Phaser phaser = new Phaser(1); String infoFunctionName = "getAddresses"; String infoFunctionScript = makeDiscoveryFunction(infoFunctionName, Collections.singletonList(service1Address)); @@ -175,7 +175,7 @@ void testUpdateNarrowNodeList() { final TarantoolClusterClient client = makeClientWithDiscoveryFeature( infoFunctionName, 0, - (ignored) -> tryAwait(barrier), + (ignored) -> phaser.arrive(), service1Address, service2Address ); @@ -184,7 +184,7 @@ void testUpdateNarrowNodeList() { final int spaceId = ids[0]; final int pkId = ids[1]; - tryAwait(barrier); // client = { srv1, srv2 }; wait for { srv1 } + tryAwait(phaser, 0); // client = { srv1, srv2 }; wait for { srv1 } expectConnected(client, spaceId, pkId); @@ -349,7 +349,7 @@ void testDelayFunctionResultFetch() { String service2Address = "127.0.0.1:" + PORTS[1]; String service3Address = "localhost:" + PORTS[2]; - CyclicBarrier barrier = new CyclicBarrier(2); + Phaser phaser = new Phaser(1); String infoFunctionName = "getAddressesFunction"; String functionBody = Stream.of(service1Address, service2Address) @@ -367,7 +367,7 @@ void testDelayFunctionResultFetch() { final TarantoolClusterClient client = makeClientWithDiscoveryFeature( infoFunctionName, 3000, - (ignored) -> tryAwait(barrier), + (ignored) -> phaser.arrive(), service1Address ); @@ -375,16 +375,16 @@ void testDelayFunctionResultFetch() { final int spaceId = ids[0]; final int pkId = ids[1]; - tryAwait(barrier); // client = { srv1 }; wait for { srv1 } + tryAwait(phaser, 0); // client = { srv1 }; wait for { srv1 } expectConnected(client, spaceId, pkId); - tryAwait(barrier); // client = { srv1 }; wait for { srv2 } + tryAwait(phaser, 1); // client = { srv1 }; wait for { srv2 } stopInstancesAndAwait(SRV1); expectConnected(client, spaceId, pkId); - tryAwait(barrier); // client = { srv2 }; wait for { srv3 } + tryAwait(phaser, 2); // client = { srv2 }; wait for { srv3 } stopInstancesAndAwait(SRV2); expectConnected(client, spaceId, pkId); @@ -393,9 +393,9 @@ void testDelayFunctionResultFetch() { expectDisconnected(client, spaceId, pkId); } - private void tryAwait(CyclicBarrier barrier) { + private void tryAwait(Phaser phaser, int phase) { try { - barrier.await(6000, TimeUnit.MILLISECONDS); + phaser.awaitAdvanceInterruptibly(phase, 6000, TimeUnit.MILLISECONDS); } catch (Throwable e) { e.printStackTrace(); } diff --git a/src/test/java/org/tarantool/ClientReconnectIT.java b/src/test/java/org/tarantool/ClientReconnectIT.java index f644fc8f..339bc82b 100644 --- a/src/test/java/org/tarantool/ClientReconnectIT.java +++ b/src/test/java/org/tarantool/ClientReconnectIT.java @@ -186,16 +186,12 @@ protected void write(Code code, Long syncId, Long schemaId, Object... args) thro testHelper.stopInstance(); - assertThrows(ExecutionException.class, new Executable() { - @Override - public void execute() throws Throwable { - mustFail.get(); - } - }); + ExecutionException executionException = assertThrows(ExecutionException.class, mustFail::get); + assertEquals(executionException.getCause().getClass(), CommunicationException.class); + writeEnabled.set(true); testHelper.startInstance(); - writeEnabled.set(true); try { client.waitAlive(RESTART_TIMEOUT, TimeUnit.MILLISECONDS); diff --git a/src/test/java/org/tarantool/FireAndForgetClientOperationsIT.java b/src/test/java/org/tarantool/FireAndForgetClientOperationsIT.java index 97cf697a..679844f5 100644 --- a/src/test/java/org/tarantool/FireAndForgetClientOperationsIT.java +++ b/src/test/java/org/tarantool/FireAndForgetClientOperationsIT.java @@ -91,12 +91,12 @@ public void execute() throws Throwable { @Test public void testFireAndForgetOperations() { - TarantoolClientOps<Integer, List<?>, Object, Long> ffOps = client.fireAndForgetOps(); + TarantoolClientOps<List<?>, Object, Long> ffOps = client.fireAndForgetOps(); - Set<Long> syncIds = new HashSet<Long>(); + Set<Long> syncIds = new HashSet<>(); - syncIds.add(ffOps.insert(spaceId, Arrays.asList(10, "10"))); - syncIds.add(ffOps.delete(spaceId, Collections.singletonList(10))); + syncIds.add(ffOps.insert(spaceId, Arrays.asList(1, "1"))); + syncIds.add(ffOps.delete(spaceId, Collections.singletonList(1))); syncIds.add(ffOps.insert(spaceId, Arrays.asList(10, "10"))); syncIds.add(ffOps.update(spaceId, Collections.singletonList(10), Arrays.asList("=", 1, "ten"))); @@ -117,9 +117,39 @@ public void testFireAndForgetOperations() { client.syncOps().ping(); // Check the effects + assertEquals(consoleSelect(SPACE_NAME, 1), Collections.emptyList()); checkRawTupleResult(consoleSelect(SPACE_NAME, 10), Arrays.asList(10, "ten")); checkRawTupleResult(consoleSelect(SPACE_NAME, 20), Arrays.asList(20, "twenty")); - assertEquals(consoleSelect(SPACE_NAME, 30), Collections.emptyList()); + assertEquals(Collections.emptyList(), consoleSelect(SPACE_NAME, 30)); + } + + @Test + public void testFireAndForgetStringOperations() { + TarantoolClientOps<List<?>, Object, Long> ffOps = client.fireAndForgetOps(); + + Set<Long> syncIds = new HashSet<>(); + + syncIds.add(ffOps.insert(SPACE_NAME, Arrays.asList(2, "2"))); + syncIds.add(ffOps.insert(SPACE_NAME, Arrays.asList(20, "20"))); + syncIds.add(ffOps.replace(SPACE_NAME, Arrays.asList(200, "200"))); + + syncIds.add(ffOps.delete(SPACE_NAME, Collections.singletonList(2))); + syncIds.add(ffOps.update(SPACE_NAME, Collections.singletonList(20), Arrays.asList("=", 1, "twenty"))); + syncIds.add(ffOps.upsert(SPACE_NAME, Collections.singletonList(200), Arrays.asList(200, "two hundred"), + Arrays.asList("=", 1, "two hundred"))); + + // Check the syncs. + assertFalse(syncIds.contains(0L)); + assertEquals(6, syncIds.size()); + + // The reply for synchronous ping will + // indicate to us that previous fire & forget operations are completed. + client.syncOps().ping(); + + // Check the effects + assertEquals(consoleSelect(SPACE_NAME, 2), Collections.emptyList()); + checkRawTupleResult(consoleSelect(SPACE_NAME, 20), Arrays.asList(20, "twenty")); + checkRawTupleResult(consoleSelect(SPACE_NAME, 200), Arrays.asList(200, "two hundred")); } private List<?> consoleSelect(String spaceName, Object key) { diff --git a/src/test/java/org/tarantool/IteratorTest.java b/src/test/java/org/tarantool/IteratorTest.java deleted file mode 100644 index 7fd68b61..00000000 --- a/src/test/java/org/tarantool/IteratorTest.java +++ /dev/null @@ -1,35 +0,0 @@ -package org.tarantool; - -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; - -class IteratorTest { - protected class MockOps extends AbstractTarantoolOps<Integer, List<?>, Object, List<?>> { - - @Override - public List exec(Code code, Object... args) { - return null; - } - - @Override - public void close() { - throw new UnsupportedOperationException(); - } - } - - @Test - void testSelectWithIteratorInsteadOfInteger() { - MockOps ops = new MockOps(); - MockOps spyOps = spy(ops); - - spyOps.select(1, 1, new ArrayList<Integer>(), 0, 1, Iterator.EQ); - - verify(spyOps, times(1)).select(1, 1, new ArrayList<Integer>(), 0, 1, 0); - } -} diff --git a/src/test/java/org/tarantool/TarantoolClientOpsIT.java b/src/test/java/org/tarantool/TarantoolClientOpsIT.java index ebaca9f5..86535f54 100644 --- a/src/test/java/org/tarantool/TarantoolClientOpsIT.java +++ b/src/test/java/org/tarantool/TarantoolClientOpsIT.java @@ -256,7 +256,7 @@ public void testReplaceMultiPartKey(SyncOpsProvider provider) { provider.close(); } - private void checkReplace(TarantoolClientOps<Integer, List<?>, Object, List<?>> clientOps, + private void checkReplace(TarantoolClientOps<List<?>, Object, List<?>> clientOps, String space, int spaceId, List key, @@ -332,7 +332,7 @@ public void testUpdateMultiPart(SyncOpsProvider provider) { provider.close(); } - private void checkUpdate(TarantoolClientOps<Integer, List<?>, Object, List<?>> clientOps, + private void checkUpdate(TarantoolClientOps<List<?>, Object, List<?>> clientOps, String space, int spaceId, List key, @@ -391,7 +391,7 @@ public void testUpsertMultiPart(SyncOpsProvider provider) { provider.close(); } - private void checkUpsert(TarantoolClientOps<Integer, List<?>, Object, List<?>> clientOps, + private void checkUpsert(TarantoolClientOps<List<?>, Object, List<?>> clientOps, String space, int spaceId, List key, @@ -446,7 +446,7 @@ public void testDeleteMultiPartKey(SyncOpsProvider provider) { provider.close(); } - private void checkDelete(TarantoolClientOps<Integer, List<?>, Object, List<?>> clientOps, + private void checkDelete(TarantoolClientOps<List<?>, Object, List<?>> clientOps, String space, int spaceId, List key, @@ -580,12 +580,10 @@ public void execute() throws Throwable { @MethodSource("getClientOps") public void testInsertDuplicateKey(SyncOpsProvider provider) { final List tup = Arrays.asList(1, "uno"); - TarantoolException ex = assertThrows(TarantoolException.class, new Executable() { - @Override - public void execute() throws Throwable { - provider.getClientOps().insert(spaceId, tup); - } - }); + TarantoolException ex = assertThrows( + TarantoolException.class, + () -> provider.getClientOps().insert(spaceId, tup) + ); assertEquals("Duplicate key exists in unique index 'pk' in space 'basic_test'", ex.getMessage()); // Check the tuple stayed intact. @@ -663,7 +661,7 @@ private static TarantoolConnection makeConnection() { } private interface SyncOpsProvider { - TarantoolClientOps<Integer, List<?>, Object, List<?>> getClientOps(); + TarantoolClientOps<List<?>, Object, List<?>> getClientOps(); void close(); } @@ -673,7 +671,7 @@ private static class ClientSyncOpsProvider implements SyncOpsProvider { private TarantoolClient client = makeTestClient(makeDefaultClientConfig(), RESTART_TIMEOUT); @Override - public TarantoolClientOps<Integer, List<?>, Object, List<?>> getClientOps() { + public TarantoolClientOps<List<?>, Object, List<?>> getClientOps() { return client.syncOps(); } @@ -689,7 +687,7 @@ private static class ConnectionSyncOpsProvider implements SyncOpsProvider { private TarantoolConnection connection = makeConnection(); @Override - public TarantoolClientOps<Integer, List<?>, Object, List<?>> getClientOps() { + public TarantoolClientOps<List<?>, Object, List<?>> getClientOps() { return connection; } diff --git a/src/test/java/org/tarantool/TestUtils.java b/src/test/java/org/tarantool/TestUtils.java index b65a5de8..8cff87fb 100644 --- a/src/test/java/org/tarantool/TestUtils.java +++ b/src/test/java/org/tarantool/TestUtils.java @@ -275,7 +275,7 @@ public static TarantoolClusterClientConfig makeDefaultClusterClientConfig() { config.username = TarantoolTestHelper.USERNAME; config.password = TarantoolTestHelper.PASSWORD; config.initTimeoutMillis = 2000; - config.operationExpiryTimeMillis = 1000; + config.operationExpiryTimeMillis = 2000; config.sharedBufferSize = 128; config.executor = null; return config; diff --git a/src/test/java/org/tarantool/cluster/ClusterServiceStoredFunctionDiscovererIT.java b/src/test/java/org/tarantool/cluster/ClusterServiceStoredFunctionDiscovererIT.java index 4e940033..858e2143 100644 --- a/src/test/java/org/tarantool/cluster/ClusterServiceStoredFunctionDiscovererIT.java +++ b/src/test/java/org/tarantool/cluster/ClusterServiceStoredFunctionDiscovererIT.java @@ -9,7 +9,6 @@ import static org.tarantool.TestUtils.makeDiscoveryFunction; import org.tarantool.CommunicationException; -import org.tarantool.TarantoolClient; import org.tarantool.TarantoolClientImpl; import org.tarantool.TarantoolClusterClientConfig; import org.tarantool.TarantoolException; @@ -36,7 +35,7 @@ public class ClusterServiceStoredFunctionDiscovererIT { private static TarantoolTestHelper testHelper; private TarantoolClusterClientConfig clusterConfig; - private TarantoolClient client; + private TarantoolClientImpl client; @BeforeAll public static void setupEnv() { diff --git a/src/test/java/org/tarantool/jdbc/JdbcConnectionTimeoutIT.java b/src/test/java/org/tarantool/jdbc/JdbcConnectionTimeoutIT.java index 7f4d2b3f..22d571f3 100644 --- a/src/test/java/org/tarantool/jdbc/JdbcConnectionTimeoutIT.java +++ b/src/test/java/org/tarantool/jdbc/JdbcConnectionTimeoutIT.java @@ -7,6 +7,7 @@ import org.tarantool.ServerVersion; import org.tarantool.TarantoolClientConfig; +import org.tarantool.TarantoolRequest; import org.tarantool.TarantoolTestHelper; import org.tarantool.protocol.TarantoolPacket; @@ -51,12 +52,12 @@ void setUp() throws SQLException { protected SQLTarantoolClientImpl makeSqlClient(String address, TarantoolClientConfig config) { return new SQLTarantoolClientImpl(address, config) { @Override - protected void completeSql(TarantoolOp<?> operation, TarantoolPacket pack) { + protected void completeSql(TarantoolRequest request, TarantoolPacket pack) { try { Thread.sleep(LONG_ENOUGH_TIMEOUT); } catch (InterruptedException ignored) { } - super.completeSql(operation, pack); + super.completeSql(request, pack); } }; } diff --git a/src/test/java/org/tarantool/jdbc/JdbcExceptionHandlingTest.java b/src/test/java/org/tarantool/jdbc/JdbcExceptionHandlingTest.java index f6a37ff4..c70e74df 100644 --- a/src/test/java/org/tarantool/jdbc/JdbcExceptionHandlingTest.java +++ b/src/test/java/org/tarantool/jdbc/JdbcExceptionHandlingTest.java @@ -46,7 +46,7 @@ public class JdbcExceptionHandlingTest { @Test public void testDatabaseMetaDataGetPrimaryKeysFormatError() throws SQLException { - TarantoolClientOps<Integer, List<?>, Object, List<?>> syncOps = mock(TarantoolClientOps.class); + TarantoolClientOps<List<?>, Object, List<?>> syncOps = mock(TarantoolClientOps.class); Object[] spc = new Object[7]; spc[FORMAT_IDX] = Collections.singletonList(new HashMap<String, Object>()); @@ -195,7 +195,7 @@ public void execute() throws Throwable { private void checkDatabaseMetaDataCommunicationException(final ThrowingConsumer<DatabaseMetaData> consumer, String msg) throws SQLException { Exception ex = new CommunicationException("TEST"); - TarantoolClientOps<Integer, List<?>, Object, List<?>> syncOps = mock(TarantoolClientOps.class); + TarantoolClientOps<List<?>, Object, List<?>> syncOps = mock(TarantoolClientOps.class); doThrow(ex).when(syncOps).select(_VSPACE, 0, Arrays.asList(), 0, SPACES_MAX, 0); doThrow(ex).when(syncOps).select(_VSPACE, 2, Arrays.asList("TEST"), 0, 1, 0); @@ -212,7 +212,7 @@ private void checkDatabaseMetaDataCommunicationException(final ThrowingConsumer< } private SQLTarantoolClientImpl buildSQLClient(SQLTarantoolClientImpl.SQLRawOps sqlOps, - TarantoolClientOps<Integer, List<?>, Object, List<?>> ops) { + TarantoolClientOps<List<?>, Object, List<?>> ops) { SQLTarantoolClientImpl client = mock(SQLTarantoolClientImpl.class); when(client.sqlRawOps()).thenReturn(sqlOps); when(client.syncOps()).thenReturn(ops); diff --git a/src/test/java/org/tarantool/schema/ClientReconnectSchemaIT.java b/src/test/java/org/tarantool/schema/ClientReconnectSchemaIT.java new file mode 100644 index 00000000..2c545b4e --- /dev/null +++ b/src/test/java/org/tarantool/schema/ClientReconnectSchemaIT.java @@ -0,0 +1,91 @@ +package org.tarantool.schema; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.tarantool.TestUtils.makeDefaultClusterClientConfig; +import static org.tarantool.dsl.Requests.selectRequest; + +import org.tarantool.TarantoolClientImpl; +import org.tarantool.TarantoolClusterClient; +import org.tarantool.TarantoolClusterClientConfig; +import org.tarantool.TarantoolTestHelper; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +public class ClientReconnectSchemaIT { + + private static final String[] SRVS = { "srv-schema-it-1", "srv-schema-it-2" }; + private static final int[] PORTS = { 3401, 3402 }; + + private static TarantoolTestHelper firstTestHelper; + private static TarantoolTestHelper secondTestHelper; + + @BeforeAll + public static void setupEnv() { + firstTestHelper = new TarantoolTestHelper(SRVS[0]); + firstTestHelper.createInstance(TarantoolTestHelper.LUA_FILE, PORTS[0], PORTS[0] + 1000); + firstTestHelper.startInstance(); + + secondTestHelper = new TarantoolTestHelper(SRVS[1]); + secondTestHelper.createInstance(TarantoolTestHelper.LUA_FILE, PORTS[1], PORTS[1] + 1000); + secondTestHelper.startInstance(); + } + + @AfterAll + public static void teardownEnv() { + firstTestHelper.stopInstance(); + secondTestHelper.stopInstance(); + } + + @Test + @DisplayName("got a result from another node after the current node had disappeared") + public void testSameNamedSpaceAfterReconnection() { + String[] firstSpace = { + "box.schema.space.create('string_space1', { format = { {name = 'id', type = 'integer'} } })", + "box.space.string_space1:create_index('primary', { type = 'TREE', parts = {'id'} })" + }; + String[] secondSpace = { + "box.schema.space.create('string_space2', { format = { {name = 'id', type = 'integer'} } })", + "box.space.string_space2:create_index('primary', { type = 'TREE', parts = {'id'} })" + }; + + // create spaces on two instances with an inverted order + // as a result, instances have same schema version but spaces have unequal IDs + firstTestHelper.executeLua(firstSpace); + firstTestHelper.executeLua(secondSpace); + firstTestHelper.executeLua("box.space.string_space1:insert{100}"); + secondTestHelper.executeLua(secondSpace); + secondTestHelper.executeLua(firstSpace); + secondTestHelper.executeLua("box.space.string_space1:insert{200}"); + assertEquals(firstTestHelper.getInstanceVersion(), secondTestHelper.getInstanceVersion()); + + int firstSpaceIdFirstInstance = firstTestHelper.evaluate("box.space.string_space1.id"); + int firstSpaceIdSecondInstance = secondTestHelper.evaluate("box.space.string_space1.id"); + assertNotEquals(firstSpaceIdFirstInstance, firstSpaceIdSecondInstance); + + final TarantoolClientImpl client = makeClusterClient( + "localhost:" + PORTS[0], + "127.0.0.1:" + PORTS[1] + ); + + List<?> result = client.syncOps().execute(selectRequest("string_space1", "primary")); + assertEquals(Arrays.asList(100), result.get(0)); + firstTestHelper.stopInstance(); + + result = client.syncOps().execute(selectRequest("string_space1", "primary")); + assertEquals(Arrays.asList(200), result.get(0)); + secondTestHelper.stopInstance(); + } + + private TarantoolClusterClient makeClusterClient(String... addresses) { + TarantoolClusterClientConfig config = makeDefaultClusterClientConfig(); + return new TarantoolClusterClient(config, addresses); + } + +} diff --git a/src/test/java/org/tarantool/schema/ClientSchemaIT.java b/src/test/java/org/tarantool/schema/ClientSchemaIT.java new file mode 100644 index 00000000..214c599c --- /dev/null +++ b/src/test/java/org/tarantool/schema/ClientSchemaIT.java @@ -0,0 +1,247 @@ +package org.tarantool.schema; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.fail; +import static org.tarantool.TestUtils.makeDefaultClientConfig; + +import org.tarantool.ServerVersion; +import org.tarantool.TarantoolClientConfig; +import org.tarantool.TarantoolClientImpl; +import org.tarantool.TarantoolTestHelper; +import org.tarantool.TestAssumptions; +import org.tarantool.schema.TarantoolIndexMeta.IndexOptions; +import org.tarantool.schema.TarantoolIndexMeta.IndexPart; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.List; + +@DisplayName("A schema meta") +public class ClientSchemaIT { + + private static TarantoolTestHelper testHelper; + + private TarantoolClientImpl client; + + @BeforeAll + public static void setupEnv() { + testHelper = new TarantoolTestHelper("client-schema-it"); + testHelper.createInstance(); + testHelper.startInstance(); + } + + @AfterAll + public static void teardownEnv() { + testHelper.stopInstance(); + } + + @BeforeEach + public void setup() { + TarantoolClientConfig config = makeDefaultClientConfig(); + + client = new TarantoolClientImpl( + TarantoolTestHelper.HOST + ":" + TarantoolTestHelper.PORT, + config + ); + } + + @AfterEach + public void tearDown() { + client.close(); + testHelper.executeLua("box.space.count_space and box.space.count_space:drop()"); + } + + @Test + @DisplayName("fetched a space with its index") + void testFetchSpaces() { + testHelper.executeLua( + "box.schema.space.create('count_space', { format = " + + "{ {name = 'id', type = 'integer'}," + + " {name = 'counts', type = 'integer'} }" + + "})" + ); + testHelper.executeLua("box.space.count_space:create_index('pk', { type = 'TREE', parts = {'id'} } )"); + + TarantoolSchemaMeta meta = new TarantoolMetaSpacesCache(client); + meta.refresh(); + + TarantoolSpaceMeta space = meta.getSpace("count_space"); + assertNotNull(space); + assertEquals("count_space", space.getName()); + + List<TarantoolSpaceMeta.SpaceField> spaceFormat = space.getFormat(); + assertEquals(2, spaceFormat.size()); + assertEquals("id", spaceFormat.get(0).getName()); + assertEquals("integer", spaceFormat.get(0).getType()); + assertEquals("counts", spaceFormat.get(1).getName()); + assertEquals("integer", spaceFormat.get(1).getType()); + + TarantoolIndexMeta primaryIndex = space.getIndex("pk"); + TarantoolIndexMeta expectedPrimaryIndex = new TarantoolIndexMeta( + 0, "pk", "TREE", + new IndexOptions(true), + Collections.singletonList(new IndexPart(0, "integer")) + ); + assertIndex(expectedPrimaryIndex, primaryIndex); + } + + @Test + @DisplayName("fetched newly created spaces and indexes") + void testFetchNewSpaces() { + // add count_space + testHelper.executeLua( + "box.schema.space.create('count_space', { format = " + + "{ {name = 'id', type = 'integer'}," + + " {name = 'counts', type = 'integer'} }" + + "})" + ); + TarantoolSchemaMeta meta = new TarantoolMetaSpacesCache(client); + meta.refresh(); + TarantoolSpaceMeta space = meta.getSpace("count_space"); + assertNotNull(space); + assertEquals("count_space", space.getName()); + assertThrows(TarantoolSpaceNotFoundException.class, () -> meta.getSpace("count_space_2")); + + // add count_space_2 + testHelper.executeLua( + "box.schema.space.create('count_space_2', { format = " + + "{ {name = 'id', type = 'integer'} } })" + ); + meta.refresh(); + space = meta.getSpace("count_space_2"); + assertNotNull(space); + assertEquals("count_space_2", space.getName()); + assertThrows(TarantoolIndexNotFoundException.class, () -> meta.getSpaceIndex("count_space_2", "pk")); + + // add a primary index for count_space_2 + testHelper.executeLua( + "box.space.count_space_2:create_index('pk', { unique = true, type = 'TREE', parts = {'id'} } )" + ); + meta.refresh(); + TarantoolIndexMeta spaceIndex = meta.getSpaceIndex("count_space_2", "pk"); + TarantoolIndexMeta expectedPrimaryIndex = new TarantoolIndexMeta( + 0, "pk", "TREE", + new IndexOptions(true), + Collections.singletonList(new IndexPart(0, "integer")) + ); + assertIndex(expectedPrimaryIndex, spaceIndex); + } + + @Test + @DisplayName("fetched space indexes of a space") + void testFetchIndexes() { + testHelper.executeLua( + "box.schema.space.create('count_space', { format = " + + "{ {name = 'id', type = 'integer'}," + + " {name = 'counts', type = 'integer'} }" + + "})" + ); + testHelper.executeLua( + "box.space.count_space:create_index('pk', { type = 'HASH', parts = {'id'} } )", + "box.space.count_space:create_index('c_index', { unique = false, type = 'TREE', parts = {'counts'} } )" + ); + + TarantoolSchemaMeta meta = new TarantoolMetaSpacesCache(client); + meta.refresh(); + + TarantoolIndexMeta primaryIndex = meta.getSpaceIndex("count_space", "pk"); + TarantoolIndexMeta expectedPrimaryIndex = new TarantoolIndexMeta( + 0, "pk", "HASH", + new IndexOptions(true), + Collections.singletonList(new IndexPart(0, "integer")) + ); + assertIndex(expectedPrimaryIndex, primaryIndex); + + TarantoolIndexMeta secondaryIndex = meta.getSpaceIndex("count_space", "c_index"); + TarantoolIndexMeta expectedSecondaryIndex = new TarantoolIndexMeta( + 1, "c_index", "TREE", + new IndexOptions(false), + Collections.singletonList(new IndexPart(1, "integer")) + ); + assertIndex(expectedSecondaryIndex, secondaryIndex); + } + + @Test + @DisplayName("fetched sql table primary index") + void testFetchSqlIndexes() { + TestAssumptions.assumeMinimalServerVersion(testHelper.getInstanceVersion(), ServerVersion.V_2_1); + testHelper.executeSql("create table my_table (id int primary key, val varchar(100))"); + + TarantoolSchemaMeta meta = new TarantoolMetaSpacesCache(client); + meta.refresh(); + + TarantoolIndexMeta primaryIndex = meta.getSpaceIndex("MY_TABLE", "pk_unnamed_MY_TABLE_1"); + TarantoolIndexMeta expectedPrimaryIndex = new TarantoolIndexMeta( + 0, "pk_unnamed_MY_TABLE_1", "tree", + new IndexOptions(true), + Collections.singletonList(new IndexPart(0, "integer")) + ); + assertIndex(expectedPrimaryIndex, primaryIndex); + } + + @Test + @DisplayName("got an error with a wrong space name") + void tesGetUnknownSpace() { + TarantoolSchemaMeta meta = new TarantoolMetaSpacesCache(client); + meta.refresh(); + + TarantoolSpaceNotFoundException exception = assertThrows( + TarantoolSpaceNotFoundException.class, + () -> meta.getSpace("unknown_space") + ); + assertEquals("unknown_space", exception.getSchemaName()); + } + + @Test + @DisplayName("got an error with a wrong space index name") + void testGetUnknownSpaceIndex() { + testHelper.executeLua( + "box.schema.space.create('count_space', { format = " + + "{ {name = 'id', type = 'integer'} } })" + ); + testHelper.executeLua("box.space.count_space:create_index('pk', { type = 'TREE', parts = {'id'} } )"); + + TarantoolSchemaMeta meta = new TarantoolMetaSpacesCache(client); + meta.refresh(); + + assertEquals("count_space", meta.getSpace("count_space").getName()); + TarantoolIndexNotFoundException exception = assertThrows( + TarantoolIndexNotFoundException.class, + () -> meta.getSpaceIndex("count_space", "wrong_pk") + ); + assertEquals("wrong_pk", exception.getIndexName()); + } + + private void assertIndex(TarantoolIndexMeta expectedIndex, TarantoolIndexMeta actualIndex) { + assertEquals(expectedIndex.getId(), actualIndex.getId()); + assertEquals(expectedIndex.getName(), actualIndex.getName()); + assertEquals(expectedIndex.getType(), actualIndex.getType()); + assertEqualsOptions(expectedIndex.getOptions(), actualIndex.getOptions()); + assertEqualsParts(expectedIndex.getParts(), actualIndex.getParts()); + } + + private void assertEqualsOptions(IndexOptions expected, IndexOptions actual) { + assertEquals(expected.isUnique(), actual.isUnique()); + } + + private void assertEqualsParts(List<IndexPart> expected, List<IndexPart> actual) { + if (expected.size() != actual.size()) { + fail("Part lists have different sizes"); + } + for (int i = 0; i < expected.size(); i++) { + IndexPart expectedPart = expected.get(i); + IndexPart actualPart = actual.get(i); + assertEquals(expectedPart.getFieldNumber(), actualPart.getFieldNumber()); + assertEquals(expectedPart.getType(), actualPart.getType()); + } + } + +} diff --git a/src/test/java/org/tarantool/schema/ClientThreadSafeSchemaIT.java b/src/test/java/org/tarantool/schema/ClientThreadSafeSchemaIT.java new file mode 100644 index 00000000..0546f44f --- /dev/null +++ b/src/test/java/org/tarantool/schema/ClientThreadSafeSchemaIT.java @@ -0,0 +1,105 @@ +package org.tarantool.schema; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.tarantool.TestUtils.makeDefaultClientConfig; +import static org.tarantool.TestUtils.makeTestClient; + +import org.tarantool.TarantoolClient; +import org.tarantool.TarantoolClientConfig; +import org.tarantool.TarantoolClientOps; +import org.tarantool.TarantoolTestHelper; +import org.tarantool.TarantoolThreadDaemonFactory; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +@DisplayName("A client") +public class ClientThreadSafeSchemaIT { + + private static TarantoolTestHelper testHelper; + + @BeforeAll + public static void setupEnv() { + testHelper = new TarantoolTestHelper("client-schema-thread-safe-it"); + testHelper.createInstance(); + testHelper.startInstance(); + } + + @AfterAll + public static void teardownEnv() { + testHelper.stopInstance(); + } + + @Test + @DisplayName("executed many DML/DDL string-operations from several threads simultaneously") + void testFetchSpaces() { + testHelper.executeLua( + makeCreateSpaceFunction(), + makeDropSpaceFunction() + ); + + TarantoolClientConfig config = makeDefaultClientConfig(); + config.operationExpiryTimeMillis = 2000; + TarantoolClient client = makeTestClient(config, 500); + + int threadsNumber = 16; + int iterations = 100; + final CountDownLatch latch = new CountDownLatch(threadsNumber); + ExecutorService executor = Executors.newFixedThreadPool( + threadsNumber, + new TarantoolThreadDaemonFactory("testWorkers") + ); + + // multiple threads can cause schema invalidation simultaneously + // that hasn't to affect affect other threads + for (int i = 0; i < threadsNumber; i++) { + int threadNumber = i; + executor.submit(() -> { + String spaceName = "my_space" + threadNumber; + for (int k = 0; k < iterations; k++) { + TarantoolClientOps<List<?>, Object, List<?>> ops = client.syncOps(); + ops.call("makeSpace", spaceName); + ops.insert(spaceName, Arrays.asList(k, threadNumber)); + ops.call("dropSpace", spaceName); + } + latch.countDown(); + }); + } + + try { + assertTrue(latch.await(20, TimeUnit.SECONDS)); + } catch (InterruptedException e) { + fail(e); + } finally { + executor.shutdownNow(); + client.close(); + } + } + + private String makeCreateSpaceFunction() { + return "function makeSpace(spaceName) " + + "box.schema.space.create(spaceName, { format = " + + "{ {name = 'id', type = 'integer'}, " + + " {name = 'counts', type = 'integer'} } " + + "}); " + + "box.space[spaceName]:create_index('pk', { type = 'TREE', parts = {'id'} } ) " + + "end"; + } + + private String makeDropSpaceFunction() { + return "function dropSpace(spaceName) " + + "box.space[spaceName]:drop() " + + "end"; + } + +}