http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProcessor.java deleted file mode 100644 index b106dfc..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProcessor.java +++ /dev/null @@ -1,697 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest; - -import org.apache.ignite.*; -import org.apache.ignite.configuration.*; -import org.apache.ignite.internal.*; -import org.apache.ignite.internal.processors.*; -import org.apache.ignite.internal.util.*; -import org.apache.ignite.lang.*; -import org.apache.ignite.plugin.security.*; -import org.apache.ignite.spi.authentication.*; -import org.apache.ignite.internal.managers.securesession.*; -import org.apache.ignite.internal.managers.security.*; -import org.gridgain.grid.kernal.processors.rest.client.message.*; -import org.gridgain.grid.kernal.processors.rest.handlers.*; -import org.gridgain.grid.kernal.processors.rest.handlers.cache.*; -import org.gridgain.grid.kernal.processors.rest.handlers.log.*; -import org.gridgain.grid.kernal.processors.rest.handlers.metadata.*; -import org.gridgain.grid.kernal.processors.rest.handlers.task.*; -import org.gridgain.grid.kernal.processors.rest.handlers.top.*; -import org.gridgain.grid.kernal.processors.rest.handlers.version.*; -import org.gridgain.grid.kernal.processors.rest.protocols.tcp.*; -import org.gridgain.grid.kernal.processors.rest.request.*; -import org.apache.ignite.internal.util.future.*; -import org.apache.ignite.internal.util.typedef.*; -import org.apache.ignite.internal.util.typedef.internal.*; -import org.apache.ignite.internal.util.worker.*; -import org.jdk8.backport.*; - -import java.lang.reflect.*; -import java.util.*; -import java.util.concurrent.*; - -import static org.apache.ignite.plugin.security.GridSecuritySubjectType.*; -import static org.gridgain.grid.kernal.processors.rest.GridRestResponse.*; - -/** - * Rest processor implementation. - */ -public class GridRestProcessor extends GridProcessorAdapter { - /** HTTP protocol class name. */ - private static final String HTTP_PROTO_CLS = - "org.gridgain.grid.kernal.processors.rest.protocols.http.jetty.GridJettyRestProtocol"; - - /** Protocols. */ - private final Collection<GridRestProtocol> protos = new ArrayList<>(); - - /** Command handlers. */ - protected final Map<GridRestCommand, GridRestCommandHandler> handlers = new EnumMap<>(GridRestCommand.class); - - /** */ - private final CountDownLatch startLatch = new CountDownLatch(1); - - /** Busy lock. */ - private final GridSpinReadWriteLock busyLock = new GridSpinReadWriteLock(); - - /** Workers count. */ - private final LongAdder workersCnt = new LongAdder(); - - /** Protocol handler. */ - private final GridRestProtocolHandler protoHnd = new GridRestProtocolHandler() { - @Override public GridRestResponse handle(GridRestRequest req) throws IgniteCheckedException { - return handleAsync(req).get(); - } - - @Override public IgniteFuture<GridRestResponse> handleAsync(GridRestRequest req) { - return handleAsync0(req); - } - }; - - /** - * @param req Request. - * @return Future. - */ - private IgniteFuture<GridRestResponse> handleAsync0(final GridRestRequest req) { - if (!busyLock.tryReadLock()) - return new GridFinishedFuture<>(ctx, - new IgniteCheckedException("Failed to handle request (received request while stopping grid).")); - - try { - final GridWorkerFuture<GridRestResponse> fut = new GridWorkerFuture<>(ctx); - - workersCnt.increment(); - - GridWorker w = new GridWorker(ctx.gridName(), "rest-proc-worker", log) { - @Override protected void body() { - try { - IgniteFuture<GridRestResponse> res = handleRequest(req); - - res.listenAsync(new IgniteInClosure<IgniteFuture<GridRestResponse>>() { - @Override public void apply(IgniteFuture<GridRestResponse> f) { - try { - fut.onDone(f.get()); - } - catch (IgniteCheckedException e) { - fut.onDone(e); - } - } - }); - } - catch (Throwable e) { - if (e instanceof Error) - U.error(log, "Client request execution failed with error.", e); - - fut.onDone(U.cast(e)); - } - finally { - workersCnt.decrement(); - } - } - }; - - fut.setWorker(w); - - try { - config().getRestExecutorService().execute(w); - } - catch (RejectedExecutionException e) { - U.error(log, "Failed to execute worker due to execution rejection " + - "(increase upper bound on REST executor service). " + - "Will attempt to process request in the current thread instead.", e); - - w.run(); - } - - return fut; - } - finally { - busyLock.readUnlock(); - } - } - - /** - * @param req Request. - * @return Future. - */ - private IgniteFuture<GridRestResponse> handleRequest(final GridRestRequest req) { - if (startLatch.getCount() > 0) { - try { - startLatch.await(); - } - catch (InterruptedException e) { - return new GridFinishedFuture<>(ctx, new IgniteCheckedException("Failed to handle request " + - "(protocol handler was interrupted when awaiting grid start).", e)); - } - } - - if (log.isDebugEnabled()) - log.debug("Received request from client: " + req); - - GridSecurityContext subjCtx = null; - - try { - subjCtx = authenticate(req); - - authorize(req, subjCtx); - } - catch (GridSecurityException e) { - assert subjCtx != null; - - GridRestResponse res = new GridRestResponse(STATUS_SECURITY_CHECK_FAILED, e.getMessage()); - - if (ctx.isEnterprise()) { - try { - res.sessionTokenBytes(updateSessionToken(req, subjCtx)); - } - catch (IgniteCheckedException e1) { - U.warn(log, "Cannot update response session token: " + e1.getMessage()); - } - } - - return new GridFinishedFuture<>(ctx, res); - } - catch (IgniteCheckedException e) { - return new GridFinishedFuture<>(ctx, new GridRestResponse(STATUS_AUTH_FAILED, e.getMessage())); - } - - interceptRequest(req); - - GridRestCommandHandler hnd = handlers.get(req.command()); - - IgniteFuture<GridRestResponse> res = hnd == null ? null : hnd.handleAsync(req); - - if (res == null) - return new GridFinishedFuture<>(ctx, - new IgniteCheckedException("Failed to find registered handler for command: " + req.command())); - - final GridSecurityContext subjCtx0 = subjCtx; - - return res.chain(new C1<IgniteFuture<GridRestResponse>, GridRestResponse>() { - @Override public GridRestResponse apply(IgniteFuture<GridRestResponse> f) { - GridRestResponse res; - - try { - res = f.get(); - } - catch (Exception e) { - LT.error(log, e, "Failed to handle request: " + req.command()); - - if (log.isDebugEnabled()) - log.debug("Failed to handle request [req=" + req + ", e=" + e + "]"); - - res = new GridRestResponse(STATUS_FAILED, e.getMessage()); - } - - assert res != null; - - if (ctx.secureSession().enabled()) { - try { - res.sessionTokenBytes(updateSessionToken(req, subjCtx0)); - } - catch (IgniteCheckedException e) { - U.warn(log, "Cannot update response session token: " + e.getMessage()); - } - } - - interceptResponse(res, req); - - return res; - } - }); - } - - /** - * @param ctx Context. - */ - public GridRestProcessor(GridKernalContext ctx) { - super(ctx); - } - - /** {@inheritDoc} */ - @Override public void start() throws IgniteCheckedException { - if (isRestEnabled()) { - // Register handlers. - addHandler(new GridCacheCommandHandler(ctx)); - addHandler(new GridCacheQueryCommandHandler(ctx)); - addHandler(new GridTaskCommandHandler(ctx)); - addHandler(new GridTopologyCommandHandler(ctx)); - addHandler(new GridVersionCommandHandler(ctx)); - addHandler(new GridLogCommandHandler(ctx)); - addHandler(new GridPortableMetadataHandler(ctx)); - - // Start protocols. - startTcpProtocol(); - startHttpProtocol(); - } - } - - /** {@inheritDoc} */ - @Override public void onKernalStart() throws IgniteCheckedException { - if (isRestEnabled()) { - for (GridRestProtocol proto : protos) - proto.onKernalStart(); - - startLatch.countDown(); - - if (log.isDebugEnabled()) - log.debug("REST processor started."); - } - } - - /** {@inheritDoc} */ - @SuppressWarnings("BusyWait") - @Override public void onKernalStop(boolean cancel) { - if (isRestEnabled()) { - busyLock.writeLock(); - - boolean interrupted = Thread.interrupted(); - - while (workersCnt.sum() != 0) { - try { - Thread.sleep(200); - } - catch (InterruptedException ignored) { - interrupted = true; - } - } - - if (interrupted) - Thread.currentThread().interrupt(); - - for (GridRestProtocol proto : protos) - proto.stop(); - - // Safety. - startLatch.countDown(); - - if (log.isDebugEnabled()) - log.debug("REST processor stopped."); - } - } - - /** {@inheritDoc} */ - @Override public void addAttributes(Map<String, Object> attrs) throws IgniteCheckedException { - for (GridRestProtocol proto : protos) { - Collection<IgniteBiTuple<String, Object>> props = proto.getProperties(); - - if (props != null) { - for (IgniteBiTuple<String, Object> p : props) { - String key = p.getKey(); - - if (key == null) - continue; - - if (attrs.containsKey(key)) - throw new IgniteCheckedException( - "Node attribute collision for attribute [processor=GridRestProcessor, attr=" + key + ']'); - - attrs.put(key, p.getValue()); - } - } - } - } - - /** - * Applies {@link org.apache.ignite.configuration.ClientMessageInterceptor} - * from {@link org.apache.ignite.configuration.ClientConnectionConfiguration#getClientMessageInterceptor()} - * to all user parameters in the request. - * - * @param req Client request. - */ - private void interceptRequest(GridRestRequest req) { - ClientMessageInterceptor interceptor = config().getClientMessageInterceptor(); - - if (interceptor == null) - return; - - if (req instanceof GridRestCacheRequest) { - GridRestCacheRequest req0 = (GridRestCacheRequest) req; - - req0.key(interceptor.onReceive(req0.key())); - req0.value(interceptor.onReceive(req0.value())); - req0.value2(interceptor.onReceive(req0.value2())); - - Map<Object, Object> oldVals = req0.values(); - - if (oldVals != null) { - Map<Object, Object> newVals = U.newHashMap(oldVals.size()); - - for (Map.Entry<Object, Object> e : oldVals.entrySet()) - newVals.put(interceptor.onReceive(e.getKey()), interceptor.onReceive(e.getValue())); - - req0.values(U.sealMap(newVals)); - } - } - else if (req instanceof GridRestTaskRequest) { - GridRestTaskRequest req0 = (GridRestTaskRequest) req; - - List<Object> oldParams = req0.params(); - - if (oldParams != null) { - Collection<Object> newParams = new ArrayList<>(oldParams.size()); - - for (Object o : oldParams) - newParams.add(interceptor.onReceive(o)); - - req0.params(U.sealList(newParams)); - } - } - } - - /** - * Applies {@link org.apache.ignite.configuration.ClientMessageInterceptor} from - * {@link org.apache.ignite.configuration.ClientConnectionConfiguration#getClientMessageInterceptor()} - * to all user objects in the response. - * - * @param res Response. - * @param req Request. - */ - private void interceptResponse(GridRestResponse res, GridRestRequest req) { - ClientMessageInterceptor interceptor = config().getClientMessageInterceptor(); - - if (interceptor != null && res.getResponse() != null) { - switch (req.command()) { - case CACHE_GET: - case CACHE_GET_ALL: - case CACHE_PUT: - case CACHE_ADD: - case CACHE_PUT_ALL: - case CACHE_REMOVE: - case CACHE_REMOVE_ALL: - case CACHE_REPLACE: - case CACHE_INCREMENT: - case CACHE_DECREMENT: - case CACHE_CAS: - case CACHE_APPEND: - case CACHE_PREPEND: - res.setResponse(interceptSendObject(res.getResponse(), interceptor)); - - break; - - case EXE: - if (res.getResponse() instanceof GridClientTaskResultBean) { - GridClientTaskResultBean taskRes = (GridClientTaskResultBean)res.getResponse(); - - taskRes.setResult(interceptor.onSend(taskRes.getResult())); - } - - break; - - default: - break; - } - } - } - - /** - * Applies interceptor to a response object. - * Specially handler {@link Map} and {@link Collection} responses. - * - * @param obj Response object. - * @param interceptor Interceptor to apply. - * @return Intercepted object. - */ - private static Object interceptSendObject(Object obj, ClientMessageInterceptor interceptor) { - if (obj instanceof Map) { - Map<Object, Object> original = (Map<Object, Object>)obj; - - Map<Object, Object> m = new HashMap<>(); - - for (Map.Entry e : original.entrySet()) - m.put(interceptor.onSend(e.getKey()), interceptor.onSend(e.getValue())); - - return m; - } - else if (obj instanceof Collection) { - Collection<Object> original = (Collection<Object>)obj; - - Collection<Object> c = new ArrayList<>(original.size()); - - for (Object e : original) - c.add(interceptor.onSend(e)); - - return c; - } - else - return interceptor.onSend(obj); - } - - /** - * Authenticates remote client. - * - * @param req Request to authenticate. - * @return Authentication subject context. - * @throws IgniteCheckedException If authentication failed. - */ - private GridSecurityContext authenticate(GridRestRequest req) throws IgniteCheckedException { - UUID clientId = req.clientId(); - - byte[] sesTok = req.sessionToken(); - - // Validate session. - if (sesTok != null) { - // Session is still valid. - GridSecureSession ses = ctx.secureSession().validateSession(REMOTE_CLIENT, clientId, sesTok, null); - - if (ses != null) - // Session is still valid. - return ses.authenticationSubjectContext(); - } - - // Authenticate client if invalid session. - AuthenticationContextAdapter authCtx = new AuthenticationContextAdapter(); - - authCtx.subjectType(REMOTE_CLIENT); - authCtx.subjectId(req.clientId()); - - GridSecurityCredentials cred; - - if (req.credentials() instanceof GridSecurityCredentials) - cred = (GridSecurityCredentials)req.credentials(); - else if (req.credentials() instanceof String) { - String credStr = (String)req.credentials(); - - int idx = credStr.indexOf(':'); - - cred = idx >= 0 && idx < credStr.length() ? - new GridSecurityCredentials(credStr.substring(0, idx), credStr.substring(idx + 1)) : - new GridSecurityCredentials(credStr, null); - } - else { - cred = new GridSecurityCredentials(); - - cred.setUserObject(req.credentials()); - } - - authCtx.address(req.address()); - - authCtx.credentials(cred); - - GridSecurityContext subjCtx = ctx.security().authenticate(authCtx); - - if (subjCtx == null) { - if (req.credentials() == null) - throw new IgniteCheckedException("Failed to authenticate remote client (secure session SPI not set?): " + req); - else - throw new IgniteCheckedException("Failed to authenticate remote client (invalid credentials?): " + req); - } - - return subjCtx; - } - - /** - * Update session token to actual state. - * - * @param req Grid est request. - * @param subjCtx Authentication subject context. - * @return Valid session token. - * @throws IgniteCheckedException If session token update process failed. - */ - private byte[] updateSessionToken(GridRestRequest req, GridSecurityContext subjCtx) throws IgniteCheckedException { - // Update token from request to actual state. - byte[] sesTok = ctx.secureSession().updateSession(REMOTE_CLIENT, req.clientId(), subjCtx, null); - - // Validate token has been created. - if (sesTok == null) - throw new IgniteCheckedException("Cannot create session token (is secure session SPI set?)."); - - return sesTok; - } - - /** - * @param req REST request. - * @param sCtx Security context. - * @throws GridSecurityException If authorization failed. - */ - private void authorize(GridRestRequest req, GridSecurityContext sCtx) throws GridSecurityException { - GridSecurityPermission perm = null; - String name = null; - - switch (req.command()) { - case CACHE_GET: - case CACHE_GET_ALL: - perm = GridSecurityPermission.CACHE_READ; - name = ((GridRestCacheRequest)req).cacheName(); - - break; - - case CACHE_QUERY_EXECUTE: - case CACHE_QUERY_FETCH: - case CACHE_QUERY_REBUILD_INDEXES: - perm = GridSecurityPermission.CACHE_READ; - name = ((GridRestCacheQueryRequest)req).cacheName(); - - break; - - case CACHE_PUT: - case CACHE_ADD: - case CACHE_PUT_ALL: - case CACHE_REPLACE: - case CACHE_INCREMENT: - case CACHE_DECREMENT: - case CACHE_CAS: - case CACHE_APPEND: - case CACHE_PREPEND: - perm = GridSecurityPermission.CACHE_PUT; - name = ((GridRestCacheRequest)req).cacheName(); - - break; - - case CACHE_REMOVE: - case CACHE_REMOVE_ALL: - perm = GridSecurityPermission.CACHE_REMOVE; - name = ((GridRestCacheRequest)req).cacheName(); - - break; - - case EXE: - case RESULT: - perm = GridSecurityPermission.TASK_EXECUTE; - name = ((GridRestTaskRequest)req).taskName(); - - break; - - case CACHE_METRICS: - case TOPOLOGY: - case NODE: - case VERSION: - case LOG: - case NOOP: - case QUIT: - case GET_PORTABLE_METADATA: - case PUT_PORTABLE_METADATA: - break; - - default: - throw new AssertionError("Unexpected command: " + req.command()); - } - - if (perm != null) - ctx.security().authorize(name, perm, sCtx); - } - - /** - * - * @return Whether or not REST is enabled. - */ - private boolean isRestEnabled() { - return !ctx.config().isDaemon() && ctx.config().getClientConnectionConfiguration() != null; - } - - /** - * @param hnd Command handler. - */ - private void addHandler(GridRestCommandHandler hnd) { - assert !handlers.containsValue(hnd); - - if (log.isDebugEnabled()) - log.debug("Added REST command handler: " + hnd); - - for (GridRestCommand cmd : hnd.supportedCommands()) { - assert !handlers.containsKey(cmd); - - handlers.put(cmd, hnd); - } - } - - /** - * Starts TCP protocol. - * - * @throws IgniteCheckedException In case of error. - */ - private void startTcpProtocol() throws IgniteCheckedException { - startProtocol(new GridTcpRestProtocol(ctx)); - } - - /** - * Starts HTTP protocol if it exists on classpath. - * - * @throws IgniteCheckedException In case of error. - */ - private void startHttpProtocol() throws IgniteCheckedException { - try { - Class<?> cls = Class.forName(HTTP_PROTO_CLS); - - Constructor<?> ctor = cls.getConstructor(GridKernalContext.class); - - GridRestProtocol proto = (GridRestProtocol)ctor.newInstance(ctx); - - startProtocol(proto); - } - catch (ClassNotFoundException ignored) { - U.quietAndWarn(log, "Failed to initialize HTTP REST protocol (consider adding gridgain-rest-http " + - "module to classpath)."); - } - catch (NoSuchMethodException | InvocationTargetException | InstantiationException | IllegalAccessException e) { - throw new IgniteCheckedException("Failed to initialize HTTP REST protocol.", e); - } - } - - /** - * @return Client configuration. - */ - private ClientConnectionConfiguration config() { - return ctx.config().getClientConnectionConfiguration(); - } - - /** - * @param proto Protocol. - * @throws IgniteCheckedException If protocol initialization failed. - */ - private void startProtocol(GridRestProtocol proto) throws IgniteCheckedException { - assert proto != null; - assert !protos.contains(proto); - - protos.add(proto); - - proto.start(protoHnd); - - if (log.isDebugEnabled()) - log.debug("Added REST protocol: " + proto); - } - - /** {@inheritDoc} */ - @Override public void printMemoryStats() { - X.println(">>>"); - X.println(">>> REST processor memory stats [grid=" + ctx.gridName() + ']'); - X.println(">>> protosSize: " + protos.size()); - X.println(">>> handlersSize: " + handlers.size()); - } -}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocol.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocol.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocol.java deleted file mode 100644 index 0ad028c..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocol.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest; - -import org.apache.ignite.*; -import org.apache.ignite.lang.*; - -import java.util.*; - -/** - * REST protocol. - */ -public interface GridRestProtocol { - /** - * @return Protocol name. - */ - public abstract String name(); - - /** - * Returns protocol properties for setting node attributes. Has meaningful result - * only after protocol start. - * - * @return Protocol properties. - */ - public abstract Collection<IgniteBiTuple<String, Object>> getProperties(); - - /** - * Starts protocol. - * - * @param hnd Command handler. - * @throws IgniteCheckedException If failed. - */ - public abstract void start(GridRestProtocolHandler hnd) throws IgniteCheckedException; - - /** - * Grid start callback. - */ - public abstract void onKernalStart(); - - /** - * Stops protocol. - */ - public abstract void stop(); -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocolHandler.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocolHandler.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocolHandler.java deleted file mode 100644 index 1d78f96..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocolHandler.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest; - -import org.apache.ignite.*; -import org.apache.ignite.lang.*; -import org.gridgain.grid.kernal.processors.rest.request.*; - -/** - * Command protocol handler. - */ -public interface GridRestProtocolHandler { - /** - * @param req Request. - * @return Response. - * @throws IgniteCheckedException In case of error. - */ - public GridRestResponse handle(GridRestRequest req) throws IgniteCheckedException; - - /** - * @param req Request. - * @return Future. - */ - public IgniteFuture<GridRestResponse> handleAsync(GridRestRequest req); -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestResponse.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestResponse.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestResponse.java deleted file mode 100644 index b069c24..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestResponse.java +++ /dev/null @@ -1,177 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest; - -import org.apache.ignite.internal.util.typedef.internal.*; -import org.apache.ignite.internal.util.tostring.*; -import org.jetbrains.annotations.*; - -import java.io.*; - -/** - * JSON response. Getters and setters must conform to JavaBean standard. - */ -public class GridRestResponse implements Externalizable { - /** */ - private static final long serialVersionUID = 0L; - - /** Command succeeded. */ - public static final int STATUS_SUCCESS = 0; - - /** Command failed. */ - public static final int STATUS_FAILED = 1; - - /** Authentication failure. */ - public static final int STATUS_AUTH_FAILED = 2; - - /** Security check failed. */ - public static final int STATUS_SECURITY_CHECK_FAILED = 3; - - /** Success status. */ - @SuppressWarnings("RedundantFieldInitialization") - private int successStatus = STATUS_SUCCESS; - - /** Session token. */ - private byte[] sesTokBytes; - - /** Session token string representation. */ - private String sesTokStr; - - /** Error. */ - private String err; - - /** Response object. */ - @GridToStringInclude - private Object obj; - - /** - * - */ - public GridRestResponse() { - // No-op. - } - - /** - * Constructs successful rest response. - * - * @param obj Response object. - */ - public GridRestResponse(Object obj) { - successStatus = STATUS_SUCCESS; - this.obj = obj; - } - - /** - * Constructs failed rest response. - * - * @param status Response status. - * @param err Error, {@code null} if success is {@code true}. - */ - public GridRestResponse(int status, @Nullable String err) { - assert status != STATUS_SUCCESS; - - successStatus = status; - this.err = err; - } - - /** - * @return Success flag. - */ - public int getSuccessStatus() { - return successStatus; - } - - /** - * @return Response object. - */ - public Object getResponse() { - return obj; - } - - /** - * @param obj Response object. - */ - public void setResponse(@Nullable Object obj) { - this.obj = obj; - } - - /** - * @return Error. - */ - public String getError() { - return err; - } - - /** - * @param err Error. - */ - public void setError(String err) { - this.err = err; - } - - /** - * @return Session token for remote client. - */ - public byte[] sessionTokenBytes() { - return sesTokBytes; - } - - /** - * @param sesTokBytes Session token for remote client. - */ - public void sessionTokenBytes(@Nullable byte[] sesTokBytes) { - this.sesTokBytes = sesTokBytes; - } - - /** - * @return String representation of session token. - */ - public String getSessionToken() { - return sesTokStr; - } - - /** - * @param sesTokStr String representation of session token. - */ - public void setSessionToken(@Nullable String sesTokStr) { - this.sesTokStr = sesTokStr; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(GridRestResponse.class, this); - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - out.writeInt(successStatus); - U.writeByteArray(out, sesTokBytes); - U.writeString(out, sesTokStr); - U.writeString(out, err); - out.writeObject(obj); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - successStatus = in.readInt(); - sesTokBytes = U.readByteArray(in); - sesTokStr = U.readString(in); - err = U.readString(in); - obj = in.readObject(); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientAbstractMessage.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientAbstractMessage.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientAbstractMessage.java deleted file mode 100644 index fdff874..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientAbstractMessage.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest.client.message; - -import org.apache.ignite.portables.*; -import org.apache.ignite.internal.util.typedef.internal.*; - -import java.io.*; -import java.util.*; - -/** - * This class provides implementation for commit message fields and cannot be used directly. - */ -public abstract class GridClientAbstractMessage implements GridClientMessage, Externalizable, PortableMarshalAware { - /** */ - private static final long serialVersionUID = 0L; - - /** Request ID (transient). */ - private transient long reqId; - - /** Client ID (transient). */ - private transient UUID id; - - /** Node ID (transient). */ - private transient UUID destId; - - /** Session token. */ - private byte[] sesTok; - - /** {@inheritDoc} */ - @Override public long requestId() { - return reqId; - } - - /** {@inheritDoc} */ - @Override public void requestId(long reqId) { - this.reqId = reqId; - } - - /** {@inheritDoc} */ - @Override public UUID clientId() { - return id; - } - - /** {@inheritDoc} */ - @Override public void clientId(UUID id) { - this.id = id; - } - - /** {@inheritDoc} */ - @Override public UUID destinationId() { - return destId; - } - - /** {@inheritDoc} */ - @Override public void destinationId(UUID destId) { - this.destId = destId; - } - - /** - * @return Session token - */ - @Override public byte[] sessionToken() { - return sesTok; - } - - /** - * @param sesTok Session token. - */ - @Override public void sessionToken(byte[] sesTok) { - this.sesTok = sesTok; - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - U.writeByteArray(out, sesTok); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - sesTok = U.readByteArray(in); - } - - /** {@inheritDoc} */ - @Override public void writePortable(PortableWriter writer) throws PortableException { - PortableRawWriter raw = writer.rawWriter(); - - raw.writeByteArray(sesTok); - } - - /** {@inheritDoc} */ - @Override public void readPortable(PortableReader reader) throws PortableException { - PortableRawReader raw = reader.rawReader(); - - sesTok = raw.readByteArray(); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(GridClientAbstractMessage.class, this, super.toString()); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientAuthenticationRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientAuthenticationRequest.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientAuthenticationRequest.java deleted file mode 100644 index a79ebe6..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientAuthenticationRequest.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest.client.message; - -import org.apache.ignite.portables.*; -import org.apache.ignite.internal.util.typedef.internal.*; - -import java.io.*; - -/** - * Client authentication request. - */ -public class GridClientAuthenticationRequest extends GridClientAbstractMessage { - /** */ - private static final long serialVersionUID = 0L; - - /** Credentials. */ - private Object cred; - - /** - * @return Credentials object. - */ - public Object credentials() { - return cred; - } - - /** - * @param cred Credentials object. - */ - public void credentials(Object cred) { - this.cred = cred; - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - super.writeExternal(out); - - out.writeObject(cred); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - super.readExternal(in); - - cred = in.readObject(); - } - - /** {@inheritDoc} */ - @Override public void writePortable(PortableWriter writer) throws PortableException { - super.writePortable(writer); - - PortableRawWriter raw = writer.rawWriter(); - - raw.writeObject(cred); - } - - /** {@inheritDoc} */ - @Override public void readPortable(PortableReader reader) throws PortableException { - super.readPortable(reader); - - PortableRawReader raw = reader.rawReader(); - - cred = raw.readObject(); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(GridClientAuthenticationRequest.class, this, super.toString()); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientCacheQueryRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientCacheQueryRequest.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientCacheQueryRequest.java deleted file mode 100644 index 1f20c5e..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientCacheQueryRequest.java +++ /dev/null @@ -1,414 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest.client.message; - -import org.apache.ignite.portables.*; -import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.jetbrains.annotations.*; - -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; - -/** - * Cache query request. - */ -public class GridClientCacheQueryRequest extends GridClientAbstractMessage { - /** */ - private static final long serialVersionUID = 0L; - - /** - * Available query operations. - */ - @SuppressWarnings("PublicInnerClass") - public enum GridQueryOperation { - /** First time query execution. Will assign query ID for executed query. */ - EXECUTE, - - /** Fetch next data page. */ - FETCH, - - /** Rebuild one or all indexes. */ - REBUILD_INDEXES; - - /** Enumerated values. */ - private static final GridQueryOperation[] VALS = values(); - - /** - * Efficiently gets enumerated value from its ordinal. - * - * @param ord Ordinal value. - * @return Enumerated value or {@code null} if ordinal out of range. - */ - @Nullable public static GridQueryOperation fromOrdinal(int ord) { - return ord >= 0 && ord < VALS.length ? VALS[ord] : null; - } - } - - /** - * Query types. - */ - @SuppressWarnings("PublicInnerClass") - public enum GridQueryType { - /** SQL query. */ - SQL, - - /** SQL fields query. */ - SQL_FIELDS, - - /** Full text query. */ - FULL_TEXT, - - /** Scan query. */ - SCAN; - - /** Enumerated values. */ - private static final GridQueryType[] VALS = values(); - - /** - * Efficiently gets enumerated value from its ordinal. - * - * @param ord Ordinal value. - * @return Enumerated value or {@code null} if ordinal out of range. - */ - @Nullable public static GridQueryType fromOrdinal(int ord) { - return ord >= 0 && ord < VALS.length ? VALS[ord] : null; - } - } - - /** Query ID linked to destination node ID. */ - private long qryId; - - /** Query operation. */ - private GridQueryOperation op; - - /** Cache name. */ - private String cacheName; - - /** Query type. */ - private GridQueryType type; - - /** Query clause. */ - private String clause; - - /** Page size. */ - private int pageSize; - - /** Timeout. */ - private long timeout; - - /** Include backups flag. */ - private boolean includeBackups; - - /** Enable dedup flag. */ - private boolean enableDedup; - - /** Keep portable flag. */ - private boolean keepPortable; - - /** Class name. */ - private String clsName; - - /** Remote reducer class name. */ - private String rmtReducerClsName; - - /** Remote transformer class name. */ - private String rmtTransformerClsName; - - /** Query arguments. */ - private Object[] qryArgs; - - /** - * @return Query ID. - */ - public long queryId() { - return qryId; - } - - /** - * @param qryId Query ID. - */ - public void queryId(long qryId) { - this.qryId = qryId; - } - - /** - * @return Operation. - */ - public GridQueryOperation operation() { - return op; - } - - /** - * @param op Operation. - */ - public void operation(GridQueryOperation op) { - this.op = op; - } - - /** - * @return Cache name. - */ - public String cacheName() { - return cacheName; - } - - /** - * @param cacheName Cache name. - */ - public void cacheName(String cacheName) { - this.cacheName = cacheName; - } - - /** - * @return Query type. - */ - public GridQueryType type() { - return type; - } - - /** - * @param type Query type. - */ - public void type(GridQueryType type) { - this.type = type; - } - - /** - * @return Query clause. - */ - public String clause() { - return clause; - } - - /** - * @param clause Query clause. - */ - public void clause(String clause) { - this.clause = clause; - } - - /** - * @return Page size. - */ - public int pageSize() { - return pageSize; - } - - /** - * @param pageSize Page size. - */ - public void pageSize(int pageSize) { - this.pageSize = pageSize; - } - - /** - * @return Query timeout. - */ - public long timeout() { - return timeout; - } - - /** - * @param timeout Query timeout. - */ - public void timeout(long timeout) { - this.timeout = timeout; - } - - /** - * @return Include backups flag. - */ - public boolean includeBackups() { - return includeBackups; - } - - /** - * @param includeBackups Include backups flag. - */ - public void includeBackups(boolean includeBackups) { - this.includeBackups = includeBackups; - } - - /** - * @return Enable de-duplication flag. - */ - public boolean enableDedup() { - return enableDedup; - } - - /** - * @return Keep portable flag. - */ - public boolean keepPortable() { - return keepPortable; - } - - /** - * @param enableDedup Enable de-duplication flag. - */ - public void enableDedup(boolean enableDedup) { - this.enableDedup = enableDedup; - } - - /** - * @return Class name. - */ - public String className() { - return clsName; - } - - /** - * @param clsName Class name. - */ - public void className(String clsName) { - this.clsName = clsName; - } - - /** - * @return Remote reducer class name. - */ - public String remoteReducerClassName() { - return rmtReducerClsName; - } - - /** - * @param rmtReducerClsName Remote reducer class name. - */ - public void remoteReducerClassName(String rmtReducerClsName) { - this.rmtReducerClsName = rmtReducerClsName; - } - - /** - * @return Remote transformer class name. - */ - public String remoteTransformerClassName() { - return rmtTransformerClsName; - } - - /** - * @param rmtTransformerClsName Remote transformer class name. - */ - public void remoteTransformerClassName(String rmtTransformerClsName) { - this.rmtTransformerClsName = rmtTransformerClsName; - } - - /** - * @return Query arguments. - */ - public Object[] queryArguments() { - return qryArgs; - } - - /** - * @param qryArgs Query arguments. - */ - public void queryArguments(Object[] qryArgs) { - this.qryArgs = qryArgs; - } - - /** {@inheritDoc} */ - @Override public void readPortable(PortableReader reader) throws PortableException { - super.readPortable(reader); - - PortableRawReader rawReader = reader.rawReader(); - - qryId = rawReader.readLong(); - op = GridQueryOperation.fromOrdinal(rawReader.readInt()); - type = GridQueryType.fromOrdinal(rawReader.readInt()); - cacheName = rawReader.readString(); - clause = rawReader.readString(); - pageSize = rawReader.readInt(); - timeout = rawReader.readLong(); - includeBackups = rawReader.readBoolean(); - enableDedup = rawReader.readBoolean(); - keepPortable = rawReader.readBoolean(); - clsName = rawReader.readString(); - rmtReducerClsName = rawReader.readString(); - rmtTransformerClsName = rawReader.readString(); - qryArgs = rawReader.readObjectArray(); - } - - /** {@inheritDoc} */ - @Override public void writePortable(PortableWriter writer) throws PortableException { - super.writePortable(writer); - - PortableRawWriter rawWriter = writer.rawWriter(); - - rawWriter.writeLong(qryId); - rawWriter.writeInt(op.ordinal()); - rawWriter.writeInt(type == null ? -1 : type.ordinal()); - rawWriter.writeString(cacheName); - rawWriter.writeString(clause); - rawWriter.writeInt(pageSize); - rawWriter.writeLong(timeout); - rawWriter.writeBoolean(includeBackups); - rawWriter.writeBoolean(enableDedup); - rawWriter.writeBoolean(keepPortable); - rawWriter.writeString(clsName); - rawWriter.writeString(rmtReducerClsName); - rawWriter.writeString(rmtTransformerClsName); - rawWriter.writeObjectArray(qryArgs); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - super.readExternal(in); - - qryId = in.readLong(); - op = GridQueryOperation.fromOrdinal(in.readInt()); - type = GridQueryType.fromOrdinal(in.readInt()); - cacheName = U.readString(in); - clause = U.readString(in); - pageSize = in.readInt(); - timeout = in.readLong(); - includeBackups = in.readBoolean(); - enableDedup = in.readBoolean(); - keepPortable = in.readBoolean(); - clsName = U.readString(in); - rmtReducerClsName = U.readString(in); - rmtTransformerClsName = U.readString(in); - qryArgs = U.readArray(in); - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - super.writeExternal(out); - - out.writeLong(qryId); - out.writeInt(op.ordinal()); - out.writeInt(type == null ? -1 : type.ordinal()); - U.writeString(out, cacheName); - U.writeString(out, clause); - out.writeInt(pageSize); - out.writeLong(timeout); - out.writeBoolean(includeBackups); - out.writeBoolean(enableDedup); - out.writeBoolean(keepPortable); - U.writeString(out, clsName); - U.writeString(out, rmtReducerClsName); - U.writeString(out, rmtTransformerClsName); - U.writeArray(out, qryArgs); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(GridClientCacheQueryRequest.class, this); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientCacheRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientCacheRequest.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientCacheRequest.java deleted file mode 100644 index 0502890..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientCacheRequest.java +++ /dev/null @@ -1,319 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest.client.message; - -import org.apache.ignite.internal.util.portable.*; -import org.apache.ignite.portables.*; -import org.apache.ignite.internal.util.typedef.internal.*; -import org.jetbrains.annotations.*; - -import java.io.*; -import java.util.*; - -/** - * Generic cache request. - */ -public class GridClientCacheRequest extends GridClientAbstractMessage { - /** */ - private static final long serialVersionUID = 0L; - - /** - * Available cache operations. - */ - @SuppressWarnings("PublicInnerClass") - public enum GridCacheOperation { - /** Cache put. */ - PUT, - - /** Cache put all. */ - PUT_ALL, - - /** Cache get. */ - GET, - - /** Cache get all. */ - GET_ALL, - - /** Cache remove. */ - RMV, - - /** Cache remove all. */ - RMV_ALL, - - /** Cache replace (put only if exists). */ - REPLACE, - - /** Cache compare and set. */ - CAS, - - /** Cache metrics request. */ - METRICS, - - /** Append requested value to already cached one. */ - APPEND, - - /** Prepend requested value to already cached one. */ - PREPEND; - - /** Enumerated values. */ - private static final GridCacheOperation[] VALS = values(); - - /** - * Efficiently gets enumerated value from its ordinal. - * - * @param ord Ordinal value. - * @return Enumerated value or {@code null} if ordinal out of range. - */ - @Nullable public static GridCacheOperation fromOrdinal(int ord) { - return ord >= 0 && ord < VALS.length ? VALS[ord] : null; - } - } - - /** Requested cache operation. */ - private GridCacheOperation op; - - /** Cache name. */ - private String cacheName; - - /** Key */ - private Object key; - - /** Value (expected value for CAS). */ - private Object val; - - /** New value for CAS. */ - private Object val2; - - /** Keys and values for put all, get all, remove all operations. */ - private Map<Object, Object> vals; - - /** Bit map of cache flags to be enabled on cache projection */ - private int cacheFlagsOn; - - /** - * Constructor for {@link Externalizable}. - */ - public GridClientCacheRequest() { - // No-op. - } - - /** - * Creates grid cache request. - * - * @param op Requested operation. - */ - public GridClientCacheRequest(GridCacheOperation op) { - this.op = op; - } - - /** - * @return Requested operation. - */ - public GridCacheOperation operation() { - return op; - } - - /** - * Gets cache name. - * - * @return Cache name, or {@code null} if not set. - */ - public String cacheName() { - return cacheName; - } - - /** - * Gets cache name. - * - * @param cacheName Cache name. - */ - public void cacheName(String cacheName) { - this.cacheName = cacheName; - } - - /** - * @return Key. - */ - public Object key() { - return key; - } - - /** - * @param key Key. - */ - public void key(Object key) { - this.key = key; - } - - /** - * @return Value 1. - */ - public Object value() { - return val; - } - - /** - * @param val Value 1. - */ - public void value(Object val) { - this.val = val; - } - - /** - * @return Value 2. - */ - public Object value2() { - return val2; - } - - /** - * @param val2 Value 2. - */ - public void value2(Object val2) { - this.val2 = val2; - } - - /** - * @return Values map for batch operations. - */ - public Map<Object, Object> values() { - return vals; - } - - /** - * @param vals Values map for batch operations. - */ - public void values(Map<Object, Object> vals) { - this.vals = vals; - } - - /** - * @param keys Keys collection - */ - public void keys(Iterable<Object> keys) { - vals = new HashMap<>(); - - for (Object k : keys) - vals.put(k, null); - } - - /** - * Set cache flags bit map. - * - * @param cacheFlagsOn Bit representation of cache flags. - */ - public void cacheFlagsOn(int cacheFlagsOn) { - this.cacheFlagsOn = cacheFlagsOn; - } - - /** - * Get cache flags bit map. - * @return Bit representation of cache flags. - */ - public int cacheFlagsOn() { - return cacheFlagsOn; - } - - /** {@inheritDoc} */ - @Override public void writePortable(PortableWriter writer) throws PortableException { - super.writePortable(writer); - - PortableRawWriterEx raw = (PortableRawWriterEx)writer.rawWriter(); - - raw.writeInt(op.ordinal()); - raw.writeString(cacheName); - raw.writeInt(cacheFlagsOn); - raw.writeObjectDetached(key); - raw.writeObjectDetached(val); - raw.writeObjectDetached(val2); - - raw.writeInt(vals != null ? vals.size() : -1); - - if (vals != null) { - for (Map.Entry<Object, Object> e : vals.entrySet()) { - raw.writeObjectDetached(e.getKey()); - raw.writeObjectDetached(e.getValue()); - } - } - } - - /** {@inheritDoc} */ - @Override public void readPortable(PortableReader reader) throws PortableException { - super.readPortable(reader); - - PortableRawReaderEx raw = (PortableRawReaderEx)reader.rawReader(); - - op = GridCacheOperation.fromOrdinal(raw.readInt()); - cacheName = raw.readString(); - cacheFlagsOn = raw.readInt(); - key = raw.readObjectDetached(); - val = raw.readObjectDetached(); - val2 = raw.readObjectDetached(); - - int valsSize = raw.readInt(); - - if (valsSize >= 0) { - vals = U.newHashMap(valsSize); - - for (int i = 0; i < valsSize; i++) - vals.put(raw.readObjectDetached(), raw.readObjectDetached()); - } - } - - /** {@inheritDoc} */ - @SuppressWarnings("deprecation") - @Override public void writeExternal(ObjectOutput out) throws IOException { - super.writeExternal(out); - - U.writeEnum0(out, op); - - U.writeString(out, cacheName); - - out.writeObject(key); - out.writeObject(val); - out.writeObject(val2); - - U.writeMap(out, vals); - - out.writeInt(cacheFlagsOn); - } - - /** {@inheritDoc} */ - @SuppressWarnings("deprecation") - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - super.readExternal(in); - - op = GridCacheOperation.fromOrdinal(U.readEnumOrdinal0(in)); - - cacheName = U.readString(in); - - key = in.readObject(); - val = in.readObject(); - val2 = in.readObject(); - - vals = U.readMap(in); - - cacheFlagsOn = in.readInt(); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return getClass().getSimpleName() + " [op=" + op + ", key=" + key + ", val=" + val + - ", val2=" + val2 + ", vals=" + vals + ", cacheFlagsOn=" + cacheFlagsOn + "]"; - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientGetMetaDataRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientGetMetaDataRequest.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientGetMetaDataRequest.java deleted file mode 100644 index 5b70d70..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientGetMetaDataRequest.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest.client.message; - -import org.apache.ignite.portables.*; -import org.apache.ignite.internal.util.typedef.internal.*; - -import java.util.*; - -/** - * Metadata request. - */ -public class GridClientGetMetaDataRequest extends GridClientAbstractMessage { - /** */ - private static final long serialVersionUID = 0L; - - /** */ - private Collection<Integer> typeIds; - - /** - * @return Type IDs. - */ - public Collection<Integer> typeIds() { - return typeIds; - } - - /** {@inheritDoc} */ - @Override public void writePortable(PortableWriter writer) throws PortableException { - super.writePortable(writer); - - PortableRawWriter raw = writer.rawWriter(); - - raw.writeCollection(typeIds); - } - - /** {@inheritDoc} */ - @Override public void readPortable(PortableReader reader) throws PortableException { - super.readPortable(reader); - - PortableRawReader raw = reader.rawReader(); - - typeIds = raw.readCollection(); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(GridClientGetMetaDataRequest.class, this); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeRequest.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeRequest.java deleted file mode 100644 index 37fcdcb..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeRequest.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest.client.message; - -import org.apache.ignite.internal.util.typedef.internal.*; - -import java.util.*; - -/** - * A client handshake request, containing version info and - * a marshaller ID. - * - * A handshake request structure is as follows: - * <ol> - * <li>Protocol version (2 bytes)</li> - * <li>Marshaller ID (2 bits)</li> - * <li>Reserved space (6 bits + 1 byte)</li> - * <li>Marshaller ID for backward compatibility (1 byte)</li> - * </ol> - */ -public class GridClientHandshakeRequest extends GridClientAbstractMessage { - /** */ - private static final long serialVersionUID = 0L; - - /** Packet size. */ - static final int PACKET_SIZE = 5; - - /** Protocol version. */ - private static final short PROTO_VER = 1; - - /** Handshake byte array. */ - private byte[] arr; - - /** Marshaller ID. */ - private byte marshId; - - /** - * @return Protocol version. - */ - public short version() { - return U.bytesToShort(arr, 0); - } - - /** - * @return Marshaller ID. - */ - public byte marshallerId() { - return (byte)((arr[2] & 0xff) >> 6); - } - - /** - * @param marshId Marshaller ID. - */ - public void marshallerId(byte marshId) { - assert marshId >= 0 && marshId <= 2; - - this.marshId = marshId; - } - - /** - * Sets bytes from specified buffer to a given value. - * - * @param buf Buffer. - * @param off Offset. - * @param len Length. - */ - public void putBytes(byte[] buf, int off, int len) { - if (arr == null) - arr = new byte[PACKET_SIZE]; - - U.arrayCopy(buf, 0, arr, off, len); - } - - /** - * @return Raw representation of this packet. - */ - public byte[] rawBytes() { - byte[] ret = new byte[PACKET_SIZE]; - - U.shortToBytes(PROTO_VER, ret, 0); - - ret[2] = (byte)(marshId << 6); - - ret[4] = marshId; // Marshaller ID for backward compatibility. - - return ret; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return getClass().getSimpleName() + " [arr=" + Arrays.toString(arr) + ']'; - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeRequestWrapper.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeRequestWrapper.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeRequestWrapper.java deleted file mode 100644 index f9e3398..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeRequestWrapper.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest.client.message; - -import org.apache.ignite.internal.util.direct.*; -import org.apache.ignite.internal.util.typedef.internal.*; - -import java.nio.*; - -/** - * Client handshake wrapper for direct marshalling. - */ -public class GridClientHandshakeRequestWrapper extends GridTcpCommunicationMessageAdapter { - /** */ - private static final long serialVersionUID = -5705048094821942662L; - - /** Signal char. */ - public static final byte HANDSHAKE_HEADER = (byte)0x91; - - /** Handshake bytes. */ - private byte[] bytes; - - /** - * - */ - public GridClientHandshakeRequestWrapper() { - // No-op. - } - - /** - * - * @param req Handshake request. - */ - public GridClientHandshakeRequestWrapper(GridClientHandshakeRequest req) { - bytes = req.rawBytes(); - } - - /** - * @return Handshake bytes. - */ - public byte[] bytes() { - return bytes; - } - - /** {@inheritDoc} */ - @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); - - if (!commState.typeWritten) { - if (!commState.putByte(directType())) - return false; - - commState.typeWritten = true; - } - - switch (commState.idx) { - case 0: - if (!commState.putByteArrayClient(bytes)) - return false; - - commState.idx++; - - } - - return true; - } - - /** {@inheritDoc} */ - @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); - - switch (commState.idx) { - case 0: - byte[] bytes0 = commState.getByteArrayClient(GridClientHandshakeRequest.PACKET_SIZE); - - if (bytes0 == BYTE_ARR_NOT_READ) - return false; - - bytes = bytes0; - - commState.idx++; - - } - - return true; - } - - /** {@inheritDoc} */ - @Override public byte directType() { - return HANDSHAKE_HEADER; - } - - /** {@inheritDoc} */ - @SuppressWarnings({"CloneDoesntCallSuperClone", "CloneCallsConstructors"}) - @Override public GridTcpCommunicationMessageAdapter clone() { - GridClientHandshakeRequestWrapper _clone = new GridClientHandshakeRequestWrapper(); - - clone0(_clone); - - return _clone; - } - - /** {@inheritDoc} */ - @Override protected void clone0(GridTcpCommunicationMessageAdapter _msg) { - GridClientHandshakeRequestWrapper _clone = (GridClientHandshakeRequestWrapper)_msg; - - _clone.bytes = bytes; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(GridClientHandshakeRequestWrapper.class, this); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeResponse.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeResponse.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeResponse.java deleted file mode 100644 index 96062ab..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeResponse.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest.client.message; - -import java.io.*; - -/** - * A client handshake response, containing result - * code. - */ -public class GridClientHandshakeResponse extends GridClientAbstractMessage { - /** */ - private static final long serialVersionUID = 0L; - - /** */ - public static final byte CODE_OK = 0; - - /** Response, indicating successful handshake. */ - public static final GridClientHandshakeResponse OK = new GridClientHandshakeResponse(CODE_OK); - - /** */ - private byte resCode; - - /** - * Constructor for {@link Externalizable}. - */ - public GridClientHandshakeResponse() { - // No-op. - } - - /** - * Constructor. - * - * @param resCode Result code. - */ - public GridClientHandshakeResponse(byte resCode) { - this.resCode = resCode; - } - - /** - * @return Result code. - */ - public byte resultCode() { - return resCode; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return getClass().getSimpleName() + " [resCode=" + resCode + ']'; - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeResponseWrapper.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeResponseWrapper.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeResponseWrapper.java deleted file mode 100644 index 070b4ce..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientHandshakeResponseWrapper.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest.client.message; - -import org.apache.ignite.internal.util.direct.*; -import org.apache.ignite.internal.util.typedef.internal.*; - -import java.nio.*; - -/** - * Client handshake wrapper for direct marshalling. - */ -public class GridClientHandshakeResponseWrapper extends GridTcpCommunicationMessageAdapter { - /** */ - private static final long serialVersionUID = -1529807975073967381L; - - /** */ - private byte code; - - /** - * - */ - public GridClientHandshakeResponseWrapper() { - // No-op. - } - - /** - * @param code Response code. - */ - public GridClientHandshakeResponseWrapper(byte code) { - this.code = code; - } - - /** {@inheritDoc} */ - @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); - - if (!commState.typeWritten) { - if (!commState.putByte(directType())) - return false; - - commState.typeWritten = true; - } - - return true; - } - - /** {@inheritDoc} */ - @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); - - return true; - } - - /** {@inheritDoc} */ - @Override public byte directType() { - return code; - } - - /** {@inheritDoc} */ - @SuppressWarnings({"CloneDoesntCallSuperClone", "CloneCallsConstructors"}) - @Override public GridTcpCommunicationMessageAdapter clone() { - GridClientHandshakeResponseWrapper _clone = new GridClientHandshakeResponseWrapper(); - - clone0(_clone); - - return _clone; - } - - /** {@inheritDoc} */ - @Override protected void clone0(GridTcpCommunicationMessageAdapter _msg) { - GridClientHandshakeResponseWrapper _clone = (GridClientHandshakeResponseWrapper)_msg; - - _clone.code = code; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(GridClientHandshakeResponseWrapper.class, this); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientLogRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientLogRequest.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientLogRequest.java deleted file mode 100644 index bc2932c..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientLogRequest.java +++ /dev/null @@ -1,141 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest.client.message; - -import org.apache.ignite.portables.*; -import org.apache.ignite.internal.util.typedef.internal.*; - -import java.io.*; - -/** - * Request for a log file. - */ -public class GridClientLogRequest extends GridClientAbstractMessage { - /** */ - private static final long serialVersionUID = 0L; - - /** Task name. */ - private String path; - - /** From line, inclusive, indexing from 0. */ - private int from = -1; - - /** To line, inclusive, indexing from 0, can exceed count of lines in log. */ - private int to = -1; - - /** - * @return Path to log file. - */ - public String path() { - return path; - } - - /** - * @param path Path to log file. - */ - public void path(String path) { - this.path = path; - } - - /** - * @return From line, inclusive, indexing from 0. - */ - public int from() { - return from; - } - - /** - * @param from From line, inclusive, indexing from 0. - */ - public void from(int from) { - this.from = from; - } - - /** - * @return To line, inclusive, indexing from 0. - */ - public int to() { - return to; - } - - /** - * @param to To line, inclusive, indexing from 0. - */ - public void to(int to) { - this.to = to; - } - - /** {@inheritDoc} */ - @Override public void writePortable(PortableWriter writer) throws PortableException { - super.writePortable(writer); - - PortableRawWriter raw = writer.rawWriter(); - - raw.writeString(path); - raw.writeInt(from); - raw.writeInt(to); - } - - /** {@inheritDoc} */ - @Override public void readPortable(PortableReader reader) throws PortableException { - super.readPortable(reader); - - PortableRawReader raw = reader.rawReader(); - - path = raw.readString(); - from = raw.readInt(); - to = raw.readInt(); - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - super.writeExternal(out); - - U.writeString(out, path); - - out.writeInt(from); - out.writeInt(to); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - super.readExternal(in); - - path = U.readString(in); - - from = in.readInt(); - to = in.readInt(); - } - - /** {@inheritDoc} */ - @Override public String toString() { - StringBuilder b = new StringBuilder(). - append("GridClientLogRequest [path="). - append(path); - - if (from != -1) - b.append(", from=").append(from); - - if (to != -1) - b.append(", to=").append(to); - - b.append(']'); - - return b.toString(); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0e45a2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientMessage.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientMessage.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientMessage.java deleted file mode 100644 index 6bae6bf..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/client/message/GridClientMessage.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.gridgain.grid.kernal.processors.rest.client.message; - -import java.io.*; -import java.util.*; - -/** - * Interface for all client messages. - */ -public interface GridClientMessage extends Serializable { - /** - * This method is used to match request and response messages. - * - * @return request ID. - */ - public long requestId(); - - /** - * Sets request id for outgoing packets. - * - * @param reqId request ID. - */ - public void requestId(long reqId); - - /** - * Gets client identifier from which this request comes. - * - * @return Client identifier. - */ - public UUID clientId(); - - /** - * Sets client identifier from which this request comes. - * - * @param id Client identifier. - */ - public void clientId(UUID id); - - /** - * Gets identifier of the node where this message should be processed. - * - * @return Client identifier. - */ - public UUID destinationId(); - - /** - * Sets identifier of the node where this message should be eventually delivered. - * - * @param id Client identifier. - */ - public void destinationId(UUID id); - - /** - * Gets client session token. - * - * @return Session token. - */ - public byte[] sessionToken(); - - /** - * Sets client session token. - * - * @param sesTok Session token. - */ - public void sessionToken(byte[] sesTok); -}
