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 2016/08/13 19:14:23 UTC

[1/3] ignite git commit: Added base infrastructure classes.

Repository: ignite
Updated Branches:
  refs/heads/ignite-3681 [created] 2376ac1d6


Added base infrastructure classes.


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

Branch: refs/heads/ignite-3681
Commit: bc33643e442192b5ef911e0ef31d7a0e3e2d9f85
Parents: 1139a9f
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Sat Aug 13 21:27:44 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Sat Aug 13 21:27:44 2016 +0300

----------------------------------------------------------------------
 .../internal/binary/BinaryClassDescriptor.java  | 36 +++++---
 .../binary/BinaryMethodWriteReplacer.java       | 59 ++++++++++++
 .../ignite/internal/binary/BinaryTreeMap.java   | 96 ++++++++++++++++++++
 .../binary/BinaryTreeMapWriteReplacer.java      | 34 +++++++
 .../ignite/internal/binary/BinaryTreeSet.java   | 93 +++++++++++++++++++
 .../binary/BinaryTreeSetWriteReplacer.java      | 34 +++++++
 .../internal/binary/BinaryWriteReplacer.java    | 33 +++++++
 .../internal/binary/BinaryWriterExImpl.java     | 35 +++----
 8 files changed, 385 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bc33643e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
index d2d715b..2588f59 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
@@ -89,8 +89,8 @@ public class BinaryClassDescriptor {
     /** */
     private final BinaryFieldAccessor[] fields;
 
-    /** */
-    private final Method writeReplaceMtd;
+    /** Write replacer. */
+    private final BinaryWriteReplacer writeReplacer;
 
     /** */
     private final Method readResolveMtd;
@@ -298,11 +298,8 @@ public class BinaryClassDescriptor {
 
                             schemaBuilder.addField(fieldId);
 
-                            if (metaDataEnabled) {
-                                assert stableFieldsMeta != null;
-
+                            if (metaDataEnabled)
                                 stableFieldsMeta.put(name, fieldInfo.mode().typeId());
-                            }
                         }
                     }
                 }
@@ -320,14 +317,19 @@ public class BinaryClassDescriptor {
                 throw new BinaryObjectException("Invalid mode: " + mode);
         }
 
+        Method writeReplaceMthd;
+
         if (mode == BinaryWriteMode.BINARY || mode == BinaryWriteMode.OBJECT) {
             readResolveMtd = U.findNonPublicMethod(cls, "readResolve");
-            writeReplaceMtd = U.findNonPublicMethod(cls, "writeReplace");
+
+            writeReplaceMthd = U.findNonPublicMethod(cls, "writeReplace");
         }
         else {
             readResolveMtd = null;
-            writeReplaceMtd = null;
+            writeReplaceMthd = null;
         }
+
+        writeReplacer = writeReplaceMthd == null ? null : new BinaryMethodWriteReplacer(writeReplaceMthd);
     }
 
     /**
@@ -469,10 +471,22 @@ public class BinaryClassDescriptor {
     }
 
     /**
-     * @return binaryWriteReplace() method
+     * @return {@code True} if write-replace should be performed for class.
      */
