You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by nt...@apache.org on 2016/03/21 13:54:04 UTC

[10/15] ignite git commit: IGNITE-2861: IGFS: Moved metadata processors into separate top-level classes to simplify code. Also cleaned up IgfsMetaManager from unused code.

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java
new file mode 100644
index 0000000..43a5d7d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java
@@ -0,0 +1,63 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * File lock entry processor.
+ */
+public class IgfsMetaFileLockProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Lock Id. */
+    private IgniteUuid lockId;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsMetaFileLockProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param lockId Lock ID.
+     */
+    public IgfsMetaFileLockProcessor(IgniteUuid lockId) {
+        this.lockId = lockId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo oldInfo = entry.getValue();
+
+        IgfsEntryInfo newInfo = oldInfo.lock(lockId);
+
+        entry.setValue(newInfo);
+
+        return newInfo;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeGridUuid(out, lockId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        lockId = U.readGridUuid(in);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java
new file mode 100644
index 0000000..5af09e5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java
@@ -0,0 +1,74 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsFileAffinityRange;
+import org.apache.ignite.internal.processors.igfs.IgfsFileMap;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * Delete range processor.
+ */
+public class IgfsMetaFileRangeDeleteProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Range. */
+    private IgfsFileAffinityRange range;
+
+    /**
+     * Constructor.
+     */
+    public IgfsMetaFileRangeDeleteProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param range Range.
+     */
+    public IgfsMetaFileRangeDeleteProcessor(IgfsFileAffinityRange range) {
+        this.range = range;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo oldInfo = entry.getValue();
+
+        IgfsFileMap newMap = new IgfsFileMap(oldInfo.fileMap());
+
+        newMap.deleteRange(range);
+
+        IgfsEntryInfo newInfo = oldInfo.fileMap(newMap);
+
+        entry.setValue(newInfo);
+
+        return newInfo;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(range);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        range = (IgfsFileAffinityRange)in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaFileRangeDeleteProcessor.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java
new file mode 100644
index 0000000..ae886c8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java
@@ -0,0 +1,81 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsFileAffinityRange;
+import org.apache.ignite.internal.processors.igfs.IgfsFileMap;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * Update range processor.
+ */
+public class IgfsMetaFileRangeUpdateProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Range. */
+    private IgfsFileAffinityRange range;
+
+    /** Status. */
+    private int status;
+
+    /**
+     * Constructor.
+     */
+    public IgfsMetaFileRangeUpdateProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param range Range.
+     * @param status Status.
+     */
+    public IgfsMetaFileRangeUpdateProcessor(IgfsFileAffinityRange range, int status) {
+        this.range = range;
+        this.status = status;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo oldInfo = entry.getValue();
+
+        IgfsFileMap newMap = new IgfsFileMap(oldInfo.fileMap());
+
+        newMap.updateRangeStatus(range, status);
+
+        IgfsEntryInfo newInfo = oldInfo.fileMap(newMap);
+
+        entry.setValue(newInfo);
+
+        return newInfo;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(range);
+        out.writeInt(status);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        range = (IgfsFileAffinityRange)in.readObject();
+        status = in.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaFileRangeUpdateProcessor.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java
new file mode 100644
index 0000000..30f8e22
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java
@@ -0,0 +1,75 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsFileAffinityRange;
+import org.apache.ignite.internal.processors.igfs.IgfsFileMap;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * File reserve space entry processor.
+ */
+public class IgfsMetaFileReserveSpaceProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Space. */
+    private long space;
+
+    /** Affinity range. */
+    private IgfsFileAffinityRange affRange;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsMetaFileReserveSpaceProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param space Space.
+     * @param affRange
+     */
+    public IgfsMetaFileReserveSpaceProcessor(long space, IgfsFileAffinityRange affRange) {
+        this.space = space;
+        this.affRange = affRange;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo oldInfo = entry.getValue();
+
+        IgfsFileMap newMap = new IgfsFileMap(oldInfo.fileMap());
+
+        newMap.addRange(affRange);
+
+        IgfsEntryInfo newInfo = oldInfo.length(oldInfo.length() + space).fileMap(newMap);
+
+        entry.setValue(newInfo);
+
+        return newInfo;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeLong(space);
+        out.writeObject(affRange);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        space = in.readLong();
+        affRange = (IgfsFileAffinityRange)in.readObject();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileUnlockProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileUnlockProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileUnlockProcessor.java
new file mode 100644
index 0000000..d535a1d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileUnlockProcessor.java
@@ -0,0 +1,60 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * File unlock entry processor.
+ */
+public class IgfsMetaFileUnlockProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Modification time. */
+    private long modificationTime;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsMetaFileUnlockProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param modificationTime Modification time.
+     */
+    public IgfsMetaFileUnlockProcessor(long modificationTime) {
+        this.modificationTime = modificationTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo old = entry.getValue();
+
+        entry.setValue(old.unlock(modificationTime));
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeLong(modificationTime);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        modificationTime = in.readLong();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java
new file mode 100644
index 0000000..32e5512
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java
@@ -0,0 +1,66 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsMetaManager;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * Update path closure.
+ */
+public final class IgfsMetaUpdatePathProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** New path. */
+    private IgfsPath path;
+
+    /**
+     * @param path Path.
+     */
+    public IgfsMetaUpdatePathProcessor(IgfsPath path) {
+        this.path = path;
+    }
+
+    /**
+     * Default constructor (required by Externalizable).
+     */
+    public IgfsMetaUpdatePathProcessor() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> e, Object... args) {
+        IgfsEntryInfo info = e.getValue();
+
+        IgfsEntryInfo newInfo = info.path(path);
+
+        e.setValue(newInfo);
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        path = (IgfsPath)in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaUpdatePathProcessor.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java
new file mode 100644
index 0000000..a97c186
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java
@@ -0,0 +1,78 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.util.GridLeanMap;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Map;
+
+/**
+ * Update properties processor.
+ */
+public class IgfsMetaUpdatePropertiesProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Properties to be updated. */
+    private Map<String, String> props;
+
+    /**
+     * Constructor.
+     */
+    public IgfsMetaUpdatePropertiesProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param props Properties.
+     */
+    public IgfsMetaUpdatePropertiesProcessor(Map<String, String> props) {
+        this.props = props;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo oldInfo = entry.getValue();
+
+        Map<String, String> tmp = oldInfo.properties();
+
+        tmp = tmp == null ? new GridLeanMap<String, String>(props.size()) : new GridLeanMap<>(tmp);
+
+        for (Map.Entry<String, String> e : props.entrySet()) {
+            if (e.getValue() == null)
+                // Remove properties with 'null' values.
+                tmp.remove(e.getKey());
+            else
+                // Add/overwrite property.
+                tmp.put(e.getKey(), e.getValue());
+        }
+
+        IgfsEntryInfo newInfo = oldInfo.properties(tmp);
+
+        entry.setValue(newInfo);
+
+        return newInfo;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeStringMap(out, props);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        props = U.readStringMap(in);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java
new file mode 100644
index 0000000..8acd289
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java
@@ -0,0 +1,68 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * Update times entry processor.
+ */
+public class IgfsMetaUpdateTimesProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Access time. */
+    private long accessTime;
+
+    /** Modification time. */
+    private long modificationTime;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsMetaUpdateTimesProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param accessTime Access time.
+     * @param modificationTime Modification time.
+     */
+    public IgfsMetaUpdateTimesProcessor(long accessTime, long modificationTime) {
+        this.accessTime = accessTime;
+        this.modificationTime = modificationTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+
+        IgfsEntryInfo oldInfo = entry.getValue();
+
+        entry.setValue(oldInfo.accessModificationTime(accessTime, modificationTime));
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeLong(accessTime);
+        out.writeLong(modificationTime);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        accessTime = in.readLong();
+        modificationTime = in.readLong();
+    }
+}