You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pt...@apache.org on 2016/09/09 12:25:54 UTC

[1/2] ignite git commit: Decouple session logic from PlatformCache

Repository: ignite
Updated Branches:
  refs/heads/ignite-3199-1 8fa16adb4 -> f22c0a022


Decouple session logic from PlatformCache


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/562dc1ee
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/562dc1ee
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/562dc1ee

Branch: refs/heads/ignite-3199-1
Commit: 562dc1ee62ff6acb1b449e017d6808d4747ca9fc
Parents: 8fa16ad
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Sep 9 15:24:01 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Sep 9 15:24:01 2016 +0300

----------------------------------------------------------------------
 .../platform/PlatformProcessorImpl.java         | 26 ++++--
 .../platform/cache/PlatformCache.java           | 42 +++++++---
 .../platform/cache/PlatformCacheExtension.java  | 28 +++++++
 .../cache/PlatformCacheExtensionResult.java     | 37 ++++++++
 .../platform/cache/PlatformCacheInvoker.java    | 88 --------------------
 .../PlatformDotnetSessionCacheExtension.java    | 77 +++++++++++++++++
 6 files changed, 191 insertions(+), 107 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/562dc1ee/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
index 0fefc33..351e4e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl;
 import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicLongImpl;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCache;
+import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension;
 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;
@@ -53,6 +54,7 @@ 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.PlatformConfigurationUtils;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
+import org.apache.ignite.internal.processors.platform.websession.PlatformDotnetSessionCacheExtension;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteFuture;
@@ -94,6 +96,9 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     /** Whether processor if stopped (or stopping). */
     private volatile boolean stopped;
 
+    /** Session state extension. */
+    private final PlatformCacheExtension sessionExt = new PlatformDotnetSessionCacheExtension();
+
     /**
      * Constructor.
      *
@@ -211,7 +216,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
         if (cache == null)
             throw new IllegalArgumentException("Cache doesn't exist: " + name);
 
-        return new PlatformCache(platformCtx, cache, false);
+        return createPlatformCache(cache);
     }
 
     /** {@inheritDoc} */
@@ -220,7 +225,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
 
         assert cache != null;
 
-        return new PlatformCache(platformCtx, cache, false);
+        return createPlatformCache(cache);
     }
 
     /** {@inheritDoc} */
@@ -229,7 +234,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
 
         assert cache != null;
 
-        return new PlatformCache(platformCtx, cache, false);
+        return createPlatformCache(cache);
     }
 
     /** {@inheritDoc} */
@@ -241,7 +246,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
             ? (IgniteCacheProxy)ctx.grid().createCache(cfg, PlatformConfigurationUtils.readNearConfiguration(reader))
             : (IgniteCacheProxy)ctx.grid().createCache(cfg);
 
-        return new PlatformCache(platformCtx, cache, false);
+        return createPlatformCache(cache);
     }
 
     /** {@inheritDoc} */
@@ -254,7 +259,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
                     PlatformConfigurationUtils.readNearConfiguration(reader))
             : (IgniteCacheProxy)ctx.grid().getOrCreateCache(cfg);
 
-        return new PlatformCache(platformCtx, cache, false);
+        return createPlatformCache(cache);
     }
 
     /** {@inheritDoc} */
@@ -408,7 +413,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
 
         IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().createNearCache(cacheName, cfg);
 
-        return new PlatformCache(platformCtx, cache, false);
+        return createPlatformCache(cache);
     }
 
     /** {@inheritDoc} */
@@ -417,7 +422,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
 
         IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().getOrCreateNearCache(cacheName, cfg);
 
-        return new PlatformCache(platformCtx, cache, false);
+        return createPlatformCache(cache);
     }
 
     /** {@inheritDoc} */
@@ -473,6 +478,13 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /**
+     * Creates new platform cache.
+     */
+    private PlatformTarget createPlatformCache(IgniteCacheProxy cache) {
+        return new PlatformCache(platformCtx, cache, false, sessionExt);
+    }
+
+    /**
      * Gets the near cache config.
      *
      * @param memPtr Memory pointer.

http://git-wip-us.apache.org/repos/asf/ignite/blob/562dc1ee/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
index 9bf1d47..1bdf139 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -29,7 +29,7 @@ import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.cache.query.TextQuery;
-import org.apache.ignite.configuration.*;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
@@ -53,8 +53,8 @@ import org.apache.ignite.internal.processors.platform.utils.PlatformWriterClosur
 import org.apache.ignite.internal.util.GridConcurrentFactory;
 import org.apache.ignite.internal.util.future.IgniteFutureImpl;
 import org.apache.ignite.internal.util.typedef.C1;
-import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgniteFuture;
 import org.jetbrains.annotations.Nullable;
 
@@ -222,6 +222,9 @@ public class PlatformCache extends PlatformAbstractTarget {
     /** Lock ID sequence. */
     private static final AtomicLong LOCK_ID_GEN = new AtomicLong();
 
+    /** Extensions. */
+    private final PlatformCacheExtension[] extensions;
+
     /**
      * Constructor.
      *
@@ -229,13 +232,15 @@ public class PlatformCache extends PlatformAbstractTarget {
      * @param cache Underlying cache.
      * @param keepBinary Keep binary flag.
      */
