Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 58DFE17B18 for ; Thu, 3 Sep 2015 02:16:42 +0000 (UTC) Received: (qmail 25430 invoked by uid 500); 3 Sep 2015 02:16:42 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 25356 invoked by uid 500); 3 Sep 2015 02:16:42 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 25077 invoked by uid 99); 3 Sep 2015 02:16:41 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 03 Sep 2015 02:16:41 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 9054BE7E1F; Thu, 3 Sep 2015 02:16:41 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: akuznetsov@apache.org To: commits@ignite.apache.org Date: Thu, 03 Sep 2015 02:16:48 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [08/15] ignite git commit: IGNITE-1347: Created real platform processor. IGNITE-1347: Created real platform processor. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/9910b6f6 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/9910b6f6 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/9910b6f6 Branch: refs/heads/ignite-843 Commit: 9910b6f6bb3755269c1655a2aad15319a8e7e557 Parents: 61aad3c Author: vozerov-gridgain Authored: Wed Sep 2 16:19:27 2015 +0300 Committer: vozerov-gridgain Committed: Wed Sep 2 16:19:27 2015 +0300 ---------------------------------------------------------------------- .../cache/store/CacheOsStoreManager.java | 32 +- .../cache/store/CacheStoreManager.java | 10 + .../store/GridCacheStoreManagerAdapter.java | 5 - .../platform/PlatformNoopProcessor.java | 11 +- .../processors/platform/PlatformProcessor.java | 10 + .../cache/store/PlatformCacheStore.java | 25 ++ .../platform/PlatformProcessorImpl.java | 360 +++++++++++++++++++ .../cache/store/PlatformCacheStore.java | 25 -- 8 files changed, 444 insertions(+), 34 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/9910b6f6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java index 6f73ad4..f7f5f4e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java @@ -17,8 +17,12 @@ package org.apache.ignite.internal.processors.cache.store; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.store.CacheStore; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.processors.platform.PlatformProcessor; +import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheStore; import org.apache.ignite.marshaller.portable.PortableMarshaller; /** @@ -43,6 +47,25 @@ public class CacheOsStoreManager extends GridCacheStoreManagerAdapter { } /** {@inheritDoc} */ + @Override protected void start0() throws IgniteCheckedException { + if (configured()) { + CacheStore store = configuredStore(); + + assert store != null; + assert !(store instanceof GridCacheWriteBehindStore); + + if (store instanceof PlatformCacheStore) { + PlatformProcessor proc = ctx.platform(); + + proc.registerStore((PlatformCacheStore)store, configuredConvertPortable()); + } + } + + super.start0(); + } + + + /** {@inheritDoc} */ @Override protected GridKernalContext igniteContext() { return ctx; } @@ -53,7 +76,12 @@ public class CacheOsStoreManager extends GridCacheStoreManagerAdapter { } /** {@inheritDoc} */ - @Override protected boolean convertPortable() { - return !(cfg.isKeepPortableInStore() && ctx.config().getMarshaller() instanceof PortableMarshaller); + @Override public boolean convertPortable() { + return configuredConvertPortable() && !(cfgStore instanceof PlatformCacheStore); + } + + /** {@inheritDoc} */ + @Override public boolean configuredConvertPortable() { + return !(ctx.config().getMarshaller() instanceof PortableMarshaller && cfg.isKeepPortableInStore()); } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/9910b6f6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java index f5fc473..509c806 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java @@ -186,4 +186,14 @@ public interface CacheStoreManager extends GridCacheManager { * @throws IgniteCheckedException If failed. */ public void forceFlush() throws IgniteCheckedException; + + /** + * @return Convert-portable flag. + */ + public boolean convertPortable(); + + /** + * @return Configured convert portable flag. + */ + public boolean configuredConvertPortable(); } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/9910b6f6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java index 420d011..174e1ce 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java @@ -858,11 +858,6 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt protected abstract CacheConfiguration cacheConfiguration(); /** - * @return Convert-portable flag. - */ - protected abstract boolean convertPortable(); - - /** * */ private static class SessionData { http://git-wip-us.apache.org/repos/asf/ignite/blob/9910b6f6/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java index 8cba731..0f108cf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java @@ -21,6 +21,7 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.GridProcessorAdapter; +import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheStore; import org.jetbrains.annotations.Nullable; /** @@ -101,8 +102,8 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf return null; } - @Override - public PlatformTarget events(PlatformTarget grp) { + /** {@inheritDoc} */ + @Override public PlatformTarget events(PlatformTarget grp) { return null; } @@ -115,4 +116,10 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf @Override public PlatformTarget extensions() { return null; } + + /** {@inheritDoc} */ + @Override public void registerStore(PlatformCacheStore store, boolean convertPortable) + throws IgniteCheckedException { + // No-op. + } } http://git-wip-us.apache.org/repos/asf/ignite/blob/9910b6f6/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java index a53c6b1..e1fa891 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java @@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.platform; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.GridProcessor; +import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheStore; import org.jetbrains.annotations.Nullable; /** @@ -159,4 +160,13 @@ public interface PlatformProcessor extends GridProcessor { * @return Platform extensions. */ public PlatformTarget extensions(); + + /** + * Register cache store. + * + * @param store Store. + * @param convertPortable Convert portable flag. + * @throws IgniteCheckedException If failed. + */ + public void registerStore(PlatformCacheStore store, boolean convertPortable) throws IgniteCheckedException; } http://git-wip-us.apache.org/repos/asf/ignite/blob/9910b6f6/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java new file mode 100644 index 0000000..3d54979 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java @@ -0,0 +1,25 @@ +/* + * 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.apache.ignite.internal.processors.platform.cache.store; + +/** + * Marker interface denoting that this instance is platform cache store. + */ +public interface PlatformCacheStore { + // No-op. +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/9910b6f6/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java ---------------------------------------------------------------------- diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java new file mode 100644 index 0000000..e5e7a57 --- /dev/null +++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java @@ -0,0 +1,360 @@ +/* + * 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.apache.ignite.internal.processors.platform; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.configuration.PlatformConfiguration; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteComputeImpl; +import org.apache.ignite.internal.cluster.ClusterGroupAdapter; +import org.apache.ignite.internal.portable.PortableRawWriterEx; +import org.apache.ignite.internal.processors.GridProcessorAdapter; +import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; +import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl; +import org.apache.ignite.internal.processors.platform.cache.PlatformCache; +import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinity; +import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheStore; +import org.apache.ignite.internal.processors.platform.cluster.PlatformClusterGroup; +import org.apache.ignite.internal.processors.platform.compute.PlatformCompute; +import org.apache.ignite.internal.processors.platform.datastreamer.PlatformDataStreamer; +import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore; +import org.apache.ignite.internal.processors.platform.events.PlatformEvents; +import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; +import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream; +import org.apache.ignite.internal.processors.platform.messaging.PlatformMessaging; +import org.apache.ignite.internal.processors.platform.services.PlatformServices; +import org.apache.ignite.internal.processors.platform.transactions.PlatformTransactions; +import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * GridGain platform processor. + */ +public class PlatformProcessorImpl extends GridProcessorAdapter implements PlatformProcessor { + /** Start latch. */ + private final CountDownLatch startLatch = new CountDownLatch(1); + + /** Stores pending initialization. */ + private final Collection pendingStores = + Collections.newSetFromMap(new ConcurrentHashMap()); + + /** Started stores. */ + private final Collection stores = + Collections.newSetFromMap(new ConcurrentHashMap()); + + /** Lock for store lifecycle operations. */ + private final ReadWriteLock storeLock = new ReentrantReadWriteLock(); + + /** Logger. */ + private final IgniteLogger log; + + /** Context. */ + private final PlatformContext platformCtx; + + /** Interop configuration. */ + private final PlatformConfigurationEx interopCfg; + + /** Whether processor is started. */ + private boolean started; + + /** Whether processor if stopped (or stopping). */ + private boolean stopped; + + /** + * Constructor. + * + * @param ctx Kernal context. + */ + public PlatformProcessorImpl(GridKernalContext ctx) { + super(ctx); + + log = ctx.log(PlatformProcessorImpl.class); + + PlatformConfiguration interopCfg0 = ctx.config().getPlatformConfiguration(); + + assert interopCfg0 != null : "Must be checked earlier during component creation."; + + if (!(interopCfg0 instanceof PlatformConfigurationEx)) + throw new IgniteException("Unsupported platform configuration: " + interopCfg0.getClass().getName()); + + interopCfg = (PlatformConfigurationEx)interopCfg0; + + if (!F.isEmpty(interopCfg.warnings())) { + for (String w : interopCfg.warnings()) + U.warn(log, w); + } + + platformCtx = new PlatformContextImpl(ctx, interopCfg.gate(), interopCfg.memory()); + } + + /** {@inheritDoc} */ + @Override public void start() throws IgniteCheckedException { + try (PlatformMemory mem = platformCtx.memory().allocate()) { + PlatformOutputStream out = mem.output(); + + PortableRawWriterEx writer = platformCtx.writer(out); + + writer.writeString(ctx.gridName()); + + out.synchronize(); + + platformCtx.gateway().onStart(mem.pointer()); + } + + // At this moment all necessary native libraries must be loaded, so we can process with store creation. + storeLock.writeLock().lock(); + + try { + for (StoreInfo store : pendingStores) + registerStore0(store.store, store.convertPortable); + + pendingStores.clear(); + + started = true; + } + finally { + storeLock.writeLock().unlock(); + } + + // Add Interop node attributes. + ctx.addNodeAttribute(PlatformUtils.ATTR_PLATFORM, interopCfg.platform()); + } + + /** {@inheritDoc} */ + @Override public void onKernalStop(boolean cancel) { + startLatch.countDown(); + } + + /** {@inheritDoc} */ + @Override public void stop(boolean cancel) throws IgniteCheckedException { + if (platformCtx != null) { + // Destroy cache stores. + storeLock.writeLock().lock(); + + try { + for (PlatformCacheStore store : stores) { + if (store != null) { + if (store instanceof PlatformDotNetCacheStore) { + PlatformDotNetCacheStore store0 = (PlatformDotNetCacheStore)store; + + try { + store0.destroy(platformCtx.kernalContext()); + } + catch (Exception e) { + U.error(log, "Failed to destroy .Net cache store [store=" + store0 + + ", err=" + e.getMessage() + ']'); + } + } + else + assert false : "Invalid interop cache store type: " + store; + } + } + } + finally { + stopped = true; + + storeLock.writeLock().unlock(); + } + + platformCtx.gateway().onStop(); + } + } + + /** {@inheritDoc} */ + @Override public Ignite ignite() { + return ctx.grid(); + } + + /** {@inheritDoc} */ + @Override public long environmentPointer() { + return platformCtx.gateway().environmentPointer(); + } + + /** {@inheritDoc} */ + public void releaseStart() { + startLatch.countDown(); + } + + /** {@inheritDoc} */ + public void awaitStart() throws IgniteCheckedException { + U.await(startLatch); + } + + /** {@inheritDoc} */ + @Override public PlatformContext context() { + return platformCtx; + } + + /** {@inheritDoc} */ + @Override public PlatformTarget cache(@Nullable String name) throws IgniteCheckedException { + IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().cache(name); + + if (cache == null) + throw new IllegalArgumentException("Cache doesn't exist: " + name); + + return new PlatformCache(platformCtx, cache.keepPortable(), false); + } + + /** {@inheritDoc} */ + @Override public PlatformTarget createCache(@Nullable String name) throws IgniteCheckedException { + IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().createCache(name); + + assert cache != null; + + return new PlatformCache(platformCtx, cache.keepPortable(), false); + } + + /** {@inheritDoc} */ + @Override public PlatformTarget getOrCreateCache(@Nullable String name) throws IgniteCheckedException { + IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().getOrCreateCache(name); + + assert cache != null; + + return new PlatformCache(platformCtx, cache.keepPortable(), false); + } + + /** {@inheritDoc} */ + @Override public PlatformTarget affinity(@Nullable String name) throws IgniteCheckedException { + return new PlatformAffinity(platformCtx, ctx, name); + } + + /** {@inheritDoc} */ + @Override public PlatformTarget dataStreamer(@Nullable String cacheName, boolean keepPortable) + throws IgniteCheckedException { + IgniteDataStreamer ldr = ctx.dataStream().dataStreamer(cacheName); + + return new PlatformDataStreamer(platformCtx, cacheName, (DataStreamerImpl)ldr, keepPortable); + } + + /** {@inheritDoc} */ + @Override public PlatformTarget transactions() { + return new PlatformTransactions(platformCtx); + } + + /** {@inheritDoc} */ + @Override public PlatformTarget projection() throws IgniteCheckedException { + return new PlatformClusterGroup(platformCtx, ctx.grid().cluster()); + } + + /** {@inheritDoc} */ + @Override public PlatformTarget compute(PlatformTarget grp) { + PlatformClusterGroup grp0 = (PlatformClusterGroup)grp; + + assert grp0.projection() instanceof ClusterGroupAdapter; // Safety for very complex ClusterGroup hierarchy. + + return new PlatformCompute(platformCtx, (IgniteComputeImpl)((ClusterGroupAdapter)grp0.projection()).compute()); + } + + /** {@inheritDoc} */ + @Override public PlatformTarget message(PlatformTarget grp) { + PlatformClusterGroup grp0 = (PlatformClusterGroup)grp; + + return new PlatformMessaging(platformCtx, grp0.projection().ignite().message(grp0.projection())); + } + + /** {@inheritDoc} */ + @Override public PlatformTarget events(PlatformTarget grp) { + PlatformClusterGroup grp0 = (PlatformClusterGroup)grp; + + return new PlatformEvents(platformCtx, grp0.projection().ignite().events(grp0.projection())); + } + + /** {@inheritDoc} */ + @Override public PlatformTarget services(PlatformTarget grp) { + PlatformClusterGroup grp0 = (PlatformClusterGroup)grp; + + return new PlatformServices(platformCtx, grp0.projection().ignite().services(grp0.projection()), false); + } + + /** {@inheritDoc} */ + @Override public PlatformTarget extensions() { + return null; + } + + /** {@inheritDoc} */ + @Override public void registerStore(PlatformCacheStore store, boolean convertPortable) + throws IgniteCheckedException { + storeLock.readLock().lock(); + + try { + if (stopped) + throw new IgniteCheckedException("Failed to initialize interop store becuase node is stopping: " + + store); + + if (started) + registerStore0(store, convertPortable); + else + pendingStores.add(new StoreInfo(store, convertPortable)); + } + finally { + storeLock.readLock().unlock(); + } + } + + /** + * Internal store initialization routine. + * + * @param store Store. + * @param convertPortable Convert portable flag. + * @throws IgniteCheckedException If failed. + */ + private void registerStore0(PlatformCacheStore store, boolean convertPortable) throws IgniteCheckedException { + if (store instanceof PlatformDotNetCacheStore) { + PlatformDotNetCacheStore store0 = (PlatformDotNetCacheStore)store; + + store0.initialize(ctx, convertPortable); + } + else + throw new IgniteCheckedException("Unsupported interop store: " + store); + } + + /** + * Store and manager pair. + */ + private static class StoreInfo { + /** Store. */ + private final PlatformCacheStore store; + + /** Convert portable flag. */ + private final boolean convertPortable; + + /** + * Constructor. + * + * @param store Store. + * @param convertPortable Convert portable flag. + */ + private StoreInfo(PlatformCacheStore store, boolean convertPortable) { + this.store = store; + this.convertPortable = convertPortable; + } + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/9910b6f6/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java ---------------------------------------------------------------------- diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java deleted file mode 100644 index 3d54979..0000000 --- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java +++ /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. - */ - -package org.apache.ignite.internal.processors.platform.cache.store; - -/** - * Marker interface denoting that this instance is platform cache store. - */ -public interface PlatformCacheStore { - // No-op. -} \ No newline at end of file