http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridClockServer.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridClockServer.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridClockServer.java deleted file mode 100644 index 80b8adc..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridClockServer.java +++ /dev/null @@ -1,206 +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.clock; - -import org.apache.ignite.*; -import org.apache.ignite.internal.*; -import org.apache.ignite.thread.*; -import org.apache.ignite.internal.util.typedef.*; -import org.apache.ignite.internal.util.typedef.internal.*; -import org.apache.ignite.internal.util.worker.*; - -import java.io.*; -import java.net.*; - -/** - * Time server that enables time synchronization between nodes. - */ -public class GridClockServer { - /** Kernal context. */ - private GridKernalContext ctx; - - /** Datagram socket for message exchange. */ - private DatagramSocket sock; - - /** Logger. */ - private IgniteLogger log; - - /** Read worker. */ - private GridWorker readWorker; - - /** Instance of time processor. */ - private GridClockSyncProcessor clockSync; - - /** - * Starts server. - * - * @param ctx Kernal context. - * @throws IgniteCheckedException If server could not be started. - */ - public void start(GridKernalContext ctx) throws IgniteCheckedException { - this.ctx = ctx; - - clockSync = ctx.clockSync(); - log = ctx.log(GridClockServer.class); - - try { - int startPort = ctx.config().getTimeServerPortBase(); - int endPort = startPort + ctx.config().getTimeServerPortRange() - 1; - - InetAddress locHost = !F.isEmpty(ctx.config().getLocalHost()) ? - InetAddress.getByName(ctx.config().getLocalHost()) : - U.getLocalHost(); - - for (int p = startPort; p <= endPort; p++) { - try { - sock = new DatagramSocket(p, locHost); - - if (log.isDebugEnabled()) - log.debug("Successfully bound time server [host=" + locHost + ", port=" + p + ']'); - - break; - } - catch (SocketException e) { - if (log.isDebugEnabled()) - log.debug("Failed to bind time server socket [host=" + locHost + ", port=" + p + - ", err=" + e.getMessage() + ']'); - } - } - - if (sock == null) - throw new IgniteCheckedException("Failed to bind time server socket within specified port range [locHost=" + - locHost + ", startPort=" + startPort + ", endPort=" + endPort + ']'); - } - catch (IOException e) { - throw new IgniteCheckedException("Failed to start time server (failed to get local host address)", e); - } - } - - /** - * After start callback. - */ - public void afterStart() { - readWorker = new ReadWorker(); - - IgniteThread th = new IgniteThread(readWorker); - - th.setPriority(Thread.MAX_PRIORITY); - - th.start(); - } - - /** - * Stops server. - */ - public void stop() { - // No-op. - } - - /** - * Before stop callback. - */ - public void beforeStop() { - if (readWorker != null) - readWorker.cancel(); - - U.closeQuiet(sock); - - if (readWorker != null) - U.join(readWorker, log); - } - - /** - * Sends packet to remote node. - * - * @param msg Message to send. - * @param addr Address. - * @param port Port. - * @throws IgniteCheckedException If send failed. - */ - public void sendPacket(GridClockMessage msg, InetAddress addr, int port) throws IgniteCheckedException { - try { - DatagramPacket packet = new DatagramPacket(msg.toBytes(), GridClockMessage.PACKET_SIZE, addr, port); - - if (log.isDebugEnabled()) - log.debug("Sending time sync packet [msg=" + msg + ", addr=" + addr + ", port=" + port); - - sock.send(packet); - } - catch (IOException e) { - if (!sock.isClosed()) - throw new IgniteCheckedException("Failed to send datagram message to remote node [addr=" + addr + - ", port=" + port + ", msg=" + msg + ']', e); - } - } - - /** - * @return Address to which this server is bound. - */ - public InetAddress host() { - return sock.getLocalAddress(); - } - - /** - * @return Port to which this server is bound. - */ - public int port() { - return sock.getLocalPort(); - } - - /** - * Message read worker. - */ - private class ReadWorker extends GridWorker { - /** - * Creates read worker. - */ - protected ReadWorker() { - super(ctx.gridName(), "grid-time-server-reader", log); - } - - /** {@inheritDoc} */ - @Override protected void body() throws InterruptedException, IgniteInterruptedException { - DatagramPacket packet = new DatagramPacket(new byte[GridClockMessage.PACKET_SIZE], - GridClockMessage.PACKET_SIZE); - - while (!isCancelled()) { - try { - // Read packet from buffer. - sock.receive(packet); - - if (log.isDebugEnabled()) - log.debug("Received clock sync message from remote node [host=" + packet.getAddress() + - ", port=" + packet.getPort() + ']'); - - GridClockMessage msg = GridClockMessage.fromBytes(packet.getData(), packet.getOffset(), - packet.getLength()); - - clockSync.onMessageReceived(msg, packet.getAddress(), packet.getPort()); - } - catch (IgniteCheckedException e) { - U.warn(log, "Failed to assemble clock server message (will ignore the packet) [host=" + - packet.getAddress() + ", port=" + packet.getPort() + ", err=" + e.getMessage() + ']'); - } - catch (IOException e) { - if (!isCancelled()) - U.warn(log, "Failed to receive message on datagram socket: " + e); - } - } - } - } -}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridClockSource.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridClockSource.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridClockSource.java deleted file mode 100644 index 35a7aba..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridClockSource.java +++ /dev/null @@ -1,30 +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.clock; - -/** - * Interface representing time source for time processor. - */ -public interface GridClockSource { - /** - * Gets current time in milliseconds past since 1 January, 1970. - * - * @return Current time in milliseconds. - */ - public long currentTimeMillis(); -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridClockSyncProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridClockSyncProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridClockSyncProcessor.java deleted file mode 100644 index a9a1453..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridClockSyncProcessor.java +++ /dev/null @@ -1,458 +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.clock; - -import org.apache.ignite.*; -import org.apache.ignite.cluster.*; -import org.apache.ignite.events.*; -import org.apache.ignite.internal.*; -import org.apache.ignite.internal.processors.*; -import org.apache.ignite.internal.util.*; -import org.apache.ignite.thread.*; -import org.apache.ignite.internal.managers.communication.*; -import org.apache.ignite.internal.managers.discovery.*; -import org.apache.ignite.internal.managers.eventstorage.*; -import org.apache.ignite.internal.util.typedef.internal.*; -import org.apache.ignite.internal.util.worker.*; - -import java.net.*; -import java.util.*; - -import static org.apache.ignite.events.IgniteEventType.*; -import static org.apache.ignite.internal.GridNodeAttributes.*; -import static org.apache.ignite.internal.GridTopic.*; -import static org.apache.ignite.internal.managers.communication.GridIoPolicy.*; - -/** - * Time synchronization processor. - */ -public class GridClockSyncProcessor extends GridProcessorAdapter { - /** Maximum size for time sync history. */ - private static final int MAX_TIME_SYNC_HISTORY = 100; - - /** Time server instance. */ - private GridClockServer srv; - - /** Shutdown lock. */ - private GridSpinReadWriteLock rw = new GridSpinReadWriteLock(); - - /** Stopping flag. */ - private volatile boolean stopping; - - /** Time coordinator thread. */ - private volatile TimeCoordinator timeCoord; - - /** Time delta history. Constructed on coorinator. */ - private NavigableMap<GridClockDeltaVersion, GridClockDeltaSnapshot> timeSyncHist = - new GridBoundedConcurrentOrderedMap<>(MAX_TIME_SYNC_HISTORY); - - /** Time source. */ - private GridClockSource clockSrc; - - /** - * @param ctx Kernal context. - */ - public GridClockSyncProcessor(GridKernalContext ctx) { - super(ctx); - } - - /** {@inheritDoc} */ - @Override public void start() throws IgniteCheckedException { - super.start(); - - clockSrc = ctx.timeSource(); - - srv = new GridClockServer(); - - srv.start(ctx); - - ctx.io().addMessageListener(TOPIC_TIME_SYNC, new GridMessageListener() { - @Override public void onMessage(UUID nodeId, Object msg) { - assert msg instanceof GridClockDeltaSnapshotMessage; - - GridClockDeltaSnapshotMessage msg0 = (GridClockDeltaSnapshotMessage)msg; - - GridClockDeltaVersion ver = msg0.snapshotVersion(); - - timeSyncHist.put(ver, new GridClockDeltaSnapshot(ver, msg0.deltas())); - } - }); - - // We care only about node leave and fail events. - ctx.event().addLocalEventListener(new GridLocalEventListener() { - @Override public void onEvent(IgniteEvent evt) { - assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED || evt.type() == EVT_NODE_JOINED; - - IgniteDiscoveryEvent discoEvt = (IgniteDiscoveryEvent)evt; - - if (evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED) - checkLaunchCoordinator(discoEvt); - - TimeCoordinator timeCoord0 = timeCoord; - - if (timeCoord0 != null) - timeCoord0.onDiscoveryEvent(discoEvt); - } - }, EVT_NODE_LEFT, EVT_NODE_FAILED, EVT_NODE_JOINED); - } - - /** {@inheritDoc} */ - @Override public void addAttributes(Map<String, Object> attrs) throws IgniteCheckedException { - super.addAttributes(attrs); - - attrs.put(ATTR_TIME_SERVER_HOST, srv.host()); - attrs.put(ATTR_TIME_SERVER_PORT, srv.port()); - } - - /** {@inheritDoc} */ - @Override public void onKernalStart() throws IgniteCheckedException { - super.onKernalStart(); - - srv.afterStart(); - - // Check at startup if this node is a fragmentizer coordinator. - IgniteDiscoveryEvent locJoinEvt = ctx.discovery().localJoinEvent(); - - checkLaunchCoordinator(locJoinEvt); - } - - /** {@inheritDoc} */ - @Override public void onKernalStop(boolean cancel) { - super.onKernalStop(cancel); - - rw.writeLock(); - - try { - stopping = false; - - if (timeCoord != null) { - timeCoord.cancel(); - - U.join(timeCoord, log); - - timeCoord = null; - } - - if (srv != null) - srv.beforeStop(); - } - finally { - rw.writeUnlock(); - } - } - - /** {@inheritDoc} */ - @Override public void stop(boolean cancel) throws IgniteCheckedException { - super.stop(cancel); - - if (srv != null) - srv.stop(); - } - - /** - * Gets current time on local node. - * - * @return Current time in milliseconds. - */ - private long currentTime() { - return clockSrc.currentTimeMillis(); - } - - /** - * @return Time sync history. - */ - public NavigableMap<GridClockDeltaVersion, GridClockDeltaSnapshot> timeSyncHistory() { - return timeSyncHist; - } - - /** - * Callback from server for message receiving. - * - * @param msg Received message. - * @param addr Remote node address. - * @param port Remote node port. - */ - public void onMessageReceived(GridClockMessage msg, InetAddress addr, int port) { - long rcvTs = currentTime(); - - if (!msg.originatingNodeId().equals(ctx.localNodeId())) { - // We received time request from remote node, set current time and reply back. - msg.replyTimestamp(rcvTs); - - try { - srv.sendPacket(msg, addr, port); - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to send time server reply to remote node: " + msg, e); - } - } - else - timeCoord.onMessage(msg, rcvTs); - } - - /** - * Checks if local node is the oldest node in topology and starts time coordinator if so. - * - * @param discoEvt Discovery event. - */ - private void checkLaunchCoordinator(IgniteDiscoveryEvent discoEvt) { - rw.readLock(); - - try { - if (stopping) - return; - - if (timeCoord == null) { - long minNodeOrder = Long.MAX_VALUE; - - Collection<ClusterNode> nodes = discoEvt.topologyNodes(); - - for (ClusterNode node : nodes) { - if (node.order() < minNodeOrder) - minNodeOrder = node.order(); - } - - ClusterNode locNode = ctx.grid().localNode(); - - if (locNode.order() == minNodeOrder) { - if (log.isDebugEnabled()) - log.debug("Detected local node to be the eldest node in topology, starting time " + - "coordinator thread [discoEvt=" + discoEvt + ", locNode=" + locNode + ']'); - - synchronized (this) { - if (timeCoord == null && !stopping) { - timeCoord = new TimeCoordinator(discoEvt); - - IgniteThread th = new IgniteThread(timeCoord); - - th.setPriority(Thread.MAX_PRIORITY); - - th.start(); - } - } - } - } - } - finally { - rw.readUnlock(); - } - } - - /** - * Gets time adjusted with time coordinator on given topology version. - * - * @param topVer Topology version. - * @return Adjusted time. - */ - public long adjustedTime(long topVer) { - // Get last synchronized time on given topology version. - Map.Entry<GridClockDeltaVersion, GridClockDeltaSnapshot> entry = timeSyncHistory().lowerEntry( - new GridClockDeltaVersion(0, topVer + 1)); - - GridClockDeltaSnapshot snap = entry == null ? null : entry.getValue(); - - long now = clockSrc.currentTimeMillis(); - - if (snap == null) - return System.currentTimeMillis(); - - Long delta = snap.deltas().get(ctx.localNodeId()); - - if (delta == null) - delta = 0L; - - return now + delta; - } - - /** - * Publishes snapshot to topology. - * - * @param snapshot Snapshot to publish. - * @param top Topology to send given snapshot to. - */ - private void publish(GridClockDeltaSnapshot snapshot, GridDiscoveryTopologySnapshot top) { - if (!rw.tryReadLock()) - return; - - try { - timeSyncHist.put(snapshot.version(), snapshot); - - for (ClusterNode n : top.topologyNodes()) { - GridClockDeltaSnapshotMessage msg = new GridClockDeltaSnapshotMessage( - snapshot.version(), snapshot.deltas()); - - try { - ctx.io().send(n, TOPIC_TIME_SYNC, msg, SYSTEM_POOL); - } - catch (IgniteCheckedException e) { - if (ctx.discovery().pingNode(n.id())) - U.error(log, "Failed to send time sync snapshot to remote node (did not leave grid?) " + - "[nodeId=" + n.id() + ", msg=" + msg + ", err=" + e.getMessage() + ']'); - else if (log.isDebugEnabled()) - log.debug("Failed to send time sync snapshot to remote node (did not leave grid?) " + - "[nodeId=" + n.id() + ", msg=" + msg + ", err=" + e.getMessage() + ']'); - } - } - } - finally { - rw.readUnlock(); - } - } - - /** - * Time coordinator thread. - */ - private class TimeCoordinator extends GridWorker { - /** Last discovery topology snapshot. */ - private volatile GridDiscoveryTopologySnapshot lastSnapshot; - - /** Snapshot being constructed. May be not null only on coordinator node. */ - private volatile GridClockDeltaSnapshot pendingSnapshot; - - /** Version counter. */ - private long verCnt = 1; - - /** - * Time coordinator thread constructor. - * - * @param evt Discovery event on which this node became a coordinator. - */ - protected TimeCoordinator(IgniteDiscoveryEvent evt) { - super(ctx.gridName(), "grid-time-coordinator", log); - - lastSnapshot = new GridDiscoveryTopologySnapshot(evt.topologyVersion(), evt.topologyNodes()); - } - - /** {@inheritDoc} */ - @Override protected void body() throws InterruptedException, IgniteInterruptedException { - while (!isCancelled()) { - GridDiscoveryTopologySnapshot top = lastSnapshot; - - if (log.isDebugEnabled()) - log.debug("Creating time sync snapshot for topology: " + top); - - GridClockDeltaSnapshot snapshot = new GridClockDeltaSnapshot( - new GridClockDeltaVersion(verCnt++, top.topologyVersion()), - ctx.localNodeId(), - top, - ctx.config().getClockSyncSamples()); - - pendingSnapshot = snapshot; - - while (!snapshot.ready()) { - if (log.isDebugEnabled()) - log.debug("Requesting time from remote nodes: " + snapshot.pendingNodeIds()); - - for (UUID nodeId : snapshot.pendingNodeIds()) - requestTime(nodeId); - - if (log.isDebugEnabled()) - log.debug("Waiting for snapshot to be ready: " + snapshot); - - // Wait for all replies - snapshot.awaitReady(1000); - } - - // No more messages should be processed. - pendingSnapshot = null; - - if (log.isDebugEnabled()) - log.debug("Collected time sync results: " + snapshot.deltas()); - - publish(snapshot, top); - - synchronized (this) { - if (top.topologyVersion() == lastSnapshot.topologyVersion()) - wait(ctx.config().getClockSyncFrequency()); - } - } - } - - /** - * @param evt Discovery event. - */ - public void onDiscoveryEvent(IgniteDiscoveryEvent evt) { - if (log.isDebugEnabled()) - log.debug("Processing discovery event: " + evt); - - if (evt.type() == IgniteEventType.EVT_NODE_FAILED || evt.type() == EVT_NODE_LEFT) - onNodeLeft(evt.eventNode().id()); - - synchronized (this) { - lastSnapshot = new GridDiscoveryTopologySnapshot(evt.topologyVersion(), evt.topologyNodes()); - - notifyAll(); - } - } - - /** - * @param msg Message received from remote node. - * @param rcvTs Receive timestamp. - */ - private void onMessage(GridClockMessage msg, long rcvTs) { - GridClockDeltaSnapshot curr = pendingSnapshot; - - if (curr != null) { - long delta = (msg.originatingTimestamp() + rcvTs) / 2 - msg.replyTimestamp(); - - boolean needMore = curr.onDeltaReceived(msg.targetNodeId(), delta); - - if (needMore) - requestTime(msg.targetNodeId()); - } - } - - /** - * Requests time from remote node. - * - * @param rmtNodeId Remote node ID. - */ - private void requestTime(UUID rmtNodeId) { - ClusterNode node = ctx.discovery().node(rmtNodeId); - - if (node != null) { - InetAddress addr = node.attribute(ATTR_TIME_SERVER_HOST); - int port = node.attribute(ATTR_TIME_SERVER_PORT); - - try { - GridClockMessage req = new GridClockMessage(ctx.localNodeId(), rmtNodeId, currentTime(), 0); - - srv.sendPacket(req, addr, port); - } - catch (IgniteCheckedException e) { - LT.warn(log, e, "Failed to send time request to remote node [rmtNodeId=" + rmtNodeId + - ", addr=" + addr + ", port=" + port + ']'); - } - } - else - onNodeLeft(rmtNodeId); - } - - /** - * Node left callback. - * - * @param nodeId Left node ID. - */ - private void onNodeLeft(UUID nodeId) { - GridClockDeltaSnapshot curr = pendingSnapshot; - - if (curr != null) - curr.onNodeLeft(nodeId); - } - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridJvmClockSource.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridJvmClockSource.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridJvmClockSource.java deleted file mode 100644 index 387cbc1..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/clock/GridJvmClockSource.java +++ /dev/null @@ -1,28 +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.clock; - -/** - * JVM time source. - */ -public class GridJvmClockSource implements GridClockSource { - /** {@inheritDoc} */ - @Override public long currentTimeMillis() { - return System.currentTimeMillis(); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java index b49b627..9f849db 100644 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java +++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java @@ -30,7 +30,6 @@ import org.apache.ignite.internal.managers.communication.*; import org.apache.ignite.internal.managers.deployment.*; import org.apache.ignite.internal.managers.eventstorage.*; import org.gridgain.grid.kernal.processors.cache.*; -import org.gridgain.grid.kernal.processors.portable.*; import org.apache.ignite.internal.util.future.*; import org.apache.ignite.internal.util.lang.*; import org.apache.ignite.internal.util.tostring.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridDrDataLoadCacheUpdater.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridDrDataLoadCacheUpdater.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridDrDataLoadCacheUpdater.java deleted file mode 100644 index 1ea269e..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridDrDataLoadCacheUpdater.java +++ /dev/null @@ -1,77 +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.dr; - -import org.apache.ignite.*; -import org.apache.ignite.cache.*; -import org.apache.ignite.dataload.*; -import org.apache.ignite.internal.*; -import org.apache.ignite.lang.*; -import org.gridgain.grid.kernal.processors.cache.*; -import org.gridgain.grid.kernal.processors.cache.dr.*; -import org.apache.ignite.internal.util.typedef.*; - -import java.util.*; - -/** - * Data center replication cache updater for data loader. - */ -public class GridDrDataLoadCacheUpdater<K, V> implements IgniteDataLoadCacheUpdater<K, V> { - /** */ - private static final long serialVersionUID = 0L; - - /** {@inheritDoc} */ - @Override public void update(IgniteCache<K, V> cache0, Collection<Map.Entry<K, V>> col) - throws IgniteCheckedException { - String cacheName = cache0.getConfiguration(CacheConfiguration.class).getName(); - - GridKernalContext ctx = ((GridKernal)cache0.unwrap(Ignite.class)).context(); - IgniteLogger log = ctx.log(GridDrDataLoadCacheUpdater.class); - GridCacheAdapter<K, V> cache = ctx.cache().internalCache(cacheName); - - assert !F.isEmpty(col); - - if (log.isDebugEnabled()) - log.debug("Running DR put job [nodeId=" + ctx.localNodeId() + ", cacheName=" + cacheName + ']'); - - IgniteFuture<?> f = cache.context().preloader().startFuture(); - - if (!f.isDone()) - f.get(); - - for (Map.Entry<K, V> entry0 : col) { - GridVersionedEntry<K, V> entry = (GridVersionedEntry<K, V>)entry0; - - entry.unmarshal(ctx.config().getMarshaller()); - - K key = entry.key(); - - GridCacheDrInfo<V> val = entry.value() != null ? entry.expireTime() != 0 ? - new GridCacheDrExpirationInfo<>(entry.value(), entry.version(), entry.ttl(), entry.expireTime()) : - new GridCacheDrInfo<>(entry.value(), entry.version()) : null; - - if (val == null) - cache.removeAllDr(Collections.singletonMap(key, entry.version())); - else - cache.putAllDr(Collections.singletonMap(key, val)); - } - - if (log.isDebugEnabled()) - log.debug("DR put job finished [nodeId=" + ctx.localNodeId() + ", cacheName=" + cacheName + ']'); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridDrType.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridDrType.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridDrType.java deleted file mode 100644 index 44f3f95..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridDrType.java +++ /dev/null @@ -1,38 +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.dr; - -/** - * Data center replication type. - */ -public enum GridDrType { - /** Do not replicate that entry. */ - DR_NONE, - - /** Regular replication on primary node. */ - DR_PRIMARY, - - /** Regular replication on backup node. */ - DR_BACKUP, - - /** Replication during load. */ - DR_LOAD, - - /** Replication during preload. */ - DR_PRELOAD -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridRawVersionedEntry.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridRawVersionedEntry.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridRawVersionedEntry.java deleted file mode 100644 index aa85685..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridRawVersionedEntry.java +++ /dev/null @@ -1,210 +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.dr; - -import org.apache.ignite.*; -import org.apache.ignite.marshaller.*; -import org.gridgain.grid.kernal.processors.cache.*; -import org.apache.ignite.internal.util.typedef.internal.*; -import org.jetbrains.annotations.*; - -import java.io.*; - -/** - * - */ -public class GridRawVersionedEntry<K, V> implements GridVersionedEntry<K, V>, Externalizable { - /** */ - private static final long serialVersionUID = 0L; - - /** Key. */ - private K key; - - /** Key bytes. */ - private byte[] keyBytes; - - /** Value. */ - private V val; - - /** Value bytes. */ - private byte[] valBytes; - - /** TTL. */ - private long ttl; - - /** Expire time. */ - private long expireTime; - - /** Version. */ - private GridCacheVersion ver; - - /** - * {@code Externalizable) support. - */ - public GridRawVersionedEntry() { - // No-op. - } - - /** - * Constructor. - * - * @param key Key. - * @param keyBytes Key bytes. - * @param val Value. - * @param valBytes Value bytes. - * @param expireTime Expire time. - * @param ttl TTL. - * @param ver Version. - */ - public GridRawVersionedEntry(K key, - @Nullable byte[] keyBytes, - @Nullable V val, - @Nullable byte[] valBytes, - long ttl, - long expireTime, - GridCacheVersion ver) { - this.key = key; - this.keyBytes = keyBytes; - this.val = val; - this.valBytes = valBytes; - this.ttl = ttl; - this.expireTime = expireTime; - this.ver = ver; - } - - /** {@inheritDoc} */ - @Override public K key() { - assert key != null : "Entry is being improperly processed."; - - return key; - } - - /** - * @return Key bytes. - */ - public byte[] keyBytes() { - return keyBytes; - } - - /** {@inheritDoc} */ - @Override public V value() { - return val; - } - - /** - * @return Value bytes. - */ - public byte[] valueBytes() { - return valBytes; - } - - /** {@inheritDoc} */ - @Override public long ttl() { - return ttl; - } - - /** {@inheritDoc} */ - @Override public long expireTime() { - return expireTime; - } - - /** {@inheritDoc} */ - @Override public GridCacheVersion version() { - return ver; - } - - /** {@inheritDoc} */ - @Override public void unmarshal(IgniteMarshaller marsh) throws IgniteCheckedException { - unmarshalKey(marsh); - - if (valBytes != null && val == null) - val = marsh.unmarshal(valBytes, null); - } - - /** - * Perform internal key unmarshal of this entry. It must be performed after entry is deserialized and before - * its restored key/value are needed. - * - * @param marsh Marshaller. - * @throws IgniteCheckedException If failed. - */ - private void unmarshalKey(IgniteMarshaller marsh) throws IgniteCheckedException { - if (key == null) - key = marsh.unmarshal(keyBytes, null); - } - - /** {@inheritDoc} */ - @Override public void marshal(IgniteMarshaller marsh) throws IgniteCheckedException { - if (keyBytes == null) - keyBytes = marsh.marshal(key); - - if (valBytes == null && val != null) - valBytes = marsh.marshal(val); - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - assert keyBytes != null; - - U.writeByteArray(out, keyBytes); - U.writeByteArray(out, valBytes); - - out.writeLong(ttl); - - if (ttl != 0) - out.writeLong(expireTime); - - out.writeObject(ver); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - keyBytes = U.readByteArray(in); - valBytes = U.readByteArray(in); - - ttl = in.readLong(); - - if (ttl != 0) - expireTime = in.readLong(); - - ver = (GridCacheVersion)in.readObject(); - - assert keyBytes != null; - } - - /** {@inheritDoc} */ - @Override public K getKey() { - return key(); - } - - /** {@inheritDoc} */ - @Override public V getValue() { - return value(); - } - - /** {@inheritDoc} */ - @Override public V setValue(V val) { - throw new UnsupportedOperationException(); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(GridRawVersionedEntry.class, this, "keyBytesLen", keyBytes != null ? keyBytes.length : "n/a", - "valBytesLen", valBytes != null ? valBytes.length : "n/a"); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridVersionedEntry.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridVersionedEntry.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridVersionedEntry.java deleted file mode 100644 index 221064d..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridVersionedEntry.java +++ /dev/null @@ -1,80 +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.dr; - -import org.apache.ignite.*; -import org.apache.ignite.marshaller.*; -import org.gridgain.grid.kernal.processors.cache.*; -import org.jetbrains.annotations.*; - -import java.util.*; - -/** - * - */ -public interface GridVersionedEntry<K, V> extends Map.Entry<K, V> { - /** - * Gets entry's key. - * - * @return Entry's key. - */ - public K key(); - - /** - * Gets entry's value. - * - * @return Entry's value. - */ - @Nullable public V value(); - - /** - * Gets entry's TTL. - * - * @return Entry's TTL. - */ - public long ttl(); - - /** - * Gets entry's expire time. - * - * @return Entry's expire time. - */ - public long expireTime(); - - /** - * @return Version. - */ - public GridCacheVersion version(); - - /** - * Perform internal marshal of this entry before it will be serialized. - * - * @param marsh Marshaller. - * @throws IgniteCheckedException If failed. - */ - public void marshal(IgniteMarshaller marsh) throws IgniteCheckedException; - - /** - * Perform internal unmarshal of this entry. It must be performed after entry is deserialized and before - * its restored key/value are needed. - * - * @param marsh Marshaller. - * @throws IgniteCheckedException If failed. - */ - public void unmarshal(IgniteMarshaller marsh) throws IgniteCheckedException; -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/package.html ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/package.html b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/package.html deleted file mode 100644 index 55ce324..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/package.html +++ /dev/null @@ -1,25 +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. - --> -<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd"> -<html> -<body> - <img alt="icon" class="javadocimg" src="{@docRoot}/img/cube.gif"/> - Data center replication processor. - <p> - @html.java.package -</body> -</html> http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropAware.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropAware.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropAware.java deleted file mode 100644 index 8274f46..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropAware.java +++ /dev/null @@ -1,49 +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.interop; - -import org.apache.ignite.*; -import org.apache.ignite.internal.*; - -/** - * Interface for interop-aware components. - */ -public interface GridInteropAware { - /** - * Sets configuration parameters. - * - * @param params Configuration parameters. - */ - public void configure(Object... params); - - /** - * Initializes interop-aware component. - * - * @param ctx Context. - * @throws IgniteCheckedException In case of error. - */ - public void initialize(GridKernalContext ctx) throws IgniteCheckedException; - - /** - * Destroys interop-aware component. - * - * @param ctx Context. - * @throws IgniteCheckedException In case of error. - */ - public void destroy(GridKernalContext ctx) throws IgniteCheckedException; -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropProcessor.java deleted file mode 100644 index cdcf86b..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropProcessor.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.interop; - -import org.apache.ignite.*; -import org.apache.ignite.internal.processors.*; -import org.jetbrains.annotations.*; - -/** - * Interop processor. - */ -public interface GridInteropProcessor extends GridProcessor { - /** - * Release start latch. - */ - public void releaseStart(); - - /** - * Await start on native side. - * - * @throws IgniteCheckedException If failed. - */ - public void awaitStart() throws IgniteCheckedException; - - /** - * @return Environment pointer. - */ - public long environmentPointer() throws IgniteCheckedException; - - /** - * @return Grid name. - */ - public String gridName(); - - /** - * Gets native wrapper for default Grid projection. - * - * @return Native compute wrapper. - * @throws IgniteCheckedException If failed. - */ - public GridInteropTarget projection() throws IgniteCheckedException; - - /** - * Gets native wrapper for cache with the given name. - * - * @param name Cache name ({@code null} for default cache). - * @return Native cache wrapper. - * @throws IgniteCheckedException If failed. - */ - public GridInteropTarget cache(@Nullable String name) throws IgniteCheckedException; - - /** - * Gets native wrapper for data loader for cache with the given name. - * - * @param cacheName Cache name ({@code null} for default cache). - * @return Native data loader wrapper. - * @throws IgniteCheckedException If failed. - */ - public GridInteropTarget dataLoader(@Nullable String cacheName) throws IgniteCheckedException; - - /** - * Stops grid. - * - * @param cancel Cancel flag. - */ - public void close(boolean cancel); -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropProcessorAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropProcessorAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropProcessorAdapter.java deleted file mode 100644 index 8c1742a..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropProcessorAdapter.java +++ /dev/null @@ -1,31 +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.interop; - -import org.apache.ignite.internal.*; -import org.apache.ignite.internal.processors.*; - -/** - * Interop processor adapter. - */ -public abstract class GridInteropProcessorAdapter extends GridProcessorAdapter implements GridInteropProcessor { - /** {@inheritDoc} */ - protected GridInteropProcessorAdapter(GridKernalContext ctx) { - super(ctx); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropTarget.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropTarget.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropTarget.java deleted file mode 100644 index 95cac5e..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/GridInteropTarget.java +++ /dev/null @@ -1,109 +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.interop; - -import org.apache.ignite.*; -import org.gridgain.grid.kernal.processors.portable.*; -import org.jetbrains.annotations.*; - -/** - * Interop target abstraction. - */ -public interface GridInteropTarget { - /** - * Synchronous IN operation. - * - * @param type Operation type. - * @param stream Input stream. - * @return Value specific for the given operation otherwise. - * @throws IgniteCheckedException In case of failure. - */ - public int inOp(int type, GridPortableInputStream stream) throws IgniteCheckedException; - - /** - * Synchronous IN operation which returns managed object as result. - * - * @param type Operation type. - * @param stream Input stream. - * @return Managed result. - * @throws IgniteCheckedException If case of failure. - */ - public Object inOpObject(int type, GridPortableInputStream stream) throws IgniteCheckedException; - - /** - * Synchronous OUT operation. - * - * @param type Operation type. - * @param stream Native stream address. - * @param arr Native array address. - * @param cap Capacity. - * @throws IgniteCheckedException In case of failure. - */ - public void outOp(int type, long stream, long arr, int cap) throws IgniteCheckedException; - - /** - * Synchronous IN-OUT operation. - * - * @param type Operation type. - * @param inStream Input stream. - * @param outStream Native stream address. - * @param outArr Native array address. - * @param outCap Capacity. - * @throws IgniteCheckedException In case of failure. - */ - public void inOutOp(int type, GridPortableInputStream inStream, long outStream, long outArr, int outCap) - throws IgniteCheckedException; - - /** - * Synchronous IN-OUT operation with optional argument. - * - * @param type Operation type. - * @param inStream Input stream. - * @param outStream Native stream address. - * @param outArr Native array address. - * @param outCap Capacity. - * @param arg Argument (optional). - * @throws IgniteCheckedException In case of failure. - */ - public void inOutOp(int type, GridPortableInputStream inStream, long outStream, long outArr, int outCap, - @Nullable Object arg) throws IgniteCheckedException; - - /** - * Asynchronous IN operation. - * - * @param type Operation type. - * @param futId Future ID. - * @param in Input stream. - * @throws IgniteCheckedException In case of failure. - */ - public void inOpAsync(int type, long futId, GridPortableInputStream in) throws IgniteCheckedException; - - /** - * Asynchronous IN-OUT operation. - * - * @param type Operation type. - * @param futId Future ID. - * @param in Input stream. - * @param outStream Native stream address. - * @param outArr Native array address. - * @param outCap Capacity. - * @throws IgniteCheckedException In case of failure. - */ - public void inOutOpAsync(int type, long futId, GridPortableInputStream in, long outStream, long outArr, int outCap) - throws IgniteCheckedException; -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/GridOsInteropProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/GridOsInteropProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/GridOsInteropProcessor.java deleted file mode 100644 index 6610a44..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/GridOsInteropProcessor.java +++ /dev/null @@ -1,80 +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.interop.os; - -import org.apache.ignite.*; -import org.apache.ignite.internal.*; -import org.gridgain.grid.kernal.processors.interop.*; -import org.jetbrains.annotations.*; - -/** - * OS interop processor. - */ -public class GridOsInteropProcessor extends GridInteropProcessorAdapter { - /** Common error message. */ - private static final String ERR_MSG = "Interop feature is not supported in OS edition."; - - /** - * Constructor. - * - * @param ctx Context. - */ - public GridOsInteropProcessor(GridKernalContext ctx) { - super(ctx); - } - - /** {@inheritDoc} */ - @Override public void releaseStart() { - throw new UnsupportedOperationException(ERR_MSG); - } - - /** {@inheritDoc} */ - @Override public void awaitStart() throws IgniteCheckedException { - throw new UnsupportedOperationException(ERR_MSG); - } - - /** {@inheritDoc} */ - @Override public long environmentPointer() throws IgniteCheckedException { - throw new UnsupportedOperationException(ERR_MSG); - } - - /** {@inheritDoc} */ - @Override public String gridName() { - throw new UnsupportedOperationException(ERR_MSG); - } - - /** {@inheritDoc} */ - @Override public void close(boolean cancel) { - throw new UnsupportedOperationException(ERR_MSG); - } - - /** {@inheritDoc} */ - @Override public GridInteropTarget projection() throws IgniteCheckedException { - throw new UnsupportedOperationException(ERR_MSG); - } - - /** {@inheritDoc} */ - @Override public GridInteropTarget cache(@Nullable String name) throws IgniteCheckedException { - throw new UnsupportedOperationException(ERR_MSG); - } - - /** {@inheritDoc} */ - @Override public GridInteropTarget dataLoader(@Nullable String cacheName) throws IgniteCheckedException { - throw new UnsupportedOperationException(ERR_MSG); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/package.html ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/package.html b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/package.html deleted file mode 100644 index 20815a0..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/package.html +++ /dev/null @@ -1,23 +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. - --> -<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd"> -<html> -<body> - <!-- Package description. --> - No-op implementation of interop processor. -</body> -</html> http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/package.html ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/package.html b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/package.html deleted file mode 100644 index 57a4e47..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/package.html +++ /dev/null @@ -1,23 +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. - --> -<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd"> -<html> -<body> - <!-- Package description. --> - Interop processor. -</body> -</html> http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableInputStream.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableInputStream.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableInputStream.java deleted file mode 100644 index 821739b..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableInputStream.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.portable; - -/** - * Portable input stream. - */ -public interface GridPortableInputStream extends GridPortableStream { - /** - * Read byte value. - * - * @return Byte value. - */ - public byte readByte(); - - /** - * Read byte array. - * - * @param cnt Expected item count. - * @return Byte array. - */ - public byte[] readByteArray(int cnt); - - /** - * Read boolean value. - * - * @return Boolean value. - */ - public boolean readBoolean(); - - /** - * Read boolean array. - * - * @param cnt Expected item count. - * @return Boolean array. - */ - public boolean[] readBooleanArray(int cnt); - - /** - * Read short value. - * - * @return Short value. - */ - public short readShort(); - - /** - * Read short array. - * - * @param cnt Expected item count. - * @return Short array. - */ - public short[] readShortArray(int cnt); - - /** - * Read char value. - * - * @return Char value. - */ - public char readChar(); - - /** - * Read char array. - * - * @param cnt Expected item count. - * @return Char array. - */ - public char[] readCharArray(int cnt); - - /** - * Read int value. - * - * @return Int value. - */ - public int readInt(); - - /** - * Read int value at the given position. - * - * @param pos Position. - * @return Value. - */ - public int readInt(int pos); - - /** - * Read int array. - * - * @param cnt Expected item count. - * @return Int array. - */ - public int[] readIntArray(int cnt); - - /** - * Read float value. - * - * @return Float value. - */ - public float readFloat(); - - /** - * Read float array. - * - * @param cnt Expected item count. - * @return Float array. - */ - public float[] readFloatArray(int cnt); - - /** - * Read long value. - * - * @return Long value. - */ - public long readLong(); - - /** - * Read long array. - * - * @param cnt Expected item count. - * @return Long array. - */ - public long[] readLongArray(int cnt); - - /** - * Read double value. - * - * @return Double value. - */ - public double readDouble(); - - /** - * Read double array. - * - * @param cnt Expected item count. - * @return Double array. - */ - public double[] readDoubleArray(int cnt); - - /** - * Read data to byte array. - * - * @param arr Array. - * @param off Offset. - * @param len Length. - * @return Amount of actual bytes read. - */ - public int read(byte[] arr, int off, int len); - - /** - * Read data to the given address. - * - * @param addr Address. - * @param len Length. - * @return Amount of actual bytes read. - */ - public int read(long addr, int len); - - /** - * Gets amount of remaining data in bytes. - * - * @return Remaining data. - */ - public int remaining(); -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableOutputStream.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableOutputStream.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableOutputStream.java deleted file mode 100644 index 11569cb..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableOutputStream.java +++ /dev/null @@ -1,172 +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.portable; - -/** - * Portable output stream. - */ -public interface GridPortableOutputStream extends GridPortableStream, AutoCloseable { - /** - * Write byte value. - * - * @param val Byte value. - */ - public void writeByte(byte val); - - /** - * Write byte array. - * - * @param val Byte array. - */ - public void writeByteArray(byte[] val); - - /** - * Write boolean value. - * - * @param val Boolean value. - */ - public void writeBoolean(boolean val); - - /** - * Write boolean array. - * - * @param val Boolean array. - */ - public void writeBooleanArray(boolean[] val); - - /** - * Write short value. - * - * @param val Short value. - */ - public void writeShort(short val); - - /** - * Write short array. - * - * @param val Short array. - */ - public void writeShortArray(short[] val); - - /** - * Write char value. - * - * @param val Char value. - */ - public void writeChar(char val); - - /** - * Write char array. - * - * @param val Char array. - */ - public void writeCharArray(char[] val); - - /** - * Write int value. - * - * @param val Int value. - */ - public void writeInt(int val); - - /** - * Write int value to the given position. - * - * @param pos Position. - * @param val Value. - */ - public void writeInt(int pos, int val); - - /** - * Write int array. - * - * @param val Int array. - */ - public void writeIntArray(int[] val); - - /** - * Write float value. - * - * @param val Float value. - */ - public void writeFloat(float val); - - /** - * Write float array. - * - * @param val Float array. - */ - public void writeFloatArray(float[] val); - - /** - * Write long value. - * - * @param val Long value. - */ - public void writeLong(long val); - - /** - * Write long array. - * - * @param val Long array. - */ - public void writeLongArray(long[] val); - - /** - * Write double value. - * - * @param val Double value. - */ - public void writeDouble(double val); - - /** - * Write double array. - * - * @param val Double array. - */ - public void writeDoubleArray(double[] val); - - /** - * Write byte array. - * - * @param arr Array. - * @param off Offset. - * @param len Length. - */ - public void write(byte[] arr, int off, int len); - - /** - * Write data from unmanaged memory. - * - * @param addr Address. - * @param cnt Count. - */ - public void write(long addr, int cnt); - - /** - * Ensure capacity. - * - * @param cnt Required byte count. - */ - public void ensureCapacity(int cnt); - - /** - * Close the stream releasing resources. - */ - @Override public void close(); -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableProcessor.java deleted file mode 100644 index e990b98..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableProcessor.java +++ /dev/null @@ -1,150 +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.portable; - -import org.apache.ignite.internal.processors.*; -import org.apache.ignite.portables.*; -import org.apache.ignite.client.marshaller.*; -import org.jetbrains.annotations.*; - -import java.nio.*; -import java.util.*; - -/** - * Portable processor. - */ -public interface GridPortableProcessor extends GridProcessor { - /** - * @param typeName Type name. - * @return Type ID. - */ - public int typeId(String typeName); - - /** - * @param obj Object to marshal. - * @param trim If {@code true} trims result byte buffer. - * @return Object bytes. - * @throws org.apache.ignite.portables.PortableException In case of error. - */ - public ByteBuffer marshal(@Nullable Object obj, boolean trim) throws PortableException; - - /** - * @param arr Byte array. - * @param off Offset. - * @return Unmarshalled object. - * @throws org.apache.ignite.portables.PortableException In case of error. - */ - public Object unmarshal(byte[] arr, int off) throws PortableException; - - /** - * @param ptr Offheap pointer. - * @param forceHeap If {@code true} creates heap-based object. - * @return Unmarshalled object. - * @throws org.apache.ignite.portables.PortableException In case of error. - */ - public Object unmarshal(long ptr, boolean forceHeap) throws PortableException; - - /** - * Converts temporary offheap object to heap-based. - * - * @param obj Object. - * @return Heap-based object. - * @throws org.apache.ignite.portables.PortableException In case of error. - */ - @Nullable public Object unwrapTemporary(@Nullable Object obj) throws PortableException; - - /** - * @param obj Object to marshal. - * @return Portable object. - * @throws org.apache.ignite.portables.PortableException In case of error. - */ - public Object marshalToPortable(@Nullable Object obj) throws PortableException; - - /** - * @param obj Object (portable or not). - * @return Detached portable object or original object. - */ - public Object detachPortable(@Nullable Object obj); - - /** - * @return Portable marshaller for client connectivity or {@code null} if it's not - * supported (in case of OS edition). - */ - @Nullable public GridClientMarshaller portableMarshaller(); - - /** - * @param marsh Client marshaller. - * @return Whether marshaller is portable. - */ - public boolean isPortable(GridClientMarshaller marsh); - - /** - * @return Builder. - */ - public PortableBuilder builder(int typeId); - - /** - * @return Builder. - */ - public PortableBuilder builder(String clsName); - - /** - * Creates builder initialized by existing portable object. - * - * @param portableObj Portable object to edit. - * @return Portable builder. - */ - public PortableBuilder builder(PortableObject portableObj); - - /** - * @param typeId Type ID. - * @param newMeta New meta data. - * @throws org.apache.ignite.portables.PortableException In case of error. - */ - public void addMeta(int typeId, final PortableMetadata newMeta) throws PortableException; - - /** - * @param typeId Type ID. - * @param typeName Type name. - * @param affKeyFieldName Affinity key field name. - * @param fieldTypeIds Fields map. - * @throws org.apache.ignite.portables.PortableException In case of error. - */ - public void updateMetaData(int typeId, String typeName, @Nullable String affKeyFieldName, - Map<String, Integer> fieldTypeIds) throws PortableException; - - /** - * @param typeId Type ID. - * @return Meta data. - * @throws org.apache.ignite.portables.PortableException In case of error. - */ - @Nullable public PortableMetadata metadata(int typeId) throws PortableException; - - /** - * @param typeIds Type ID. - * @return Meta data. - * @throws org.apache.ignite.portables.PortableException In case of error. - */ - public Map<Integer, PortableMetadata> metadata(Collection<Integer> typeIds) throws PortableException; - - /** - * @return Metadata for all types. - * @throws org.apache.ignite.portables.PortableException In case of error. - */ - public Collection<PortableMetadata> metadata() throws PortableException; -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableStream.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableStream.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableStream.java deleted file mode 100644 index 8a0a7ba..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/GridPortableStream.java +++ /dev/null @@ -1,53 +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.portable; - -/** - * Portable stream. - */ -public interface GridPortableStream { - /** - * @return Position. - */ - public int position(); - - /** - * @param pos Position. - */ - public void position(int pos); - - /** - * @return Underlying array. - */ - public byte[] array(); - - /** - * @return Copy of data in the stream. - */ - public byte[] arrayCopy(); - - /** - * @return Offheap pointer if stream is offheap based, otherwise {@code 0}. - */ - public long offheapPointer(); - - /** - * @return {@code True} is stream is array based. - */ - public boolean hasArray(); -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/os/GridOsPortableProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/os/GridOsPortableProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/os/GridOsPortableProcessor.java deleted file mode 100644 index c17c84d..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/os/GridOsPortableProcessor.java +++ /dev/null @@ -1,126 +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.portable.os; - -import org.apache.ignite.internal.*; -import org.apache.ignite.internal.processors.*; -import org.apache.ignite.portables.*; -import org.apache.ignite.client.marshaller.*; -import org.gridgain.grid.kernal.processors.portable.*; -import org.jetbrains.annotations.*; - -import java.nio.*; -import java.util.*; - -/** - * No-op implementation of {@link GridPortableProcessor}. - */ -public class GridOsPortableProcessor extends GridProcessorAdapter implements GridPortableProcessor { - /** - * @param ctx Kernal context. - */ - public GridOsPortableProcessor(GridKernalContext ctx) { - super(ctx); - } - - /** {@inheritDoc} */ - @Override public int typeId(String typeName) { - return 0; - } - - /** {@inheritDoc} */ - @Override public ByteBuffer marshal(@Nullable Object obj, boolean trim) throws PortableException { - return null; - } - - /** {@inheritDoc} */ - @Nullable @Override public Object unmarshal(byte[] arr, int off) throws PortableException { - return null; - } - - /** {@inheritDoc} */ - @Override public Object unmarshal(long ptr, boolean forceHeap) throws PortableException { - return null; - } - - /** {@inheritDoc} */ - @Override public Object unwrapTemporary(Object obj) throws PortableException { - return null; - } - - /** {@inheritDoc} */ - @Nullable @Override public Object marshalToPortable(@Nullable Object obj) throws PortableException { - return obj; - } - - /** {@inheritDoc} */ - @Override public Object detachPortable(@Nullable Object obj) { - return obj; - } - - /** {@inheritDoc} */ - @Nullable @Override public GridClientMarshaller portableMarshaller() { - return null; - } - - /** {@inheritDoc} */ - @Override public boolean isPortable(GridClientMarshaller marsh) { - return false; - } - - /** {@inheritDoc} */ - @Override public PortableBuilder builder(int typeId) { - return null; - } - - /** {@inheritDoc} */ - @Override public PortableBuilder builder(String clsName) { - return null; - } - - /** {@inheritDoc} */ - @Override public PortableBuilder builder(PortableObject portableObj) { - return null; - } - - /** {@inheritDoc} */ - @Override public void addMeta(int typeId, PortableMetadata newMeta) throws PortableException { - // No-op. - } - - /** {@inheritDoc} */ - @Override public void updateMetaData(int typeId, String typeName, String affKeyFieldName, - Map<String, Integer> fieldTypeIds) throws PortableException { - // No-op. - } - - /** {@inheritDoc} */ - @Nullable @Override public PortableMetadata metadata(int typeId) { - return null; - } - - /** {@inheritDoc} */ - @Override public Map<Integer, PortableMetadata> metadata(Collection<Integer> typeIds) { - return null; - } - - /** {@inheritDoc} */ - @Override public Collection<PortableMetadata> metadata() throws PortableException { - return Collections.emptyList(); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/os/package.html ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/os/package.html b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/os/package.html deleted file mode 100644 index 4febccc..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/os/package.html +++ /dev/null @@ -1,23 +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. - --> -<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd"> -<html> -<body> - <!-- Package description. --> - No-op implementation of portable processor. -</body> -</html> http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c602549a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/package.html ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/package.html b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/package.html deleted file mode 100644 index c3c7e57..0000000 --- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/portable/package.html +++ /dev/null @@ -1,23 +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. - --> -<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd"> -<html> -<body> - <!-- Package description. --> - Portable processor. -</body> -</html>