-    @Nullable Method getWriteReplaceMethod() {
-        return writeReplaceMtd;
+    public boolean isWriteReplace() {
+        return writeReplacer != null;
+    }
+
+    /**
+     * Perform write replace.
+     *
+     * @param obj Original object.
+     * @return Replaced object.
+     */
+    public Object writeReplace(Object obj) {
+        assert isWriteReplace();
+
+        return writeReplacer.replace(obj);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/bc33643e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMethodWriteReplacer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMethodWriteReplacer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMethodWriteReplacer.java
new file mode 100644
index 0000000..783048e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMethodWriteReplacer.java
@@ -0,0 +1,59 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.jetbrains.annotations.Nullable;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+/**
+ * Write replacer based on method invocation.
+ */
+public class BinaryMethodWriteReplacer implements BinaryWriteReplacer {
+    /** Method. */
+    private final Method mthd;
+
+    /**
+     * Constructor.
+     *
+     * @param mthd Method.
+     */
+    public BinaryMethodWriteReplacer(Method mthd) {
+        assert mthd != null;
+
+        this.mthd = mthd;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object replace(Object target) {
+        try {
+            return mthd.invoke(target);
+        }
+        catch (IllegalAccessException e) {
+            throw new RuntimeException(e);
+        }
+        catch (InvocationTargetException e) {
+            if (e.getTargetException() instanceof BinaryObjectException)
+                throw (BinaryObjectException)e.getTargetException();
+
+            throw new BinaryObjectException("Failed to execute writeReplace() method on " + target, e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bc33643e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMap.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMap.java
new file mode 100644
index 0000000..6a7cf9b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMap.java
@@ -0,0 +1,96 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+
+import java.io.ObjectStreamException;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * Binary {@link TreeMap} wrapper.
+ */
+public class BinaryTreeMap implements Binarylizable {
+    /** Original map. */
+    private TreeMap map;
+
+    /**
+     * Default constructor.
+     */
+    public BinaryTreeMap() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param map Original map.
+     */
+    public BinaryTreeMap(TreeMap map) {
+        this.map = map;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter rawWriter = writer.rawWriter();
+
+        rawWriter.writeObject(map.comparator());
+
+        int size = map.size();
+
+        rawWriter.writeInt(size);
+
+        for (Map.Entry<Object, Object> entry : ((TreeMap<Object, Object>)map).entrySet()) {
+            rawWriter.writeObject(entry.getKey());
+            rawWriter.writeObject(entry.getValue());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader rawReader = reader.rawReader();
+
+        Comparator comp =  rawReader.readObject();
+
+        map = comp == null ? new TreeMap() : new TreeMap(comp);
+
+        int size = rawReader.readInt();
+
+        for (int i = 0; i < size; i++)
+            map.put(rawReader.readObject(), rawReader.readObject());
+    }
+
+    /**
+     * Reconstructs object on unmarshalling.
+     *
+     * @return Reconstructed object.
+     * @throws ObjectStreamException Thrown in case of unmarshalling error.
+     */
+    protected Object readResolve() throws ObjectStreamException {
+        return map;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bc33643e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMapWriteReplacer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMapWriteReplacer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMapWriteReplacer.java
new file mode 100644
index 0000000..049db8e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMapWriteReplacer.java
@@ -0,0 +1,34 @@
+/*
+ * 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.binary;
+
+import org.jetbrains.annotations.Nullable;
+
+import java.util.TreeMap;
+
+/**
+ * Binary {@link TreeMap} write replacer.
+ */
+public class BinaryTreeMapWriteReplacer implements BinaryWriteReplacer {
+    /** {@inheritDoc} */
+    @Nullable @Override public Object replace(Object target) {
+        assert target instanceof TreeMap;
+
+        return new BinaryTreeMap((TreeMap)target);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/bc33643e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSet.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSet.java
new file mode 100644
index 0000000..2b01528
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSet.java
@@ -0,0 +1,93 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+
+import java.io.ObjectStreamException;
+import java.util.Comparator;
+import java.util.TreeSet;
+
+/**
+ * Binary {@link TreeSet} wrapper.
+ */
+public class BinaryTreeSet implements Binarylizable {
+    /** Original set. */
+    private TreeSet set;
+
+    /**
+     * Default constructor.
+     */
+    public BinaryTreeSet() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param set Original set.
+     */
+    public BinaryTreeSet(TreeSet set) {
+        this.set = set;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter rawWriter = writer.rawWriter();
+
+        rawWriter.writeObject(set.comparator());
+
+        int size = set.size();
+
+        rawWriter.writeInt(size);
+
+        for (Object val : set)
+            rawWriter.writeObject(val);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader rawReader = reader.rawReader();
+
+        Comparator comp =  rawReader.readObject();
+
+        set = comp == null ? new TreeSet() : new TreeSet(comp);
+
+        int size = rawReader.readInt();
+
+        for (int i = 0; i < size; i++)
+            set.add(rawReader.readObject());
+    }
+
+    /**
+     * Reconstructs object on unmarshalling.
+     *
+     * @return Reconstructed object.
+     * @throws ObjectStreamException Thrown in case of unmarshalling error.
+     */
+    protected Object readResolve() throws ObjectStreamException {
+        return set;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bc33643e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSetWriteReplacer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSetWriteReplacer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSetWriteReplacer.java
new file mode 100644
index 0000000..4350777
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSetWriteReplacer.java
@@ -0,0 +1,34 @@
+/*
+ * 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.binary;
+
+import org.jetbrains.annotations.Nullable;
+
+import java.util.TreeSet;
+
+/**
+ * Binary {@link TreeSet} write replacer.
+ */
+public class BinaryTreeSetWriteReplacer implements BinaryWriteReplacer {
+    /** {@inheritDoc} */
+    @Nullable @Override public Object replace(Object target) {
+        assert target instanceof TreeSet;
+
+        return new BinaryTreeSet((TreeSet)target);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/bc33643e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteReplacer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteReplacer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteReplacer.java
new file mode 100644
index 0000000..9376c37
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteReplacer.java
@@ -0,0 +1,33 @@
+/*
+ * 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.binary;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Interface to perform write replace.
+ */
+public interface BinaryWriteReplacer {
+    /**
+     * Perform replace.
+     *
+     * @param target Original object.
+     * @return Replaced object.
+     */
+    @Nullable public Object replace(Object target);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bc33643e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
index 30710f4..9450482 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
@@ -17,10 +17,18 @@
 
 package org.apache.ignite.internal.binary;
 
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
+import org.apache.ignite.internal.binary.streams.BinaryOutputStream;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.jetbrains.annotations.Nullable;
+
 import java.io.IOException;
 import java.io.ObjectOutput;
 import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Proxy;
 import java.math.BigDecimal;
 import java.math.BigInteger;
@@ -29,14 +37,6 @@ import java.util.Collection;
 import java.util.Date;
 import java.util.Map;
 import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryRawWriter;
-import org.apache.ignite.binary.BinaryWriter;
-import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
-import org.apache.ignite.internal.binary.streams.BinaryOutputStream;
-import org.apache.ignite.internal.util.typedef.internal.A;
-import org.jetbrains.annotations.Nullable;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
@@ -170,21 +170,8 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
             return;
         }
 
-        if (enableReplace && desc.getWriteReplaceMethod() != null) {
-            Object replacedObj;
-
-            try {
-                replacedObj = desc.getWriteReplaceMethod().invoke(obj);
-            }
-            catch (IllegalAccessException e) {
-                throw new RuntimeException(e);
-            }
-            catch (InvocationTargetException e) {
-                if (e.getTargetException() instanceof BinaryObjectException)
-                    throw (BinaryObjectException)e.getTargetException();
-
-                throw new BinaryObjectException("Failed to execute writeReplace() method on " + obj, e);
-            }
+        if (enableReplace && desc.isWriteReplace()) {
+            Object replacedObj = desc.writeReplace(obj);
 
             if (replacedObj == null) {
                 out.writeByte(GridBinaryMarshaller.NULL);


[3/3] ignite git commit: Finished.

Posted by vo...@apache.org.
Finished.


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

Branch: refs/heads/ignite-3681
Commit: 2376ac1d66d94c9a7781ed57bacfdb4cc809dfca
Parents: 71e9ec0
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Sat Aug 13 22:08:58 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Sat Aug 13 22:08:58 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinaryClassDescriptor.java |  2 +-
 .../apache/ignite/internal/binary/BinaryContext.java  | 14 +++++++++++++-
 .../apache/ignite/internal/binary/BinaryUtils.java    | 12 ++++++------
 3 files changed, 20 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2376ac1d/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
index c00a211..083057d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
@@ -147,7 +147,7 @@ public class BinaryClassDescriptor {
 
         initialSerializer = serializer;
 
-        // If serializer is not defined at this point, then we have to user OptimizedMarshaller.
+        // If serializer is not defined at this point, then we have to use OptimizedMarshaller.
         useOptMarshaller = serializer == null;
 
         // Reset reflective serializer so that we rely on existing reflection-based serialization.

http://git-wip-us.apache.org/repos/asf/ignite/blob/2376ac1d/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index a603894..94164aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -105,6 +105,8 @@ import java.util.LinkedHashSet;
 import java.util.LinkedList;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
 import java.util.jar.JarEntry;
@@ -182,6 +184,11 @@ public class BinaryContext {
         sysClss.add(GridClosureProcessor.C4V2.class.getName());
         sysClss.add(GridClosureProcessor.C4MLAV2.class.getName());
 
+        if (BinaryUtils.WRAP_TREES) {
+            sysClss.add(TreeMap.class.getName());
+            sysClss.add(TreeSet.class.getName());
+        }
+
         BINARYLIZABLE_SYS_CLSS = Collections.unmodifiableSet(sysClss);
     }
 
@@ -332,11 +339,16 @@ public class BinaryContext {
      * @param cls Class.
      * @return {@code True} if must be deserialized.
      */
+    @SuppressWarnings("SimplifiableIfStatement")
     public boolean mustDeserialize(Class cls) {
         BinaryClassDescriptor desc = descByCls.get(cls);
 
-        if (desc == null)
+        if (desc == null) {
+            if (BinaryUtils.WRAP_TREES && (cls == TreeMap.class || cls == TreeSet.class))
+                return false;
+
             return marshCtx.isSystemType(cls.getName()) || serializerForClass(cls) == null;
+        }
         else
             return desc.useOptimizedMarshaller();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/2376ac1d/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
index a963840..c2dd6d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
@@ -121,6 +121,10 @@ public class BinaryUtils {
     /** Field ID length. */
     public static final int FIELD_ID_LEN = 4;
 
+    /** Whether to skip TreeMap/TreeSet wrapping. */
+    public static final boolean WRAP_TREES =
+        !IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES);
+
     /** Field type names. */
     private static final String[] FIELD_TYPE_NAMES;
 
@@ -130,10 +134,6 @@ public class BinaryUtils {
     /** FNV1 hash prime. */
     private static final int FNV1_PRIME = 0x01000193;
 
-    /** Whether to skip TreeMap/TreeSet wrapping. */
-    private static final boolean WRAP_TREES =
-        !IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES);
-
     /**
      * Static class initializer.
      */
@@ -604,7 +604,7 @@ public class BinaryUtils {
 
         return cls == HashMap.class ||
             cls == LinkedHashMap.class ||
-            cls == TreeMap.class ||
+            (!WRAP_TREES && cls == TreeMap.class) ||
             cls == ConcurrentHashMap8.class ||
             cls == ConcurrentHashMap.class;
     }
@@ -623,7 +623,7 @@ public class BinaryUtils {
             return U.newHashMap(((Map)map).size());
         else if (cls == LinkedHashMap.class)
             return U.newLinkedHashMap(((Map)map).size());
-        else if (cls == TreeMap.class)
+        else if (!WRAP_TREES && cls == TreeMap.class)
             return new TreeMap<>(((TreeMap<Object, Object>)map).comparator());
         else if (cls == ConcurrentHashMap8.class)
             return new ConcurrentHashMap8<>(U.capacity(((Map)map).size()));


[2/3] ignite git commit: Wired up write-replace semantics.

Posted by vo...@apache.org.
Wired up write-replace semantics.


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

Branch: refs/heads/ignite-3681
Commit: 71e9ec04f05e38addc0a4512a7b286015015fc00
Parents: bc33643
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Sat Aug 13 21:38:21 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Sat Aug 13 21:38:21 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |  7 +++++++
 .../internal/binary/BinaryClassDescriptor.java  |  7 ++++++-
 .../ignite/internal/binary/BinaryUtils.java     | 22 ++++++++++++++++++++
 3 files changed, 35 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/71e9ec04/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 0c22c9d..3d455b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -452,6 +452,13 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_SERVICES_COMPATIBILITY_MODE = "IGNITE_SERVICES_COMPATIBILITY_MODE";
 
     /**
+     * When set to {@code true} tree-based data structures - {@code TreeMap} and {@code TreeSet} - will not be
+     * wrapped into special holders introduced to overcome serialization issue caused by missing {@code Comparable}
+     * interface on {@code BinaryObject}.
+     */
+    public static final String IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES = "IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES";
+
+    /**
      * Enforces singleton.
      */
     private IgniteSystemProperties() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/71e9ec04/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
index 2588f59..c00a211 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
@@ -317,6 +317,8 @@ public class BinaryClassDescriptor {
                 throw new BinaryObjectException("Invalid mode: " + mode);
         }
 
+        BinaryWriteReplacer writeReplacer0 = BinaryUtils.writeReplacer(cls);
+
         Method writeReplaceMthd;
 
         if (mode == BinaryWriteMode.BINARY || mode == BinaryWriteMode.OBJECT) {
@@ -329,7 +331,10 @@ public class BinaryClassDescriptor {
             writeReplaceMthd = null;
         }
 
-        writeReplacer = writeReplaceMthd == null ? null : new BinaryMethodWriteReplacer(writeReplaceMthd);
+        if (writeReplaceMthd != null && writeReplacer0 == null)
+            writeReplacer0 = new BinaryMethodWriteReplacer(writeReplaceMthd);
+
+        writeReplacer = writeReplacer0;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/71e9ec04/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
index 76e5b31..a963840 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
@@ -82,6 +82,9 @@ public class BinaryUtils {
     public static final boolean USE_STR_SERIALIZATION_VER_2 = IgniteSystemProperties.getBoolean(
         IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2, false);
 
+    /** Map from class to associated write replacer. */
+    public static final Map<Class, BinaryWriteReplacer> CLS_TO_WRITE_REPLACER = new HashMap<>();
+
     /** {@code true} if serialized value of this type cannot contain references to objects. */
     private static final boolean[] PLAIN_TYPE_FLAG = new boolean[102];
 
@@ -127,6 +130,10 @@ public class BinaryUtils {
     /** FNV1 hash prime. */
     private static final int FNV1_PRIME = 0x01000193;
 
+    /** Whether to skip TreeMap/TreeSet wrapping. */
+    private static final boolean WRAP_TREES =
+        !IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES);
+
     /**
      * Static class initializer.
      */
@@ -244,6 +251,11 @@ public class BinaryUtils {
         FIELD_TYPE_NAMES[GridBinaryMarshaller.TIMESTAMP_ARR] = "Timestamp[]";
         FIELD_TYPE_NAMES[GridBinaryMarshaller.OBJ_ARR] = "Object[]";
         FIELD_TYPE_NAMES[GridBinaryMarshaller.ENUM_ARR] = "Enum[]";
+
+        if (WRAP_TREES) {
+            CLS_TO_WRITE_REPLACER.put(TreeMap.class, new BinaryTreeMapWriteReplacer());
+            CLS_TO_WRITE_REPLACER.put(TreeSet.class, new BinaryTreeSetWriteReplacer());
+        }
     }
 
     /**
@@ -2214,6 +2226,16 @@ public class BinaryUtils {
     }
 
     /**
+     * Get predefined write-replacer associated with class.
+     *
+     * @param cls Class.
+     * @return Write replacer.
+     */
+    public static BinaryWriteReplacer writeReplacer(Class cls) {
+        return cls != null ? CLS_TO_WRITE_REPLACER.get(cls) : null;
+    }
+
+    /**
      * Enum type.
      */
     private static class EnumType {