You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2015/11/10 13:21:48 UTC

[8/9] ignite git commit: IGNITE-1847: Refactoring metadata handlers.

IGNITE-1847: Refactoring metadata handlers.


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

Branch: refs/heads/ignite-1847
Commit: fc49be04494b80d202db564d766e8aa9526042b8
Parents: a6e2148
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Nov 10 15:10:00 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Nov 10 15:10:00 2015 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    |  2 +-
 .../portable/BinaryFieldsAbstractSelfTest.java  | 24 +++--------
 ...idPortableMarshallerCtxDisabledSelfTest.java | 33 +++++---------
 .../GridPortableMarshallerSelfTest.java         | 13 +-----
 .../portable/GridPortableWildcardsSelfTest.java | 23 +++-------
 .../PortableCompactOffsetsAbstractSelfTest.java | 21 +++------
 .../internal/portable/TestMetadataHandler.java  | 45 ++++++++++++++++++++
 .../portable/TestNoopMetadataHandler.java       | 36 ++++++++++++++++
 ...ridPortableCacheEntryMemorySizeSelfTest.java | 14 +-----
 9 files changed, 113 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index c4deaa0..4d1145d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -1875,7 +1875,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             if (field0 == null)
             {
-                field0 = obj.fieldDescriptor(propName);
+                field0 = obj.type().field(propName);
 
                 assert field0 != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
index 4fa80b4..1ee36d8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
@@ -17,15 +17,14 @@
 
 package org.apache.ignite.internal.portable;
 
+import org.apache.ignite.binary.BinaryField;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.binary.BinaryField;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import java.math.BigDecimal;
@@ -38,17 +37,6 @@ import java.util.UUID;
  * Contains tests for portable object fields.
  */
 public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTest {
-    /** Dummy metadata handler. */
-    protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, BinaryType meta) {
-            // No-op.
-        }
-
-        @Override public BinaryType metadata(int typeId) {
-            return null;
-        }
-    };
-
     /** Marshaller. */
     protected PortableMarshaller dfltMarsh;
 
@@ -59,7 +47,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
      * @throws Exception If failed.
      */
     protected static PortableMarshaller createMarshaller() throws Exception {
-        PortableContext ctx = new PortableContext(META_HND, new IgniteConfiguration());
+        PortableContext ctx = new PortableContext(new TestMetadataHandler(), new IgniteConfiguration());
 
         PortableMarshaller marsh = new PortableMarshaller();
 
@@ -485,7 +473,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
 
         BinaryObjectEx portObj = toPortable(marsh, obj);
 
-        BinaryField field = portObj.fieldDescriptor(fieldName);
+        BinaryField field = portObj.type().field(fieldName);
 
         return new TestContext(obj, portObj, field);
     }
@@ -508,7 +496,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
 
         assert portObj != null;
 
-        BinaryField field = portObj.fieldDescriptor(fieldName);
+        BinaryField field = portObj.type().field(fieldName);
 
         return new TestContext(obj, portObj, field);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
index ee91167..d10d3f3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
@@ -17,38 +17,27 @@
 
 package org.apache.ignite.internal.portable;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.Arrays;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.MarshallerContextAdapter;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.Binarylizable;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryReader;
-import org.apache.ignite.binary.BinaryWriter;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Arrays;
+
 /**
  *
  */
 public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstractTest {
-    /** */
-    protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, BinaryType meta) {
-            // No-op.
-        }
-
-        @Override public BinaryType metadata(int typeId) {
-            return null;
-        }
-    };
-
     /**
      * @throws Exception If failed.
      */
@@ -56,7 +45,7 @@ public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstrac
         PortableMarshaller marsh = new PortableMarshaller();
         marsh.setContext(new MarshallerContextWithNoStorage());
 
-        PortableContext context = new PortableContext(META_HND, new IgniteConfiguration());
+        PortableContext context = new PortableContext(new TestNoopMetadataHandler(), new IgniteConfiguration());
 
         IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", context);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
index ea96842..d7007de 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
@@ -82,17 +82,6 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     /** */
     protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
 
-    /** */
-    protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, BinaryType meta) {
-            // No-op.
-        }
-
-        @Override public BinaryType metadata(int typeId) {
-            return null;
-        }
-    };
-
     /**
      * @throws Exception If failed.
      */