-    public PlatformCache(PlatformContext platformCtx, IgniteCache cache, boolean keepBinary) {
+    public PlatformCache(PlatformContext platformCtx, IgniteCache cache, boolean keepBinary,
+        PlatformCacheExtension... extensions) {
         super(platformCtx);
 
         cacheRaw = cache;
 
         this.cache = (IgniteCacheProxy)cache.withKeepBinary();
         this.keepBinary = keepBinary;
+        this.extensions = extensions;
     }
 
     /**
@@ -247,7 +252,7 @@ public class PlatformCache extends PlatformAbstractTarget {
         if (cache.delegate().skipStore())
             return this;
 
-        return new PlatformCache(platformCtx, cacheRaw.withSkipStore(), keepBinary);
+        return clone(cacheRaw.withSkipStore(), keepBinary);
     }
 
     /**
@@ -259,7 +264,7 @@ public class PlatformCache extends PlatformAbstractTarget {
         if (keepBinary)
             return this;
 
-        return new PlatformCache(platformCtx, cacheRaw.withKeepBinary(), true);
+        return clone(cacheRaw.withKeepBinary(), true);
     }
 
     /**
@@ -273,7 +278,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     public PlatformCache withExpiryPolicy(final long create, final long update, final long access) {
         IgniteCache cache0 = cacheRaw.withExpiryPolicy(new InteropExpiryPolicy(create, update, access));
 
-        return new PlatformCache(platformCtx, cache0, keepBinary);
+        return clone(cache0, keepBinary);
     }
 
     /**
@@ -285,7 +290,7 @@ public class PlatformCache extends PlatformAbstractTarget {
         if (cache.isAsync())
             return this;
 
-        return new PlatformCache(platformCtx, cacheRaw.withAsync(), keepBinary);
+        return clone(cacheRaw.withAsync(), keepBinary);
     }
 
     /**
@@ -299,7 +304,7 @@ public class PlatformCache extends PlatformAbstractTarget {
         if (opCtx != null && opCtx.noRetries())
             return this;
 
-        return new PlatformCache(platformCtx, cacheRaw.withNoRetries(), keepBinary);
+        return clone(cacheRaw.withNoRetries(), keepBinary);
     }
 
     /** {@inheritDoc} */
@@ -464,10 +469,16 @@ public class PlatformCache extends PlatformAbstractTarget {
                     return registerLock(cache.lockAll(PlatformUtils.readCollection(reader)));
 
                 case OP_EXTENSION:
-                    // TODO: Refactor this
-                    // Either inject this logic somehow (use negative op ids or something)
-                    // Or inherit the class (needs a new JNI call)
-                    return writeResult(mem, PlatformCacheInvoker.invoke(reader, cacheRaw));
+                    int opCode = reader.readInt();
+
+                    for (PlatformCacheExtension extension : extensions) {
+                        PlatformCacheExtensionResult res = extension.invoke(opCode, reader, cacheRaw);
+
+                        if (res.isMatch())
+                            return writeResult(mem, res.result());
+                    }
+
+                    throw new IgniteCheckedException("Unsupported cache extension type: " + type);
             }
         }
         catch (Exception e) {
@@ -1020,6 +1031,13 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /**
+     * Clones this instance.
+     */
+    private PlatformCache clone(IgniteCache cache, boolean keepBinary) {
+        return new PlatformCache(platformCtx, cache, keepBinary, extensions);
+    }
+
+    /**
      * Writes error with EntryProcessorException cause.
      */
     private static class GetAllWriter implements PlatformFutureUtils.Writer {

http://git-wip-us.apache.org/repos/asf/ignite/blob/562dc1ee/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtension.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtension.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtension.java
new file mode 100644
index 0000000..386cd9e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtension.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryRawReader;
+
+/**
+ * Platform cache extension.
+ */
+public interface PlatformCacheExtension {
+    PlatformCacheExtensionResult invoke(int opCode, BinaryRawReader reader, IgniteCache cache);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/562dc1ee/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtensionResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtensionResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtensionResult.java
new file mode 100644
index 0000000..a6b8d1f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtensionResult.java
@@ -0,0 +1,37 @@
+package org.apache.ignite.internal.processors.platform.cache;
+
+/**
+ * Result of {@link PlatformCacheExtension} invocation.
+ */
+public class PlatformCacheExtensionResult {
+    /** Whether extension handled the case. */
+    private final boolean isMatch;
+
+    /** Result. */
+    private final Object result;
+
+    /**
+     * Ctor.
+     *
+     * @param isMatch Whether extension handled the case.
+     * @param result Invocation result.
+     */
+    public PlatformCacheExtensionResult(boolean isMatch, Object result) {
+        this.isMatch = isMatch;
+        this.result = result;
+    }
+
+    /**
+     * @return Whether extension handled the case.
+     */
+    public boolean isMatch() {
+        return isMatch;
+    }
+
+    /**
+     * @return Invocation result.
+     */
+    public Object result() {
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/562dc1ee/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheInvoker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheInvoker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheInvoker.java
deleted file mode 100644
index a923a61..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheInvoker.java
+++ /dev/null
@@ -1,88 +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;
-
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.binary.BinaryRawReader;
-import org.apache.ignite.internal.processors.platform.websession.PlatformDotnetSessionData;
-import org.apache.ignite.internal.processors.platform.websession.PlatformDotnetSessionLockProcessor;
-import org.apache.ignite.internal.processors.platform.websession.PlatformDotnetSessionSetAndUnlockProcessor;
-
-import java.sql.Timestamp;
-import java.util.UUID;
-
-/**
- * Custom entry processor invoker.
- */
-public class PlatformCacheInvoker {
-    /** */
-    public static final int OP_SESSION_LOCK = 1;
-
-    /** */
-    public static final int OP_SESSION_SET_AND_UNLOCK = 2;
-
-    /**
-     * Invokes the custom processor.
-     *
-     * @param reader Reader.
-     * @param cache Cache.
-     *
-     * @return Result.
-     */
-    @SuppressWarnings("unchecked")
-    public static Object invoke(BinaryRawReader reader, IgniteCache cache) {
-        int opCode = reader.readInt();
-
-        String key = reader.readString();
-
-        Object res = null;
-
-        switch (opCode) {
-            case OP_SESSION_LOCK: {
-                UUID lockNodeId = reader.readUuid();
-                long lockId = reader.readLong();
-                Timestamp lockTime = reader.readTimestamp();
-
-                res = cache.invoke(key, new PlatformDotnetSessionLockProcessor(lockNodeId, lockId, lockTime));
-
-                break;
-            }
-
-            case OP_SESSION_SET_AND_UNLOCK:
-                PlatformDotnetSessionSetAndUnlockProcessor proc;
-
-                if (reader.readBoolean()) {
-                    PlatformDotnetSessionData data = reader.readObject();
-
-                    proc = new PlatformDotnetSessionSetAndUnlockProcessor(data);
-                }
-                else {
-                    UUID lockNodeId = reader.readUuid();
-                    long lockId = reader.readLong();
-
-                    proc = new PlatformDotnetSessionSetAndUnlockProcessor(lockNodeId, lockId);
-                }
-
-                cache.invoke(key, proc);
-
-                break;
-        }
-
-        return res;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/562dc1ee/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotnetSessionCacheExtension.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotnetSessionCacheExtension.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotnetSessionCacheExtension.java
new file mode 100644
index 0000000..7776c13
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotnetSessionCacheExtension.java
@@ -0,0 +1,77 @@
+/*
+ * 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.websession;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension;
+import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtensionResult;
+
+import java.sql.Timestamp;
+import java.util.UUID;
+
+/**
+ * Custom entry processor invoker.
+ */
+public class PlatformDotnetSessionCacheExtension implements PlatformCacheExtension {
+    /** */
+    public static final int OP_SESSION_LOCK = 1;
+
+    /** */
+    public static final int OP_SESSION_SET_AND_UNLOCK = 2;
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public PlatformCacheExtensionResult invoke(int opCode, BinaryRawReader reader, IgniteCache cache) {
+        switch (opCode) {
+            case OP_SESSION_LOCK: {
+                String key = reader.readString();
+                UUID lockNodeId = reader.readUuid();
+                long lockId = reader.readLong();
+                Timestamp lockTime = reader.readTimestamp();
+
+                Object res = cache.invoke(key, new PlatformDotnetSessionLockProcessor(lockNodeId, lockId, lockTime));
+
+                return new PlatformCacheExtensionResult(true, res);
+            }
+
+            case OP_SESSION_SET_AND_UNLOCK:
+                String key = reader.readString();
+
+                PlatformDotnetSessionSetAndUnlockProcessor proc;
+
+                if (reader.readBoolean()) {
+                    PlatformDotnetSessionData data = reader.readObject();
+
+                    proc = new PlatformDotnetSessionSetAndUnlockProcessor(data);
+                }
+                else {
+                    UUID lockNodeId = reader.readUuid();
+                    long lockId = reader.readLong();
+
+                    proc = new PlatformDotnetSessionSetAndUnlockProcessor(lockNodeId, lockId);
+                }
+
+                cache.invoke(key, proc);
+
+                return new PlatformCacheExtensionResult(true, null);
+        }
+
+        return new PlatformCacheExtensionResult(false, null);
+    }
+}


[2/2] ignite git commit: Fix file header

Posted by pt...@apache.org.
Fix file header


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/f22c0a02
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/f22c0a02
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/f22c0a02

Branch: refs/heads/ignite-3199-1
Commit: f22c0a0228167b604f33eaa6b74ce63c5382d824
Parents: 562dc1e
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Sep 9 15:24:18 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Sep 9 15:24:18 2016 +0300

----------------------------------------------------------------------
 .../cache/PlatformCacheExtensionResult.java        | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f22c0a02/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtensionResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtensionResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtensionResult.java
index a6b8d1f..bec5f46 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtensionResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtensionResult.java
@@ -1,3 +1,20 @@
+/*
+ * 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;
 
 /**