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/12/23 15:17:10 UTC

[12/12] ignite git commit: IGNITE-2206: Fixed a bug with file system factory deserialization.

IGNITE-2206: Fixed a bug with file system factory deserialization.


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

Branch: refs/heads/ignite-2206
Commit: 04827ca7f2f96d50e890b98603f1a9f41e1c853a
Parents: 061c222
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Dec 23 17:17:40 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Dec 23 17:17:40 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsPaths.java     | 55 +++++++++++++-------
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    | 35 ++++++++-----
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    | 41 +++++++++------
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |  4 --
 4 files changed, 81 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/04827ca7/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java
index cd34655..0bb581b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java
@@ -27,6 +27,8 @@ import java.io.ObjectOutput;
 import java.io.ObjectOutputStream;
 import java.util.ArrayList;
 import java.util.List;
+
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.igfs.IgfsMode;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.util.typedef.T2;
@@ -41,7 +43,7 @@ public class IgfsPaths implements Externalizable {
     private static final long serialVersionUID = 0L;
 
     /** */
-    private Object payload;
+    private byte[] payloadBytes;
 
     /** Default IGFS mode. */
     private IgfsMode dfltMode;
@@ -62,11 +64,27 @@ public class IgfsPaths implements Externalizable {
      * @param payload Payload.
      * @param dfltMode Default IGFS mode.
      * @param pathModes Path modes.
+     * @throws IgniteCheckedException If failed.
      */
-    public IgfsPaths(Object payload, IgfsMode dfltMode, @Nullable List<T2<IgfsPath, IgfsMode>> pathModes) {
-        this.payload = payload;
+    public IgfsPaths(Object payload, IgfsMode dfltMode, @Nullable List<T2<IgfsPath, IgfsMode>> pathModes)
+        throws IgniteCheckedException {
         this.dfltMode = dfltMode;
         this.pathModes = pathModes;
+
+        if (payload == null)
+            payloadBytes = null;
+        else {
+            ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+            try (ObjectOutput oo = new ObjectOutputStream(baos)) {
+                oo.writeObject(payload);
+            }
+            catch (IOException e) {
+                throw new IgniteCheckedException("Failed to serialize secondary file system factory: " + payload, e);
+            }
+
+            payloadBytes = baos.toByteArray();
+        }
     }
 
     /**
@@ -85,20 +103,25 @@ public class IgfsPaths implements Externalizable {
 
     /**
      * @return Payload.
+     *
+     * @throws IgniteCheckedException If failed to deserialize the payload.
      */
-    @Nullable public Object getPayload() {
-        return payload;
+    @Nullable public Object getPayload() throws IgniteCheckedException {
+        if (payloadBytes == null)
+            return null;
+        else {
+            try (ObjectInput oi = new ObjectInputStream(new ByteArrayInputStream(payloadBytes))) {
+                return oi.readObject();
+            }
+            catch (IOException | ClassNotFoundException e) {
+                throw new IgniteCheckedException("Failed to deserialize secondary file system factory. ", e);
+            }
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-        try (ObjectOutput oo = new ObjectOutputStream(baos)) {
-            oo.writeObject(payload);
-        }
-
-        U.writeByteArray(out, baos.toByteArray());
+        U.writeByteArray(out, payloadBytes);
 
         U.writeEnum(out, dfltMode);
 
@@ -120,13 +143,7 @@ public class IgfsPaths implements Externalizable {
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        byte[] factoryBytes = U.readByteArray(in);
-
-        assert factoryBytes != null;
-
-        try (ObjectInput oi = new ObjectInputStream(new ByteArrayInputStream(factoryBytes))) {
-            payload = oi.readObject();
-        }
+        payloadBytes = U.readByteArray(in);
 
         dfltMode = IgfsMode.fromOrdinal(in.readByte());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/04827ca7/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
index 545c905..20cd9ca 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
@@ -17,19 +17,6 @@
 
 package org.apache.ignite.hadoop.fs.v1;
 
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
@@ -43,6 +30,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progressable;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory;
 import org.apache.ignite.igfs.IgfsBlockLocation;
@@ -71,6 +59,20 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.jetbrains.annotations.Nullable;
 
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
 import static org.apache.ignite.configuration.FileSystemConfiguration.DFLT_IGFS_LOG_BATCH_SIZE;
 import static org.apache.ignite.configuration.FileSystemConfiguration.DFLT_IGFS_LOG_DIR;
 import static org.apache.ignite.igfs.IgfsMode.PROXY;
@@ -329,7 +331,12 @@ public class IgniteHadoopFileSystem extends FileSystem {
             }
 
             if (initSecondary) {
-                factory = (HadoopFileSystemFactory)paths.getPayload();
+                try {
+                    factory = (HadoopFileSystemFactory) paths.getPayload();
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IOException("Failed to get secondary file system factory.", e);
+                }
 
                 A.ensure(factory != null, "Secondary file system factory should not be null.");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/04827ca7/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
index d665d4c..59d870c 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
@@ -17,22 +17,6 @@
 
 package org.apache.ignite.hadoop.fs.v2;
 
-import java.io.BufferedOutputStream;
-import java.io.Closeable;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -51,6 +35,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.igfs.IgfsBlockLocation;
 import org.apache.ignite.igfs.IgfsFile;
@@ -76,6 +61,23 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.jetbrains.annotations.Nullable;
 
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
 import static org.apache.ignite.configuration.FileSystemConfiguration.DFLT_IGFS_LOG_BATCH_SIZE;
 import static org.apache.ignite.configuration.FileSystemConfiguration.DFLT_IGFS_LOG_DIR;
 import static org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem.getFsHadoopUser;
@@ -336,7 +338,12 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
             }
 
             if (initSecondary) {
-                factory = (HadoopAbstractFileSystemFactory)paths.getPayload();
+                try {
+                    factory = (HadoopAbstractFileSystemFactory) paths.getPayload();
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IOException("Failed to get secondary file system factory.", e);
+                }
 
                 A.ensure(secondaryUri != null, "File system factory uri should not be null.");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/04827ca7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
index 2b20639..5808d01 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
@@ -112,10 +112,6 @@ public class HadoopLazyConcurrentMap<K, V extends Closeable> {
             if (closed)
                 return;
 
-            // TODO: debug:
-            System.out.println("##### closed: " + System.identityHashCode(this));
-            Thread.dumpStack();
-
             closed = true;
 
             Exception err = null;