@@ -2414,7 +2403,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     protected PortableContext initPortableContext(PortableMarshaller marsh) throws IgniteCheckedException {
         IgniteConfiguration iCfg = new IgniteConfiguration();
 
-        PortableContext ctx = new PortableContext(META_HND, iCfg);
+        PortableContext ctx = new PortableContext(new TestNoopMetadataHandler(), iCfg);
 
         marsh.setContext(new MarshallerContextTestImpl(null));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
index 22125af..ee90beb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
@@ -17,37 +17,26 @@
 
 package org.apache.ignite.internal.portable;
 
-import java.util.Arrays;
-import java.util.Map;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.binary.BinaryTypeIdMapper;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.binary.BinaryTypeIdMapper;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.util.Arrays;
+import java.util.Map;
+
 /**
  * Wildcards test.
  */
 public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, BinaryType meta) {
-            // No-op.
-        }
-
-        @Override public BinaryType metadata(int typeId) {
-            return null;
-        }
-    };
-
     /**
      * @return Portable context.
      */
     private PortableContext portableContext() {
-        return new PortableContext(META_HND, new IgniteConfiguration());
+        return new PortableContext(new TestNoopMetadataHandler(), new IgniteConfiguration());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
index db3c821..4fa2324 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java
@@ -17,9 +17,7 @@
 
 package org.apache.ignite.internal.portable;
 
-import java.util.Arrays;
 import org.apache.ignite.binary.BinaryField;
-import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
@@ -27,6 +25,8 @@ import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.util.Arrays;
+
 /**
  * Contains tests for compact offsets.
  */
@@ -37,17 +37,6 @@ public abstract class PortableCompactOffsetsAbstractSelfTest extends GridCommonA
     /** 2 pow 16. */
     private static int POW_16 = 1 << 16;
 
-    /** Dummy metadata handler. */
-    protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, BinaryType meta) {
-            // No-op.
-        }
-
-        @Override public BinaryType metadata(int typeId) {
-            return null;
-        }
-    };
-
     /** Marshaller. */
     protected PortableMarshaller marsh;
 
@@ -58,7 +47,7 @@ public abstract class PortableCompactOffsetsAbstractSelfTest extends GridCommonA
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
 
-        ctx = new PortableContext(META_HND, new IgniteConfiguration());
+        ctx = new PortableContext(new TestMetadataHandler(), new IgniteConfiguration());
 
         marsh = new PortableMarshaller();
 
@@ -139,8 +128,8 @@ public abstract class PortableCompactOffsetsAbstractSelfTest extends GridCommonA
         assert obj.field2 == field2;
 
         // 2. Test fields API.
-        BinaryField field1Desc = portObj.fieldDescriptor("field1");
-        BinaryField field2Desc = portObj.fieldDescriptor("field2");
+        BinaryField field1Desc = portObj.type().field("field1");
+        BinaryField field2Desc = portObj.type().field("field2");
 
         assert field1Desc.exists(portObj);
         assert field2Desc.exists(portObj);

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java
new file mode 100644
index 0000000..c25028d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java
@@ -0,0 +1,45 @@
+/*
+ * 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.portable;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryType;
+
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Test metadata handler.
+ */
+public class TestMetadataHandler implements PortableMetaDataHandler {
+    /** Cached metadatas. */
+    private final ConcurrentHashMap<Integer, BinaryType> metas = new ConcurrentHashMap<>();
+
+    /** {@inheritDoc} */
+    @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException {
+        BinaryType otherType = metas.put(typeId, meta);
+
+        if (otherType != null)
+            throw new IllegalStateException("Metadata replacement is not allowed in " +
+                TestMetadataHandler.class.getSimpleName() + '.');
+    }
+
+    /** {@inheritDoc} */
+    @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
+        return metas.get(typeId);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java
new file mode 100644
index 0000000..c504365
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java
@@ -0,0 +1,36 @@
+/*
+ * 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.portable;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryType;
+
+/**
+ * No-op metadata handler.
+ */
+public class TestNoopMetadataHandler implements PortableMetaDataHandler {
+    /** {@inheritDoc} */
+    @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc49be04/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
index d277801..c7a80e3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
@@ -20,14 +20,12 @@ package org.apache.ignite.internal.processors.cache.portable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.portable.PortableContext;
-import org.apache.ignite.internal.portable.PortableMetaDataHandler;
+import org.apache.ignite.internal.portable.TestNoopMetadataHandler;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryMemorySizeSelfTest;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryType;
 
 /**
  *
@@ -39,15 +37,7 @@ public class GridPortableCacheEntryMemorySizeSelfTest extends GridCacheEntryMemo
 
         marsh.setContext(new MarshallerContextTestImpl(null));
 
-        PortableContext pCtx = new PortableContext(new PortableMetaDataHandler() {
-            @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException {
-                // No-op
-            }
-
-            @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
-                return null;
-            }
-        }, new IgniteConfiguration());
+        PortableContext pCtx = new PortableContext(new TestNoopMetadataHandler(), new IgniteConfiguration());
 
         IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", pCtx);