From 5fdd65f5f4f5df1d28b0fb4f7efed226d5db1b3c Mon Sep 17 00:00:00 2001 From: M S Vishwanath Bhat Date: Fri, 24 Feb 2012 13:18:56 +0530 Subject: renaming hdfs -> glusterfs-hadoop Change-Id: Ibb937af1231f6bbed9a2d4eaeabc6e9d4000887f BUG: 797064 Signed-off-by: M S Vishwanath Bhat Reviewed-on: http://review.gluster.com/2811 Tested-by: Gluster Build System Reviewed-by: Vijay Bellur --- glusterfs-hadoop/0.20.2/conf/core-site.xml | 38 ++ glusterfs-hadoop/0.20.2/pom.xml | 36 ++ .../hadoop/fs/glusterfs/GlusterFSBrickClass.java | 109 +++++ .../hadoop/fs/glusterfs/GlusterFSBrickRepl.java | 52 +++ .../apache/hadoop/fs/glusterfs/GlusterFSXattr.java | 471 ++++++++++++++++++++ .../fs/glusterfs/GlusterFUSEInputStream.java | 205 +++++++++ .../fs/glusterfs/GlusterFUSEOutputStream.java | 86 ++++ .../hadoop/fs/glusterfs/GlusterFileSystem.java | 492 +++++++++++++++++++++ .../org/apache/hadoop/fs/glusterfs/AppTest.java | 38 ++ glusterfs-hadoop/0.20.2/tools/build-deploy-jar.py | 212 +++++++++ glusterfs-hadoop/COPYING | 202 +++++++++ glusterfs-hadoop/README | 182 ++++++++ hdfs/0.20.2/conf/core-site.xml | 38 -- hdfs/0.20.2/pom.xml | 36 -- .../hadoop/fs/glusterfs/GlusterFSBrickClass.java | 109 ----- .../hadoop/fs/glusterfs/GlusterFSBrickRepl.java | 52 --- .../apache/hadoop/fs/glusterfs/GlusterFSXattr.java | 471 -------------------- .../fs/glusterfs/GlusterFUSEInputStream.java | 205 --------- .../fs/glusterfs/GlusterFUSEOutputStream.java | 86 ---- .../hadoop/fs/glusterfs/GlusterFileSystem.java | 492 --------------------- .../org/apache/hadoop/fs/glusterfs/AppTest.java | 38 -- hdfs/0.20.2/tools/build-deploy-jar.py | 212 --------- hdfs/COPYING | 202 --------- hdfs/README | 182 -------- 24 files changed, 2123 insertions(+), 2123 deletions(-) create mode 100644 glusterfs-hadoop/0.20.2/conf/core-site.xml create mode 100644 glusterfs-hadoop/0.20.2/pom.xml create mode 100644 glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickClass.java create mode 100644 glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickRepl.java create mode 100644 glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSXattr.java create mode 100644 glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEInputStream.java create mode 100644 glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEOutputStream.java create mode 100644 glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFileSystem.java create mode 100644 glusterfs-hadoop/0.20.2/src/test/java/org/apache/hadoop/fs/glusterfs/AppTest.java create mode 100644 glusterfs-hadoop/0.20.2/tools/build-deploy-jar.py create mode 100644 glusterfs-hadoop/COPYING create mode 100644 glusterfs-hadoop/README delete mode 100644 hdfs/0.20.2/conf/core-site.xml delete mode 100644 hdfs/0.20.2/pom.xml delete mode 100644 hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickClass.java delete mode 100644 hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickRepl.java delete mode 100644 hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSXattr.java delete mode 100644 hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEInputStream.java delete mode 100644 hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEOutputStream.java delete mode 100644 hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFileSystem.java delete mode 100644 hdfs/0.20.2/src/test/java/org/apache/hadoop/fs/glusterfs/AppTest.java delete mode 100644 hdfs/0.20.2/tools/build-deploy-jar.py delete mode 100644 hdfs/COPYING delete mode 100644 hdfs/README diff --git a/glusterfs-hadoop/0.20.2/conf/core-site.xml b/glusterfs-hadoop/0.20.2/conf/core-site.xml new file mode 100644 index 00000000..d7f75fca --- /dev/null +++ b/glusterfs-hadoop/0.20.2/conf/core-site.xml @@ -0,0 +1,38 @@ + + + + + + + + + fs.glusterfs.impl + org.apache.hadoop.fs.glusterfs.GlusterFileSystem + + + + fs.default.name + glusterfs://192.168.1.36:9000 + + + + fs.glusterfs.volname + volume-dist-rep + + + + fs.glusterfs.mount + /mnt/glusterfs + + + + fs.glusterfs.server + 192.168.1.36 + + + + quick.slave.io + Off + + + diff --git a/glusterfs-hadoop/0.20.2/pom.xml b/glusterfs-hadoop/0.20.2/pom.xml new file mode 100644 index 00000000..fe661d40 --- /dev/null +++ b/glusterfs-hadoop/0.20.2/pom.xml @@ -0,0 +1,36 @@ + + 4.0.0 + org.apache.hadoop.fs.glusterfs + glusterfs + jar + 0.20.2-0.1 + glusterfs + http://maven.apache.org + + + junit + junit + 3.8.1 + test + + + org.apache.hadoop + hadoop-core + 0.20.2 + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 2.3.2 + + 1.5 + 1.5 + + + + + diff --git a/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickClass.java b/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickClass.java new file mode 100644 index 00000000..e633b8aa --- /dev/null +++ b/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickClass.java @@ -0,0 +1,109 @@ +/** + * + * Copyright (c) 2011 Gluster, Inc. + * This file is part of GlusterFS. + * + * Licensed 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.hadoop.fs.glusterfs; + +import java.io.*; + +public class GlusterFSBrickClass { + String host; + String exportedFile; + long start; + long end; + boolean isChunked; + int stripeSize; // Stripe size in bytes + int nrStripes; // number of stripes + int switchCount; // for SR, DSR - number of replicas of each stripe + // -1 for others + + public GlusterFSBrickClass (String brick, long start, long len, boolean flag, + int stripeSize, int nrStripes, int switchCount) + throws IOException { + this.host = brick2host(brick); + this.exportedFile = brick2file(brick); + this.start = start; + this.end = start + len; + this.isChunked = flag; + this.stripeSize = stripeSize; + this.nrStripes = nrStripes; + this.switchCount = switchCount; + } + + public boolean isChunked () { + return isChunked; + } + + public String brickIsLocal(String hostname) { + String path = null; + File f = null; + if (host.equals(hostname)) + path = exportedFile; + + return path; + } + + public int[] getBrickNumberInTree(long start, int len) { + long end = len; + int startNodeInTree = ((int) (start / stripeSize)) % nrStripes; + int endNodeInTree = ((int) ((start + len) / stripeSize)) % nrStripes; + + if (startNodeInTree != endNodeInTree) { + end = (start - (start % stripeSize)) + stripeSize; + end -= start; + } + + return new int[] {startNodeInTree, endNodeInTree, (int) end}; + } + + public boolean brickHasFilePart(int nodeInTree, int nodeLoc) { + if (switchCount == -1) + return (nodeInTree == nodeLoc); + + nodeInTree *= switchCount; + for (int i = nodeInTree; i < (nodeInTree + switchCount); i++) { + if (i == nodeLoc) + return true; + } + + return false; + } + + public String brick2host (String brick) + throws IOException { + String[] hf = null; + + hf = brick.split(":"); + if (hf.length != 2) + throw new IOException("Error getting hostname from brick"); + + return hf[0]; + } + + public String brick2file (String brick) + throws IOException { + String[] hf = null; + + hf = brick.split(":"); + if (hf.length != 2) + throw new IOException("Error getting hostname from brick"); + + return hf[1]; + } + +} diff --git a/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickRepl.java b/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickRepl.java new file mode 100644 index 00000000..11454e63 --- /dev/null +++ b/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickRepl.java @@ -0,0 +1,52 @@ +/** + * + * Copyright (c) 2011 Gluster, Inc. + * This file is part of GlusterFS. + * + * Licensed 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.hadoop.fs.glusterfs; + +import java.io.*; + +public class GlusterFSBrickRepl { + private String[] replHost; + private long start; + private long len; + private int cnt; + + GlusterFSBrickRepl(int replCount, long start, long len) { + this.replHost = new String[replCount]; + this.start = start; + this.len = len; + this.cnt = 0; + } + + public void addHost (String host) { + this.replHost[cnt++] = host; + } + + public String[] getReplHosts () { + return this.replHost; + } + + public long getStartLen () { + return this.start; + } + + public long getOffLen () { + return this.len; + } +} diff --git a/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSXattr.java b/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSXattr.java new file mode 100644 index 00000000..18e9003b --- /dev/null +++ b/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSXattr.java @@ -0,0 +1,471 @@ +/** + * + * Copyright (c) 2011 Gluster, Inc. + * This file is part of GlusterFS. + * + * Licensed 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.hadoop.fs.glusterfs; + +import java.net.*; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.io.*; +import java.util.HashMap; +import java.util.TreeMap; +import java.util.ArrayList; +import java.util.Iterator; + +import org.apache.hadoop.fs.BlockLocation; + +public class GlusterFSXattr { + + public enum LAYOUT { D, S, R, DS, DR, SR, DSR } + public enum CMD { GET_HINTS, GET_REPLICATION, GET_BLOCK_SIZE, CHECK_FOR_QUICK_IO } + + private static String hostname; + + public GlusterFSXattr() { } + + public static String brick2host (String brick) + throws IOException { + String[] hf = null; + + hf = brick.split(":"); + if (hf.length != 2) { + System.out.println("brick not of format hostname:path"); + throw new IOException("Error getting hostname from brick"); + } + + return hf[0]; + } + + public static String brick2file (String brick) + throws IOException { + String[] hf = null; + + hf = brick.split(":"); + if (hf.length != 2) { + System.out.println("brick not of format hostname:path"); + throw new IOException("Error getting hostname from brick"); + } + + return hf[1]; + } + + public static BlockLocation[] getPathInfo (String filename, long start, long len) + throws IOException { + HashMap> vol = null; + HashMap meta = new HashMap(); + + vol = execGetFattr(filename, meta, CMD.GET_HINTS); + + return getHints(vol, meta, start, len, null); + } + + public static long getBlockSize (String filename) + throws IOException { + HashMap> vol = null; + HashMap meta = new HashMap(); + + vol = execGetFattr(filename, meta, CMD.GET_BLOCK_SIZE); + + if (!meta.containsKey("block-size")) + return 0; + + return (long) meta.get("block-size"); + + } + + public static short getReplication (String filename) + throws IOException { + HashMap> vol = null; + HashMap meta = new HashMap(); + + vol = execGetFattr(filename, meta, CMD.GET_REPLICATION); + + return (short) getReplicationFromLayout(vol, meta); + + } + + public static TreeMap quickIOPossible (String filename, long start, + long len) + throws IOException { + String realpath = null; + HashMap> vol = null; + HashMap meta = new HashMap(); + TreeMap hnts = new TreeMap(); + + vol = execGetFattr(filename, meta, CMD.GET_HINTS); + getHints(vol, meta, start, len, hnts); + + if (hnts.size() == 0) + return null; // BOOM !! + + // DEBUG - dump hnts here + return hnts; + } + + public static HashMap> execGetFattr (String filename, + HashMap meta, + CMD cmd) + throws IOException { + Process p = null; + BufferedReader brInput = null; + String s = null; + String cmdOut = null; + String getfattrCmd = null; + String xlator = null; + String enclosingXl = null; + String enclosingXlVol = null; + String key = null; + String layout = ""; + int rcount = 0; + int scount = 0; + int dcount = 0; + int count = 0; + + HashMap> vol = new HashMap>(); + + getfattrCmd = "getfattr -m . -n trusted.glusterfs.pathinfo " + filename; + + p = Runtime.getRuntime().exec(getfattrCmd); + brInput = new BufferedReader(new InputStreamReader(p.getInputStream())); + + cmdOut = ""; + while ( (s = brInput.readLine()) != null ) + cmdOut += s; + + /** + * TODO: Use a single regex for extracting posix paths as well + * as xlator counts for layout matching. + */ + + Pattern pattern = Pattern.compile("<(.*?)[:\\(](.*?)>"); + Matcher matcher = pattern.matcher(cmdOut); + + Pattern p_px = Pattern.compile(".*?:(.*)"); + Matcher m_px; + String gibberish_path; + + s = null; + while (matcher.find()) { + xlator = matcher.group(1); + if (xlator.equalsIgnoreCase("posix")) { + if (enclosingXl.equalsIgnoreCase("replicate")) + count = rcount; + else if (enclosingXl.equalsIgnoreCase("stripe")) + count = scount; + else if (enclosingXl.equalsIgnoreCase("distribute")) + count = dcount; + else + throw new IOException("Unknown Translator: " + enclosingXl); + + key = enclosingXl + "-" + count; + + if (vol.get(key) == null) + vol.put(key, new ArrayList()); + + gibberish_path = matcher.group(2); + + /* extract posix path from the gibberish string */ + m_px = p_px.matcher(gibberish_path); + if (!m_px.find()) + throw new IOException("Cannot extract posix path"); + + vol.get(key).add(m_px.group(1)); + continue; + } + + enclosingXl = xlator; + enclosingXlVol = matcher.group(2); + + if (xlator.equalsIgnoreCase("replicate")) + if (rcount++ != 0) + continue; + + if (xlator.equalsIgnoreCase("stripe")) { + if (scount++ != 0) + continue; + + + Pattern ps = Pattern.compile("\\[(\\d+)\\]"); + Matcher ms = ps.matcher(enclosingXlVol); + + if (ms.find()) { + if (((cmd == CMD.GET_BLOCK_SIZE) || (cmd == CMD.GET_HINTS)) + && (meta != null)) + meta.put("block-size", Integer.parseInt(ms.group(1))); + } else + throw new IOException("Cannot get stripe size"); + } + + if (xlator.equalsIgnoreCase("distribute")) + if (dcount++ != 0) + continue; + + layout += xlator.substring(0, 1); + } + + if ((dcount == 0) && (scount == 0) && (rcount == 0)) + throw new IOException("Cannot get layout"); + + if (meta != null) { + meta.put("dcount", dcount); + meta.put("scount", scount); + meta.put("rcount", rcount); + } + + vol.put("layout", new ArrayList(1)); + vol.get("layout").add(layout); + + return vol; + } + + static BlockLocation[] getHints (HashMap> vol, + HashMap meta, + long start, long len, + TreeMap hnts) + throws IOException { + String brick = null; + String key = null; + boolean done = false; + int i = 0; + int counter = 0; + int stripeSize = 0; + long stripeStart = 0; + long stripeEnd = 0; + int nrAllocs = 0; + int allocCtr = 0; + BlockLocation[] result = null; + ArrayList brickList = null; + ArrayList stripedBricks = null; + Iterator it = null; + + String[] blks = null; + GlusterFSBrickRepl[] repl = null; + int dcount, scount, rcount; + + LAYOUT l = LAYOUT.valueOf(vol.get("layout").get(0)); + dcount = meta.get("dcount"); + scount = meta.get("scount"); + rcount = meta.get("rcount"); + + switch (l) { + case D: + key = "DISTRIBUTE-" + dcount; + brick = vol.get(key).get(0); + + if (hnts == null) { + result = new BlockLocation[1]; + result[0] = new BlockLocation(null, new String[] {brick2host(brick)}, start, len); + } else + hnts.put(0, new GlusterFSBrickClass(brick, start, len, false, -1, -1, -1)); + break; + + case R: + case DR: + /* just the name says it's striped - the volume isn't */ + stripedBricks = new ArrayList(); + + for (i = 1; i <= rcount; i++) { + key = "REPLICATE-" + i; + brickList = vol.get(key); + it = brickList.iterator(); + while (it.hasNext()) { + stripedBricks.add(it.next()); + } + } + + nrAllocs = stripedBricks.size(); + if (hnts == null) { + result = new BlockLocation[1]; + blks = new String[nrAllocs]; + } + + for (i = 0; i < nrAllocs; i++) { + if (hnts == null) + blks[i] = brick2host(stripedBricks.get(i)); + else + hnts.put(i, new GlusterFSBrickClass(stripedBricks.get(i), start, len, false, -1, -1, -1)); + } + + if (hnts == null) + result[0] = new BlockLocation(null, blks, start, len); + + break; + + case SR: + case DSR: + int rsize = 0; + ArrayList> replicas = new ArrayList>(); + + stripedBricks = new ArrayList(); + + if (rcount == 0) + throw new IOException("got replicated volume with replication count 0"); + + for (i = 1; i <= rcount; i++) { + key = "REPLICATE-" + i; + brickList = vol.get(key); + it = brickList.iterator(); + replicas.add(i - 1, new ArrayList()); + while (it.hasNext()) { + replicas.get(i - 1).add(it.next()); + } + } + + rsize = replicas.get(0).size(); + stripeSize = meta.get("block-size"); + + nrAllocs = (int) (((len - start) / stripeSize) + 1); + if (hnts == null) { + result = new BlockLocation[nrAllocs]; + repl = new GlusterFSBrickRepl[nrAllocs]; + } + + // starting stripe position + counter = (int) ((start / stripeSize) % rcount); + stripeStart = start; + + key = null; + int currAlloc = 0; + boolean hntsDone = false; + while ((stripeStart < len) && !done) { + stripeEnd = (stripeStart - (stripeStart % stripeSize)) + stripeSize - 1; + if (stripeEnd > start + len) { + stripeEnd = start + len - 1; + done = true; + } + + if (hnts == null) + repl[allocCtr] = new GlusterFSBrickRepl(rsize, stripeStart, (stripeEnd - stripeStart)); + + for (i = 0; i < rsize; i++) { + brick = replicas.get(counter).get(i); + currAlloc = (allocCtr * rsize) + i; + + if (hnts == null) + repl[allocCtr].addHost(brick2host(brick)); + else + if (currAlloc <= (rsize * rcount) - 1) { + hnts.put(currAlloc, new GlusterFSBrickClass(brick, stripeStart, + (stripeEnd - stripeStart), + true, stripeSize, rcount, rsize)); + } else + hntsDone = true; + } + + if (hntsDone) + break; + + stripeStart = stripeEnd + 1; + + allocCtr++; + counter++; + + if (counter >= replicas.size()) + counter = 0; + } + + if (hnts == null) + for (int k = 0; k < nrAllocs; k++) + result[k] = new BlockLocation(null, repl[k].getReplHosts(), repl[k].getStartLen(), repl[k].getOffLen()); + + break; + + case S: + case DS: + if (scount == 0) + throw new IOException("got striped volume with stripe count 0"); + + stripedBricks = new ArrayList(); + stripeSize = meta.get("block-size"); + + key = "STRIPE-" + scount; + brickList = vol.get(key); + it = brickList.iterator(); + while (it.hasNext()) { + stripedBricks.add(it.next()); + } + + nrAllocs = (int) ((len - start) / stripeSize) + 1; + if (hnts == null) + result = new BlockLocation[nrAllocs]; + + // starting stripe position + counter = (int) ((start / stripeSize) % stripedBricks.size()); + stripeStart = start; + + key = null; + while ((stripeStart < len) && !done) { + brick = stripedBricks.get(counter); + + stripeEnd = (stripeStart - (stripeStart % stripeSize)) + stripeSize - 1; + if (stripeEnd > start + len) { + stripeEnd = start + len - 1; + done = true; + } + + if (hnts == null) + result[allocCtr] = new BlockLocation(null, new String[] {brick2host(brick)}, + stripeStart, (stripeEnd - stripeStart)); + else + if (allocCtr <= stripedBricks.size()) { + hnts.put(allocCtr, new GlusterFSBrickClass(brick, stripeStart, (stripeEnd - stripeStart), + true, stripeSize, stripedBricks.size(), -1)); + } else + break; + + stripeStart = stripeEnd + 1; + + counter++; + allocCtr++; + + if (counter >= stripedBricks.size()) + counter = 0; + } + + break; + } + + return result; + } + + /* TODO: use meta{dcount,scount,rcount} for checking */ + public static int getReplicationFromLayout (HashMap> vol, + HashMap meta) + throws IOException { + int replication = 0; + LAYOUT l = LAYOUT.valueOf(vol.get("layout").get(0)); + + switch (l) { + case D: + case S: + case DS: + replication = 1; + break; + + case R: + case DR: + case SR: + case DSR: + final String key = "REPLICATION-1"; + replication = vol.get(key).size(); + } + + return replication; + } +} diff --git a/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEInputStream.java b/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEInputStream.java new file mode 100644 index 00000000..e92237ae --- /dev/null +++ b/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEInputStream.java @@ -0,0 +1,205 @@ +/** + * + * Copyright (c) 2011 Gluster, Inc. + * This file is part of GlusterFS. + * + * Licensed 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.hadoop.fs.glusterfs; + +import java.io.*; +import java.util.TreeMap; + +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.FileSystem; + + +public class GlusterFUSEInputStream extends FSInputStream { + File f; + boolean lastActive; + long pos; + boolean closed; + String thisHost; + RandomAccessFile fuseInputStream; + RandomAccessFile fsInputStream; + GlusterFSBrickClass thisBrick; + int nodeLocation; + TreeMap hnts; + + public GlusterFUSEInputStream (File f, TreeMap hnts, + String hostname) throws IOException { + this.f = f; + this.pos = 0; + this.closed = false; + this.hnts = hnts; + this.thisHost = hostname; + this.fsInputStream = null; + this.fuseInputStream = new RandomAccessFile(f.getPath(), "r"); + + this.lastActive = true; // true == FUSE, false == backed file + + String directFilePath = null; + if (this.hnts != null) { + directFilePath = findLocalFile(f.getPath(), this.hnts); + if (directFilePath != null) { + this.fsInputStream = new RandomAccessFile(directFilePath, "r"); + this.lastActive = !this.lastActive; + } + } + } + + public String findLocalFile (String path, TreeMap hnts) { + int i = 0; + String actFilePath = null; + GlusterFSBrickClass gfsBrick = null; + + gfsBrick = hnts.get(0); + + /* do a linear search for the matching host not worrying + about file stripes */ + for (i = 0; i < hnts.size(); i++) { + gfsBrick = hnts.get(i); + actFilePath = gfsBrick.brickIsLocal(this.thisHost); + if (actFilePath != null) { + this.thisBrick = gfsBrick; + this.nodeLocation = i; + break; + } + } + + return actFilePath; + } + + public long getPos () throws IOException { + return pos; + } + + public synchronized int available () throws IOException { + return (int) ((f.length()) - getPos()); + } + + public void seek (long pos) throws IOException { + fuseInputStream.seek(pos); + if (fsInputStream != null) + fsInputStream.seek(pos); + } + + public boolean seekToNewSource (long pos) throws IOException { + return false; + } + + public RandomAccessFile chooseStream (long start, int[] nlen) + throws IOException { + GlusterFSBrickClass gfsBrick = null; + RandomAccessFile in = fuseInputStream; + boolean oldActiveStream = lastActive; + lastActive = true; + + if ((hnts != null) && (fsInputStream != null)) { + gfsBrick = hnts.get(0); + if (!gfsBrick.isChunked()) { + in = fsInputStream; + lastActive = false; + } else { + // find the current location in the tree and the amount of data it can serve + int[] nodeInTree = thisBrick.getBrickNumberInTree(start, nlen[0]); + + // does this node hold the byte ranges we have been requested for ? + if ((nodeInTree[2] != 0) && thisBrick.brickHasFilePart(nodeInTree[0], nodeLocation)) { + in = fsInputStream; + nlen[0] = nodeInTree[2]; // the amount of data that can be read from the stripe + lastActive = false; + } + } + } + + return in; + } + + public synchronized int read () throws IOException { + int byteRead = 0; + RandomAccessFile in = null; + + if (closed) + throw new IOException("Stream Closed."); + + int[] nlen = { 1 }; + + in = chooseStream(getPos(), nlen); + + byteRead = in.read(); + if (byteRead >= 0) { + pos++; + syncStreams(1); + } + + return byteRead; + } + + public synchronized int read (byte buff[], int off, int len) throws + IOException { + int result = 0; + RandomAccessFile in = null; + + if (closed) + throw new IOException("Stream Closed."); + + int[] nlen = {len}; // hack to make len mutable + in = chooseStream(pos, nlen); + + result = in.read(buff, off, nlen[0]); + if (result > 0) { + pos += result; + syncStreams(result); + } + + return result; + } + + /** + * TODO: use seek() insted of skipBytes(); skipBytes does I/O + */ + public void syncStreams (int bytes) throws IOException { + if ((hnts != null) && (hnts.get(0).isChunked()) && (fsInputStream != null)) + if (!this.lastActive) + fuseInputStream.skipBytes(bytes); + else + fsInputStream.skipBytes(bytes); + } + + public synchronized void close () throws IOException { + if (closed) + throw new IOException("Stream closed."); + + super.close(); + if (fsInputStream != null) + fsInputStream.close(); + fuseInputStream.close(); + + closed = true; + } + + // Not supported - mark () and reset () + + public boolean markSupported () { + return false; + } + + public void mark (int readLimit) {} + + public void reset () throws IOException { + throw new IOException("Mark/Reset not supported."); + } +} diff --git a/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEOutputStream.java b/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEOutputStream.java new file mode 100644 index 00000000..5192a0a5 --- /dev/null +++ b/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEOutputStream.java @@ -0,0 +1,86 @@ +/** + * + * Copyright (c) 2011 Gluster, Inc. + * This file is part of GlusterFS. + * + * Licensed 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.hadoop.fs.glusterfs; + +import java.io.*; + +import org.apache.hadoop.fs.FSOutputSummer; +import org.apache.hadoop.fs.FileSystem; + +public class GlusterFUSEOutputStream extends OutputStream { + File f; + long pos; + boolean closed; + OutputStream fuseOutputStream; + + public GlusterFUSEOutputStream (String file, boolean append) throws + IOException { + this.f = new File(file); /* not needed ? */ + this.pos = 0; + this.fuseOutputStream = new FileOutputStream(file, append); + this.closed = false; + } + + public long getPos () throws IOException { + return pos; + } + + public void write (int v) throws IOException { + if (closed) + throw new IOException("Stream closed."); + + byte[] b = new byte[1]; + b[0] = (byte) v; + + write(b, 0, 1); + } + + public void write (byte b[]) throws IOException { + if (closed) + throw new IOException("Stream closed."); + + fuseOutputStream.write(b, 0, b.length); + pos += (long) b.length; + } + + public void write (byte b[], int off, int len) throws IOException { + if (closed) + throw new IOException("Stream closed."); + + fuseOutputStream.write(b, off, len); + pos += (long) len; + } + + public void flush () throws IOException { + if (closed) + throw new IOException("Stream closed."); + + fuseOutputStream.flush(); + } + + public void close () throws IOException { + if (closed) + throw new IOException("Stream closed."); + + flush(); + fuseOutputStream.close(); + closed = true; + } +} diff --git a/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFileSystem.java b/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFileSystem.java new file mode 100644 index 00000000..b0501cce --- /dev/null +++ b/glusterfs-hadoop/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFileSystem.java @@ -0,0 +1,492 @@ +/** + * + * Copyright (c) 2011 Gluster, Inc. + * This file is part of GlusterFS. + * + * Licensed 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. + * + */ + +/** + * Implements the Hadoop FileSystem Interface to allow applications to store + * files on GlusterFS and run Map/Reduce jobs on the data. + */ + +package org.apache.hadoop.fs.glusterfs; + +import java.io.*; +import java.net.*; + +import java.util.regex.*; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.util.Progressable; + +import java.util.TreeMap; + +/* + * This package provides interface for hadoop jobs (incl. Map/Reduce) + * to access files in GlusterFS backed file system via FUSE mount + */ +public class GlusterFileSystem extends FileSystem { + + private FileSystem glusterFs = null; + private URI uri = null; + private Path workingDir = null; + private String glusterMount = null; + private boolean mounted = false; + + /* for quick IO */ + private boolean quickSlaveIO = false; + + /* extended attribute class */ + private GlusterFSXattr xattr = null; + + /* hostname of this machine */ + private static String hostname; + + public GlusterFileSystem () { + + } + + public URI getUri () { + return uri; + } + + public boolean FUSEMount (String volname, String server, String mount) + throws IOException, InterruptedException { + boolean ret = true; + int retVal = 0; + Process p = null; + String s = null; + String mountCmd = null; + + mountCmd = "mount -t glusterfs " + server + ":" + "/" + volname + " " + mount; + + try { + p = Runtime.getRuntime().exec(mountCmd); + + retVal = p.waitFor(); + if (retVal != 0) + ret = false; + + } catch (IOException e) { + System.out.println ("Problem mounting FUSE mount on: " + mount); + e.printStackTrace(); + System.exit(-1); + } + + return ret; + } + + public void initialize (URI uri, Configuration conf) throws IOException { + boolean ret = false; + String volName = null; + String remoteGFSServer = null; + String needQuickRead = null; + + if (this.mounted) + return; + + System.out.println("Initializing GlusterFS"); + + try { + volName = conf.get("fs.glusterfs.volname", ""); + glusterMount = conf.get("fs.glusterfs.mount", ""); + remoteGFSServer = conf.get("fs.glusterfs.server", ""); + needQuickRead = conf.get("quick.slave.io", ""); + + /* + * bail out if we do not have enough information to do a FUSE + * mount + */ + if ( (volName.length() == 0) || (remoteGFSServer.length() == 0) || + (glusterMount.length() == 0) ) + System.exit (-1); + + ret = FUSEMount(volName, remoteGFSServer, glusterMount); + if (!ret) { + System.out.println("Failed to initialize GlusterFS"); + System.exit(-1); + } + + if ((needQuickRead.length() != 0) + && (needQuickRead.equalsIgnoreCase("yes") + || needQuickRead.equalsIgnoreCase("on") + || needQuickRead.equals("1"))) + this.quickSlaveIO = true; + + this.mounted = true; + this.glusterFs = FileSystem.getLocal(conf); + this.workingDir = new Path(glusterMount); + this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority()); + + this.xattr = new GlusterFSXattr(); + + InetAddress addr = InetAddress.getLocalHost(); + this.hostname = addr.getHostName(); + + setConf(conf); + + } catch (Exception e) { + e.printStackTrace(); + System.out.println("Unable to initialize GlusterFS"); + System.exit(-1); + } + } + + @Deprecated + public String getName () { + return getUri().toString(); + } + + public Path getWorkingDirectory () { + return this.workingDir; + } + + public Path getHomeDirectory () { + return this.workingDir; + } + + public Path makeAbsolute (Path path) { + String pth = path.toUri().getPath(); + if (pth.startsWith(workingDir.toUri().getPath())) { + return path; + } + + return new Path(workingDir + "/" + pth); + } + + public void setWorkingDirectory (Path dir) { + this.workingDir = makeAbsolute(dir); + } + + public boolean exists (Path path) throws IOException { + Path absolute = makeAbsolute(path); + File f = new File(absolute.toUri().getPath()); + + return f.exists(); + } + + public boolean mkdirs (Path path, FsPermission permission + ) throws IOException { + boolean created = false; + Path absolute = makeAbsolute(path); + File f = new File(absolute.toUri().getPath()); + + if (f.exists()) { + System.out.println("Directory " + f.getPath() + " already exist"); + return true; + } + + return f.mkdirs(); + } + + @Deprecated + public boolean isDirectory (Path path) throws IOException { + Path absolute = makeAbsolute(path); + File f = new File(absolute.toUri().getPath()); + + return f.isDirectory(); + } + + public boolean isFile (Path path) throws IOException { + return !isDirectory(path); + } + + public Path[] listPaths (Path path) throws IOException { + Path absolute = makeAbsolute(path); + File f = new File (absolute.toUri().getPath()); + String relPath = path.toUri().getPath(); + String[] fileList = null; + Path[] filePath = null; + int fileCnt = 0; + + fileList = f.list(); + + filePath = new Path[fileList.length]; + + for (; fileCnt < fileList.length; fileCnt++) { + filePath[fileCnt] = new Path(relPath + "/" + fileList[fileCnt]); + } + + return filePath; + } + + public FileStatus[] listStatus (Path path) throws IOException { + int fileCnt = 0; + Path absolute = makeAbsolute(path); + String relpath = path.toUri().getPath(); + String[] strFileList = null; + FileStatus[] fileStatus = null; + File f = new File(absolute.toUri().getPath()); + + if (!f.exists()) { + return null; + } + + if (f.isFile()) + return new FileStatus[] { + getFileStatus(path) + }; + + if (relpath.charAt(relpath.length() - 1) != '/') + relpath += "/"; + + strFileList = f.list(); + + fileStatus = new FileStatus[strFileList.length]; + + for (; fileCnt < strFileList.length; fileCnt++) { + fileStatus[fileCnt] = getFileStatusFromFileString(relpath + strFileList[fileCnt]); + } + + return fileStatus; + } + + public FileStatus getFileStatusFromFileString (String path) + throws IOException { + Path nPath = new Path(path); + return getFileStatus(nPath); + } + + public FileStatus getFileStatus (Path path) throws IOException { + Path absolute = makeAbsolute(path); + File f = new File(absolute.toUri().getPath()); + + if (!f.exists ()) + throw new FileNotFoundException("File " + f.getPath() + " does not exist."); + + if (f.isDirectory ()) + return new FileStatus(0, true, 1, 0, f.lastModified(), path.makeQualified(this)); + else + return new FileStatus(f.length(), false, 0, getDefaultBlockSize(), + f.lastModified(), path.makeQualified(this)); + + } + + /* + * creates a new file in glusterfs namespace. internally the file + * descriptor is an instance of OutputStream class. + */ + public FSDataOutputStream create (Path path, FsPermission permission, + boolean overwrite, int bufferSize, + short replication, long blockSize, + Progressable progress) + throws IOException { + Path absolute = makeAbsolute(path); + Path parent = null; + File f = null; + File fParent = null; + FSDataOutputStream glusterFileStream = null; + + f = new File(absolute.toUri().getPath()); + + if (f.exists ()) { + if (overwrite) + f.delete (); + else + throw new IOException(f.getPath() + " already exist"); + } + + parent = path.getParent(); + fParent = new File ((makeAbsolute(parent)).toUri().getPath()); + if ((parent != null) && (fParent != null) && (!fParent.exists())) + if (!fParent.mkdirs()) + throw new IOException("cannot create parent directory: " + fParent.getPath()); + + glusterFileStream = new FSDataOutputStream(new GlusterFUSEOutputStream + (f.getPath(), false)); + + return glusterFileStream; + } + + /* + * open the file in read mode (internally the file descriptor is an + * instance of InputStream class). + * + * if quick read mode is set then read the file by by-passing FUSE + * if we are on same slave where the file exist + */ + public FSDataInputStream open (Path path) throws IOException { + Path absolute = makeAbsolute(path); + File f = new File(absolute.toUri().getPath()); + FSDataInputStream glusterFileStream = null; + TreeMap hnts = null; + + if (!f.exists()) + throw new IOException("File " + f.getPath() + " does not exist."); + + if (quickSlaveIO) + hnts = xattr.quickIOPossible(f.getPath(), 0, f.length()); + + glusterFileStream = new FSDataInputStream(new GlusterFUSEInputStream(f, hnts, hostname)); + return glusterFileStream; + } + + public FSDataInputStream open (Path path, int bufferSize) throws IOException { + return open(path); + } + + public FSDataOutputStream append (Path f, int bufferSize, Progressable progress) + throws IOException { + throw new IOException ("append not supported (as yet)."); + } + + public boolean rename (Path src, Path dst) throws IOException { + Path absoluteSrc = makeAbsolute(src); + Path absoluteDst = makeAbsolute(dst); + + File fSrc = new File(absoluteSrc.toUri().getPath()); + File fDst = new File(absoluteDst.toUri().getPath()); + + if (fDst.isDirectory()) { + fDst = null; + String newPath = absoluteDst.toUri().getPath() + "/" + fSrc.getName(); + fDst = new File(newPath); + } + return fSrc.renameTo(fDst); + } + + @Deprecated + public boolean delete (Path path) throws IOException { + return delete(path, true); + } + + public boolean delete (Path path, boolean recursive) throws IOException { + Path absolute = makeAbsolute(path); + File f = new File(absolute.toUri().getPath()); + + if (f.isFile()) + return f.delete(); + + FileStatus[] dirEntries = listStatus(absolute); + if ((!recursive) && (dirEntries != null) && (dirEntries.length != 0)) + throw new IOException ("Directory " + path.toString() + " is not empty"); + + if (dirEntries != null) + for (int i = 0; i < dirEntries.length; i++) + delete(new Path(absolute, dirEntries[i].getPath()), recursive); + + return f.delete(); + } + + @Deprecated + public long getLength (Path path) throws IOException { + Path absolute = makeAbsolute(path); + File f = new File(absolute.toUri().getPath()); + + if (!f.exists()) + throw new IOException(f.getPath() + " does not exist."); + + return f.length(); + } + + @Deprecated + public short getReplication (Path path) throws IOException { + Path absolute = makeAbsolute(path); + File f = new File(absolute.toUri().getPath()); + + if (!f.exists()) + throw new IOException(f.getPath() + " does not exist."); + + return xattr.getReplication(f.getPath()); + } + + public short getDefaultReplication (Path path) throws IOException { + return getReplication(path); + } + + public boolean setReplication (Path path, short replication) + throws IOException { + return true; + } + + public long getBlockSize (Path path) throws IOException { + long blkSz; + Path absolute = makeAbsolute(path); + File f = new File(absolute.toUri().getPath()); + + blkSz = xattr.getBlockSize(f.getPath()); + if (blkSz == 0) + blkSz = getLength(path); + + return blkSz; + } + + public long getDefaultBlockSize () { + return 1 << 26; /* default's from hdfs, kfs */ + } + + @Deprecated + public void lock (Path path, boolean shared) throws IOException { + } + + @Deprecated + public void release (Path path) throws IOException { + } + + public BlockLocation[] getFileBlockLocations (FileStatus file, long start, long len) + throws IOException { + + Path absolute = makeAbsolute(file.getPath()); + File f = new File(absolute.toUri().getPath()); + BlockLocation[] result = null; + + if (file == null) + return null; + + result = xattr.getPathInfo(f.getPath(), start, len); + if (result == null) { + System.out.println("Problem getting destination host for file " + + f.getPath()); + return null; + } + + return result; + } + + // getFileBlockLocations (FileStatus, long, long) is called by hadoop + public BlockLocation[] getFileBlockLocations (Path p, long start, long len) + throws IOException { + return null; + } + + public void copyFromLocalFile (boolean delSrc, Path src, Path dst) + throws IOException { + FileUtil.copy(glusterFs, src, this, dst, delSrc, getConf()); + } + + public void copyToLocalFile (boolean delSrc, Path src, Path dst) + throws IOException { + FileUtil.copy(this, src, glusterFs, dst, delSrc, getConf()); + } + + public Path startLocalOutput (Path fsOutputFile, Path tmpLocalFile) + throws IOException { + return tmpLocalFile; + } + + public void completeLocalOutput (Path fsOutputFile, Path tmpLocalFile) + throws IOException { + moveFromLocalFile(tmpLocalFile, fsOutputFile); + } +} diff --git a/glusterfs-hadoop/0.20.2/src/test/java/org/apache/hadoop/fs/glusterfs/AppTest.java b/glusterfs-hadoop/0.20.2/src/test/java/org/apache/hadoop/fs/glusterfs/AppTest.java new file mode 100644 index 00000000..21e188c5 --- /dev/null +++ b/glusterfs-hadoop/0.20.2/src/test/java/org/apache/hadoop/fs/glusterfs/AppTest.java @@ -0,0 +1,38 @@ +package org.apache.hadoop.fs.glusterfs; + +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; + +/** + * Unit test for simple App. + */ +public class AppTest + extends TestCase +{ + /** + * Create the test case + * + * @param testName name of the test case + */ + public AppTest( String testName ) + { + super( testName ); + } + + /** + * @return the suite of tests being tested + */ + public static Test suite() + { + return new TestSuite( AppTest.class ); + } + + /** + * Rigourous Test :-) + */ + public void testApp() + { + assertTrue( true ); + } +} diff --git a/glusterfs-hadoop/0.20.2/tools/build-deploy-jar.py b/glusterfs-hadoop/0.20.2/tools/build-deploy-jar.py new file mode 100644 index 00000000..450e08fb --- /dev/null +++ b/glusterfs-hadoop/0.20.2/tools/build-deploy-jar.py @@ -0,0 +1,212 @@ +#!/usr/bin/python + +## + # + # Copyright (c) 2011 Gluster, Inc. + # This file is part of GlusterFS. + # + # Licensed 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. + # + ## + +import getopt +import glob +import sys, os +import shutil +import subprocess, shlex + +def usage(): + print "usage: python build-deploy-jar.py [-b/--build] -d/--dir [-c/--core] [-m/--mapred] [-h/--henv]" + +def addSlash(s): + if not (s[-1] == '/'): + s = s + '/' + + return s + +def whereis(program): + abspath = None + for path in (os.environ.get('PATH', '')).split(':'): + abspath = os.path.join(path, program) + if os.path.exists(abspath) and not os.path.isdir(abspath): + return abspath + + return None + +def getLatestJar(targetdir): + latestJar = None + glusterfsJar = glob.glob(targetdir + "*.jar") + if len(glusterfsJar) == 0: + print "No GlusterFS jar file found in %s ... exiting" % (targetdir) + return None + + # pick up the latest jar file - just in case ... + stat = latestJar = None + ctime = 0 + + for jar in glusterfsJar: + stat = os.stat(jar) + if stat.st_ctime > ctime: + latestJar = jar + ctime = stat.st_ctime + + return latestJar + +# build the glusterfs hadoop plugin using maven +def build_jar(): + location = whereis('mvn') + + if location == None: + print "Cannot find maven to build glusterfs hadoop jar" + print "please install maven or if it's already installed then fix your PATH environ" + return None + + # do a clean packaging + targetdir = "./target/" + if os.path.exists(targetdir) and os.path.isdir(targetdir): + print "Cleaning up directories ... [ " + targetdir + " ]" + shutil.rmtree(targetdir) + + print "Building glusterfs jar ..." + process = subprocess.Popen(['package'], shell=True, + executable=location, stdout=subprocess.PIPE, stderr=subprocess.PIPE) + + try: + (pout, perr) = process.communicate() + except: + process.wait() + if not process.returncode == 0: + print "Building glusterfs jar failed" + return None + + latestJar = getLatestJar(targetdir) + return latestJar + +def rcopy(f, host, libdir): + print " * doing remote copy to host %s" % (host) + scpCmd = "scp %s %s:%s" % (f, host, libdir) + + os.system(scpCmd); + +def deployInSlave(f, confdir, libdir, cc, cm, he): + slavefile = confdir + "slaves" + + ccFile = confdir + "core-site.xml" + cmFile = confdir + "mapred-site.xml" + heFile = confdir + "hadoop-env.sh" + + sf = open(slavefile, 'r') + for host in sf: + host = host.rstrip('\n') + print " >>> Deploying %s on %s ..." % (os.path.basename(f), host) + rcopy(f, host, libdir) + + if cc: + print " >>> Deploying [%s] on %s ..." % (os.path.basename(ccFile), host) + rcopy(ccFile, host, confdir) + + if cm: + print " >>> Deploying [%s] on %s ..." % (os.path.basename(cmFile), host) + rcopy(cmFile, host, confdir) + + if he: + print " >>> Deploying [%s] on %s ..." % (os.path.basename(heFile), host) + rcopy(heFile, host, confdir); + + print "<<< Done\n" + + sf.close() + +def deployInMaster(f, confdir, libdir): + import socket + masterfile = confdir + "masters" + + mf = open(masterfile, 'r') + for host in mf: + host = host.rstrip('\n') + print " >>> Deploying %s on %s ..." % (os.path.basename(f), host) + h = host + try: + socket.inet_aton(host) + h = socket.getfqdn(host) + except socket.error: + # host is not a ip adddress + pass + + if h == socket.gethostname() or h == 'localhost': + # local cp + print " * doing local copy" + shutil.copy(f, libdir) + else: + # scp the file + rcopy(f, h, libdir) + + print "<<< Done\n" + + mf.close() + +if __name__ == '__main__': + opt = args = [] + try: + opt, args = getopt.getopt(sys.argv[1:], "bd:cmh", ["build", "dir=", "core", "mapred", "henv"]); + except getopt.GetoptError, err: + print str(err) + usage() + sys.exit(1) + + needbuild = hadoop_dir = copyCore = copyMapred = copyHadoopEnv = None + + for k, v in opt: + if k in ("-b", "--build"): + needbuild = True + elif k in ("-d", "--dir"): + hadoop_dir = v + elif k in ("-c", "--core"): + copyCore = True + elif k in ("-m", "--mapred"): + copyMapred = True + elif k in ("-h", "--henv"): + copyHadoopEnv = True + else: + assert False, "unhandled option" + + assert not hadoop_dir == None, "hadoop directory missing" + + if needbuild: + jar = build_jar() + if jar == None: + sys.exit(1) + else: + jar = getLatestJar('./target/') + if jar == None: + print "Maybe you want to build it ? -b option" + sys.exit(1) + + print "" + print "*** Deploying %s *** " % (jar) + + # copy jar to local hadoop distribution (master) + hadoop_home = addSlash(hadoop_dir) + if not (os.path.exists(hadoop_home) and os.path.isdir(hadoop_home)): + print "path " + hadoop_home + " does not exist or is not adiretory"; + sys.exit(1); + + hadoop_conf = hadoop_home + "conf/" + hadoop_lib = hadoop_home + "lib/" + + print " >>> Scanning hadoop master file for host(s) to deploy" + deployInMaster(jar, hadoop_conf, hadoop_lib) + + print "" + print " >>> Scanning hadoop slave file for host(s) to deploy" + deployInSlave(jar, hadoop_conf, hadoop_lib, copyCore, copyMapred, copyHadoopEnv) diff --git a/glusterfs-hadoop/COPYING b/glusterfs-hadoop/COPYING new file mode 100644 index 00000000..d6456956 --- /dev/null +++ b/glusterfs-hadoop/COPYING @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/glusterfs-hadoop/README b/glusterfs-hadoop/README new file mode 100644 index 00000000..3026f11c --- /dev/null +++ b/glusterfs-hadoop/README @@ -0,0 +1,182 @@ +GlusterFS Hadoop Plugin +======================= + +INTRODUCTION +------------ + +This document describes how to use GlusterFS (http://www.gluster.org/) as a backing store with Hadoop. + + +REQUIREMENTS +------------ + +* Supported OS is GNU/Linux +* GlusterFS and Hadoop installed on all machines in the cluster +* Java Runtime Environment (JRE) +* Maven (needed if you are building the plugin from source) +* JDK (needed if you are building the plugin from source) + +NOTE: Plugin relies on two *nix command line utilities to function properly. They are: + +* mount: Used to mount GlusterFS volumes. +* getfattr: Used to fetch Extended Attributes of a file + +Make sure they are installed on all hosts in the cluster and their locations are in $PATH +environment variable. + + +INSTALLATION +------------ + +** NOTE: Example below is for Hadoop version 0.20.2 ($GLUSTER_HOME/hdfs/0.20.2) ** + +* Building the plugin from source [Maven (http://maven.apache.org/) and JDK is required to build the plugin] + + Change to glusterfs-hadoop directory in the GlusterFS source tree and build the plugin. + + # cd $GLUSTER_HOME/hdfs/0.20.2 + # mvn package + + On a successful build the plugin will be present in the `target` directory. + (NOTE: version number will be a part of the plugin) + + # ls target/ + classes glusterfs-0.20.2-0.1.jar maven-archiver surefire-reports test-classes + ^^^^^^^^^^^^^^^^^^ + + Copy the plugin to lib/ directory in your $HADOOP_HOME dir. + + # cp target/glusterfs-0.20.2-0.1.jar $HADOOP_HOME/lib + + Copy the sample configuration file that ships with this source (conf/core-site.xml) to conf + directory in your $HADOOP_HOME dir. + + # cp conf/core-site.xml $HADOOP_HOME/conf + +* Installing the plugin from RPM + + See the plugin documentation for installing from RPM. + + +CLUSTER INSTALLATION +-------------------- + + In case it is tedious to do the above steps(s) on all hosts in the cluster; use the build-and-deploy.py script to + build the plugin in one place and deploy it (along with the configuration file on all other hosts). + + This should be run on the host which is that hadoop master [Job Tracker]. + +* STEPS (You would have done Step 1 and 2 anyway while deploying Hadoop) + + 1. Edit conf/slaves file in your hadoop distribution; one line for each slave. + 2. Setup password-less ssh b/w hadoop master and slave(s). + 3. Edit conf/core-site.xml with all glusterfs related configurations (see CONFIGURATION) + 4. Run the following + # cd $GLUSTER_HOME/hdfs/0.20.2/tools + # python ./build-and-deploy.py -b -d /path/to/hadoop/home -c + + This will build the plugin and copy it (and the config file) to all slaves (mentioned in $HADOOP_HOME/conf/slaves). + + Script options: + -b : build the plugin + -d : location of hadoop directory + -c : deploy core-site.xml + -m : deploy mapred-site.xml + -h : deploy hadoop-env.sh + + +CONFIGURATION +------------- + + All plugin configuration is done in a single XML file (core-site.xml) with tags in each + block. + + Brief explanation of the tunables and the values they accept (change them where-ever needed) are mentioned below + + name: fs.glusterfs.impl + value: org.apache.hadoop.fs.glusterfs.GlusterFileSystem + + The default FileSystem API to use (there is little reason to modify this). + + name: fs.default.name + value: glusterfs://server:port + + The default name that hadoop uses to represent file as a URI (typically a server:port tuple). Use any host + in the cluster as the server and any port number. This option has to be in server:port format for hadoop + to create file URI; but is not used by plugin. + + name: fs.glusterfs.volname + value: volume-dist-rep + + The volume to mount. + + + name: fs.glusterfs.mount + value: /mnt/glusterfs + + This is the directory that the plugin will use to mount (FUSE mount) the volume. + + name: fs.glusterfs.server + value: 192.168.1.36, hackme.zugzug.org + + To mount a volume the plugin needs to know the hostname or the IP of a GlusterFS server in the cluster. + Mention it here. + + name: quick.slave.io + value: [On/Off], [Yes/No], [1/0] + + NOTE: This option is not tested as of now. + + This is a performance tunable option. Hadoop schedules jobs to hosts that contain the file data part. The job + then does I/O on the file (via FUSE in case of GlusterFS). When this option is set, the plugin will try to + do I/O directly from the backed filesystem (ext3, ext4 etc..) the file resides on. Hence read performance + will improve and job would run faster. + + +USAGE +----- + + Once configured, start Hadoop Map/Reduce daemons + + # cd $HADOOP_HOME + # ./bin/start-mapred.sh + + If the map/reduce job/task trackers are up, all I/O will be done to GlusterFS. + + +FOR HACKERS +----------- + +* Source Layout + +** version specific: hdfs/ ** +./src +./src/main +./src/main/java +./src/main/java/org +./src/main/java/org/apache +./src/main/java/org/apache/hadoop +./src/main/java/org/apache/hadoop/fs +./src/main/java/org/apache/hadoop/fs/glusterfs +./src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickClass.java +./src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSXattr.java <--- Fetch/Parse Extended Attributes of a file +./src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEInputStream.java <--- Input Stream (instantiated during open() calls; quick read from backed FS) +./src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickRepl.java +./src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEOutputStream.java <--- Output Stream (instantiated during creat() calls) +./src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFileSystem.java <--- Entry Point for the plugin (extends Hadoop FileSystem class) +./src/test +./src/test/java +./src/test/java/org +./src/test/java/org/apache +./src/test/java/org/apache/hadoop +./src/test/java/org/apache/hadoop/fs +./src/test/java/org/apache/hadoop/fs/glusterfs +./src/test/java/org/apache/hadoop/fs/glusterfs/AppTest.java <--- Your test cases go here (if any :-)) +./tools/build-deploy-jar.py <--- Build and Deployment Script +./conf +./conf/core-site.xml <--- Sample configuration file +./pom.xml <--- build XML file (used by maven) + +** toplevel: hdfs/ ** +./COPYING <--- License +./README <--- This file diff --git a/hdfs/0.20.2/conf/core-site.xml b/hdfs/0.20.2/conf/core-site.xml deleted file mode 100644 index d7f75fca..00000000 --- a/hdfs/0.20.2/conf/core-site.xml +++ /dev/null @@ -1,38 +0,0 @@ - - - - - - - - - fs.glusterfs.impl - org.apache.hadoop.fs.glusterfs.GlusterFileSystem - - - - fs.default.name - glusterfs://192.168.1.36:9000 - - - - fs.glusterfs.volname - volume-dist-rep - - - - fs.glusterfs.mount - /mnt/glusterfs - - - - fs.glusterfs.server - 192.168.1.36 - - - - quick.slave.io - Off - - - diff --git a/hdfs/0.20.2/pom.xml b/hdfs/0.20.2/pom.xml deleted file mode 100644 index fe661d40..00000000 --- a/hdfs/0.20.2/pom.xml +++ /dev/null @@ -1,36 +0,0 @@ - - 4.0.0 - org.apache.hadoop.fs.glusterfs - glusterfs - jar - 0.20.2-0.1 - glusterfs - http://maven.apache.org - - - junit - junit - 3.8.1 - test - - - org.apache.hadoop - hadoop-core - 0.20.2 - - - - - - org.apache.maven.plugins - maven-compiler-plugin - 2.3.2 - - 1.5 - 1.5 - - - - - diff --git a/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickClass.java b/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickClass.java deleted file mode 100644 index e633b8aa..00000000 --- a/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickClass.java +++ /dev/null @@ -1,109 +0,0 @@ -/** - * - * Copyright (c) 2011 Gluster, Inc. - * This file is part of GlusterFS. - * - * Licensed 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.hadoop.fs.glusterfs; - -import java.io.*; - -public class GlusterFSBrickClass { - String host; - String exportedFile; - long start; - long end; - boolean isChunked; - int stripeSize; // Stripe size in bytes - int nrStripes; // number of stripes - int switchCount; // for SR, DSR - number of replicas of each stripe - // -1 for others - - public GlusterFSBrickClass (String brick, long start, long len, boolean flag, - int stripeSize, int nrStripes, int switchCount) - throws IOException { - this.host = brick2host(brick); - this.exportedFile = brick2file(brick); - this.start = start; - this.end = start + len; - this.isChunked = flag; - this.stripeSize = stripeSize; - this.nrStripes = nrStripes; - this.switchCount = switchCount; - } - - public boolean isChunked () { - return isChunked; - } - - public String brickIsLocal(String hostname) { - String path = null; - File f = null; - if (host.equals(hostname)) - path = exportedFile; - - return path; - } - - public int[] getBrickNumberInTree(long start, int len) { - long end = len; - int startNodeInTree = ((int) (start / stripeSize)) % nrStripes; - int endNodeInTree = ((int) ((start + len) / stripeSize)) % nrStripes; - - if (startNodeInTree != endNodeInTree) { - end = (start - (start % stripeSize)) + stripeSize; - end -= start; - } - - return new int[] {startNodeInTree, endNodeInTree, (int) end}; - } - - public boolean brickHasFilePart(int nodeInTree, int nodeLoc) { - if (switchCount == -1) - return (nodeInTree == nodeLoc); - - nodeInTree *= switchCount; - for (int i = nodeInTree; i < (nodeInTree + switchCount); i++) { - if (i == nodeLoc) - return true; - } - - return false; - } - - public String brick2host (String brick) - throws IOException { - String[] hf = null; - - hf = brick.split(":"); - if (hf.length != 2) - throw new IOException("Error getting hostname from brick"); - - return hf[0]; - } - - public String brick2file (String brick) - throws IOException { - String[] hf = null; - - hf = brick.split(":"); - if (hf.length != 2) - throw new IOException("Error getting hostname from brick"); - - return hf[1]; - } - -} diff --git a/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickRepl.java b/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickRepl.java deleted file mode 100644 index 11454e63..00000000 --- a/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickRepl.java +++ /dev/null @@ -1,52 +0,0 @@ -/** - * - * Copyright (c) 2011 Gluster, Inc. - * This file is part of GlusterFS. - * - * Licensed 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.hadoop.fs.glusterfs; - -import java.io.*; - -public class GlusterFSBrickRepl { - private String[] replHost; - private long start; - private long len; - private int cnt; - - GlusterFSBrickRepl(int replCount, long start, long len) { - this.replHost = new String[replCount]; - this.start = start; - this.len = len; - this.cnt = 0; - } - - public void addHost (String host) { - this.replHost[cnt++] = host; - } - - public String[] getReplHosts () { - return this.replHost; - } - - public long getStartLen () { - return this.start; - } - - public long getOffLen () { - return this.len; - } -} diff --git a/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSXattr.java b/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSXattr.java deleted file mode 100644 index 18e9003b..00000000 --- a/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSXattr.java +++ /dev/null @@ -1,471 +0,0 @@ -/** - * - * Copyright (c) 2011 Gluster, Inc. - * This file is part of GlusterFS. - * - * Licensed 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.hadoop.fs.glusterfs; - -import java.net.*; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import java.io.*; -import java.util.HashMap; -import java.util.TreeMap; -import java.util.ArrayList; -import java.util.Iterator; - -import org.apache.hadoop.fs.BlockLocation; - -public class GlusterFSXattr { - - public enum LAYOUT { D, S, R, DS, DR, SR, DSR } - public enum CMD { GET_HINTS, GET_REPLICATION, GET_BLOCK_SIZE, CHECK_FOR_QUICK_IO } - - private static String hostname; - - public GlusterFSXattr() { } - - public static String brick2host (String brick) - throws IOException { - String[] hf = null; - - hf = brick.split(":"); - if (hf.length != 2) { - System.out.println("brick not of format hostname:path"); - throw new IOException("Error getting hostname from brick"); - } - - return hf[0]; - } - - public static String brick2file (String brick) - throws IOException { - String[] hf = null; - - hf = brick.split(":"); - if (hf.length != 2) { - System.out.println("brick not of format hostname:path"); - throw new IOException("Error getting hostname from brick"); - } - - return hf[1]; - } - - public static BlockLocation[] getPathInfo (String filename, long start, long len) - throws IOException { - HashMap> vol = null; - HashMap meta = new HashMap(); - - vol = execGetFattr(filename, meta, CMD.GET_HINTS); - - return getHints(vol, meta, start, len, null); - } - - public static long getBlockSize (String filename) - throws IOException { - HashMap> vol = null; - HashMap meta = new HashMap(); - - vol = execGetFattr(filename, meta, CMD.GET_BLOCK_SIZE); - - if (!meta.containsKey("block-size")) - return 0; - - return (long) meta.get("block-size"); - - } - - public static short getReplication (String filename) - throws IOException { - HashMap> vol = null; - HashMap meta = new HashMap(); - - vol = execGetFattr(filename, meta, CMD.GET_REPLICATION); - - return (short) getReplicationFromLayout(vol, meta); - - } - - public static TreeMap quickIOPossible (String filename, long start, - long len) - throws IOException { - String realpath = null; - HashMap> vol = null; - HashMap meta = new HashMap(); - TreeMap hnts = new TreeMap(); - - vol = execGetFattr(filename, meta, CMD.GET_HINTS); - getHints(vol, meta, start, len, hnts); - - if (hnts.size() == 0) - return null; // BOOM !! - - // DEBUG - dump hnts here - return hnts; - } - - public static HashMap> execGetFattr (String filename, - HashMap meta, - CMD cmd) - throws IOException { - Process p = null; - BufferedReader brInput = null; - String s = null; - String cmdOut = null; - String getfattrCmd = null; - String xlator = null; - String enclosingXl = null; - String enclosingXlVol = null; - String key = null; - String layout = ""; - int rcount = 0; - int scount = 0; - int dcount = 0; - int count = 0; - - HashMap> vol = new HashMap>(); - - getfattrCmd = "getfattr -m . -n trusted.glusterfs.pathinfo " + filename; - - p = Runtime.getRuntime().exec(getfattrCmd); - brInput = new BufferedReader(new InputStreamReader(p.getInputStream())); - - cmdOut = ""; - while ( (s = brInput.readLine()) != null ) - cmdOut += s; - - /** - * TODO: Use a single regex for extracting posix paths as well - * as xlator counts for layout matching. - */ - - Pattern pattern = Pattern.compile("<(.*?)[:\\(](.*?)>"); - Matcher matcher = pattern.matcher(cmdOut); - - Pattern p_px = Pattern.compile(".*?:(.*)"); - Matcher m_px; - String gibberish_path; - - s = null; - while (matcher.find()) { - xlator = matcher.group(1); - if (xlator.equalsIgnoreCase("posix")) { - if (enclosingXl.equalsIgnoreCase("replicate")) - count = rcount; - else if (enclosingXl.equalsIgnoreCase("stripe")) - count = scount; - else if (enclosingXl.equalsIgnoreCase("distribute")) - count = dcount; - else - throw new IOException("Unknown Translator: " + enclosingXl); - - key = enclosingXl + "-" + count; - - if (vol.get(key) == null) - vol.put(key, new ArrayList()); - - gibberish_path = matcher.group(2); - - /* extract posix path from the gibberish string */ - m_px = p_px.matcher(gibberish_path); - if (!m_px.find()) - throw new IOException("Cannot extract posix path"); - - vol.get(key).add(m_px.group(1)); - continue; - } - - enclosingXl = xlator; - enclosingXlVol = matcher.group(2); - - if (xlator.equalsIgnoreCase("replicate")) - if (rcount++ != 0) - continue; - - if (xlator.equalsIgnoreCase("stripe")) { - if (scount++ != 0) - continue; - - - Pattern ps = Pattern.compile("\\[(\\d+)\\]"); - Matcher ms = ps.matcher(enclosingXlVol); - - if (ms.find()) { - if (((cmd == CMD.GET_BLOCK_SIZE) || (cmd == CMD.GET_HINTS)) - && (meta != null)) - meta.put("block-size", Integer.parseInt(ms.group(1))); - } else - throw new IOException("Cannot get stripe size"); - } - - if (xlator.equalsIgnoreCase("distribute")) - if (dcount++ != 0) - continue; - - layout += xlator.substring(0, 1); - } - - if ((dcount == 0) && (scount == 0) && (rcount == 0)) - throw new IOException("Cannot get layout"); - - if (meta != null) { - meta.put("dcount", dcount); - meta.put("scount", scount); - meta.put("rcount", rcount); - } - - vol.put("layout", new ArrayList(1)); - vol.get("layout").add(layout); - - return vol; - } - - static BlockLocation[] getHints (HashMap> vol, - HashMap meta, - long start, long len, - TreeMap hnts) - throws IOException { - String brick = null; - String key = null; - boolean done = false; - int i = 0; - int counter = 0; - int stripeSize = 0; - long stripeStart = 0; - long stripeEnd = 0; - int nrAllocs = 0; - int allocCtr = 0; - BlockLocation[] result = null; - ArrayList brickList = null; - ArrayList stripedBricks = null; - Iterator it = null; - - String[] blks = null; - GlusterFSBrickRepl[] repl = null; - int dcount, scount, rcount; - - LAYOUT l = LAYOUT.valueOf(vol.get("layout").get(0)); - dcount = meta.get("dcount"); - scount = meta.get("scount"); - rcount = meta.get("rcount"); - - switch (l) { - case D: - key = "DISTRIBUTE-" + dcount; - brick = vol.get(key).get(0); - - if (hnts == null) { - result = new BlockLocation[1]; - result[0] = new BlockLocation(null, new String[] {brick2host(brick)}, start, len); - } else - hnts.put(0, new GlusterFSBrickClass(brick, start, len, false, -1, -1, -1)); - break; - - case R: - case DR: - /* just the name says it's striped - the volume isn't */ - stripedBricks = new ArrayList(); - - for (i = 1; i <= rcount; i++) { - key = "REPLICATE-" + i; - brickList = vol.get(key); - it = brickList.iterator(); - while (it.hasNext()) { - stripedBricks.add(it.next()); - } - } - - nrAllocs = stripedBricks.size(); - if (hnts == null) { - result = new BlockLocation[1]; - blks = new String[nrAllocs]; - } - - for (i = 0; i < nrAllocs; i++) { - if (hnts == null) - blks[i] = brick2host(stripedBricks.get(i)); - else - hnts.put(i, new GlusterFSBrickClass(stripedBricks.get(i), start, len, false, -1, -1, -1)); - } - - if (hnts == null) - result[0] = new BlockLocation(null, blks, start, len); - - break; - - case SR: - case DSR: - int rsize = 0; - ArrayList> replicas = new ArrayList>(); - - stripedBricks = new ArrayList(); - - if (rcount == 0) - throw new IOException("got replicated volume with replication count 0"); - - for (i = 1; i <= rcount; i++) { - key = "REPLICATE-" + i; - brickList = vol.get(key); - it = brickList.iterator(); - replicas.add(i - 1, new ArrayList()); - while (it.hasNext()) { - replicas.get(i - 1).add(it.next()); - } - } - - rsize = replicas.get(0).size(); - stripeSize = meta.get("block-size"); - - nrAllocs = (int) (((len - start) / stripeSize) + 1); - if (hnts == null) { - result = new BlockLocation[nrAllocs]; - repl = new GlusterFSBrickRepl[nrAllocs]; - } - - // starting stripe position - counter = (int) ((start / stripeSize) % rcount); - stripeStart = start; - - key = null; - int currAlloc = 0; - boolean hntsDone = false; - while ((stripeStart < len) && !done) { - stripeEnd = (stripeStart - (stripeStart % stripeSize)) + stripeSize - 1; - if (stripeEnd > start + len) { - stripeEnd = start + len - 1; - done = true; - } - - if (hnts == null) - repl[allocCtr] = new GlusterFSBrickRepl(rsize, stripeStart, (stripeEnd - stripeStart)); - - for (i = 0; i < rsize; i++) { - brick = replicas.get(counter).get(i); - currAlloc = (allocCtr * rsize) + i; - - if (hnts == null) - repl[allocCtr].addHost(brick2host(brick)); - else - if (currAlloc <= (rsize * rcount) - 1) { - hnts.put(currAlloc, new GlusterFSBrickClass(brick, stripeStart, - (stripeEnd - stripeStart), - true, stripeSize, rcount, rsize)); - } else - hntsDone = true; - } - - if (hntsDone) - break; - - stripeStart = stripeEnd + 1; - - allocCtr++; - counter++; - - if (counter >= replicas.size()) - counter = 0; - } - - if (hnts == null) - for (int k = 0; k < nrAllocs; k++) - result[k] = new BlockLocation(null, repl[k].getReplHosts(), repl[k].getStartLen(), repl[k].getOffLen()); - - break; - - case S: - case DS: - if (scount == 0) - throw new IOException("got striped volume with stripe count 0"); - - stripedBricks = new ArrayList(); - stripeSize = meta.get("block-size"); - - key = "STRIPE-" + scount; - brickList = vol.get(key); - it = brickList.iterator(); - while (it.hasNext()) { - stripedBricks.add(it.next()); - } - - nrAllocs = (int) ((len - start) / stripeSize) + 1; - if (hnts == null) - result = new BlockLocation[nrAllocs]; - - // starting stripe position - counter = (int) ((start / stripeSize) % stripedBricks.size()); - stripeStart = start; - - key = null; - while ((stripeStart < len) && !done) { - brick = stripedBricks.get(counter); - - stripeEnd = (stripeStart - (stripeStart % stripeSize)) + stripeSize - 1; - if (stripeEnd > start + len) { - stripeEnd = start + len - 1; - done = true; - } - - if (hnts == null) - result[allocCtr] = new BlockLocation(null, new String[] {brick2host(brick)}, - stripeStart, (stripeEnd - stripeStart)); - else - if (allocCtr <= stripedBricks.size()) { - hnts.put(allocCtr, new GlusterFSBrickClass(brick, stripeStart, (stripeEnd - stripeStart), - true, stripeSize, stripedBricks.size(), -1)); - } else - break; - - stripeStart = stripeEnd + 1; - - counter++; - allocCtr++; - - if (counter >= stripedBricks.size()) - counter = 0; - } - - break; - } - - return result; - } - - /* TODO: use meta{dcount,scount,rcount} for checking */ - public static int getReplicationFromLayout (HashMap> vol, - HashMap meta) - throws IOException { - int replication = 0; - LAYOUT l = LAYOUT.valueOf(vol.get("layout").get(0)); - - switch (l) { - case D: - case S: - case DS: - replication = 1; - break; - - case R: - case DR: - case SR: - case DSR: - final String key = "REPLICATION-1"; - replication = vol.get(key).size(); - } - - return replication; - } -} diff --git a/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEInputStream.java b/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEInputStream.java deleted file mode 100644 index e92237ae..00000000 --- a/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEInputStream.java +++ /dev/null @@ -1,205 +0,0 @@ -/** - * - * Copyright (c) 2011 Gluster, Inc. - * This file is part of GlusterFS. - * - * Licensed 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.hadoop.fs.glusterfs; - -import java.io.*; -import java.util.TreeMap; - -import org.apache.hadoop.fs.FSInputStream; -import org.apache.hadoop.fs.FileSystem; - - -public class GlusterFUSEInputStream extends FSInputStream { - File f; - boolean lastActive; - long pos; - boolean closed; - String thisHost; - RandomAccessFile fuseInputStream; - RandomAccessFile fsInputStream; - GlusterFSBrickClass thisBrick; - int nodeLocation; - TreeMap hnts; - - public GlusterFUSEInputStream (File f, TreeMap hnts, - String hostname) throws IOException { - this.f = f; - this.pos = 0; - this.closed = false; - this.hnts = hnts; - this.thisHost = hostname; - this.fsInputStream = null; - this.fuseInputStream = new RandomAccessFile(f.getPath(), "r"); - - this.lastActive = true; // true == FUSE, false == backed file - - String directFilePath = null; - if (this.hnts != null) { - directFilePath = findLocalFile(f.getPath(), this.hnts); - if (directFilePath != null) { - this.fsInputStream = new RandomAccessFile(directFilePath, "r"); - this.lastActive = !this.lastActive; - } - } - } - - public String findLocalFile (String path, TreeMap hnts) { - int i = 0; - String actFilePath = null; - GlusterFSBrickClass gfsBrick = null; - - gfsBrick = hnts.get(0); - - /* do a linear search for the matching host not worrying - about file stripes */ - for (i = 0; i < hnts.size(); i++) { - gfsBrick = hnts.get(i); - actFilePath = gfsBrick.brickIsLocal(this.thisHost); - if (actFilePath != null) { - this.thisBrick = gfsBrick; - this.nodeLocation = i; - break; - } - } - - return actFilePath; - } - - public long getPos () throws IOException { - return pos; - } - - public synchronized int available () throws IOException { - return (int) ((f.length()) - getPos()); - } - - public void seek (long pos) throws IOException { - fuseInputStream.seek(pos); - if (fsInputStream != null) - fsInputStream.seek(pos); - } - - public boolean seekToNewSource (long pos) throws IOException { - return false; - } - - public RandomAccessFile chooseStream (long start, int[] nlen) - throws IOException { - GlusterFSBrickClass gfsBrick = null; - RandomAccessFile in = fuseInputStream; - boolean oldActiveStream = lastActive; - lastActive = true; - - if ((hnts != null) && (fsInputStream != null)) { - gfsBrick = hnts.get(0); - if (!gfsBrick.isChunked()) { - in = fsInputStream; - lastActive = false; - } else { - // find the current location in the tree and the amount of data it can serve - int[] nodeInTree = thisBrick.getBrickNumberInTree(start, nlen[0]); - - // does this node hold the byte ranges we have been requested for ? - if ((nodeInTree[2] != 0) && thisBrick.brickHasFilePart(nodeInTree[0], nodeLocation)) { - in = fsInputStream; - nlen[0] = nodeInTree[2]; // the amount of data that can be read from the stripe - lastActive = false; - } - } - } - - return in; - } - - public synchronized int read () throws IOException { - int byteRead = 0; - RandomAccessFile in = null; - - if (closed) - throw new IOException("Stream Closed."); - - int[] nlen = { 1 }; - - in = chooseStream(getPos(), nlen); - - byteRead = in.read(); - if (byteRead >= 0) { - pos++; - syncStreams(1); - } - - return byteRead; - } - - public synchronized int read (byte buff[], int off, int len) throws - IOException { - int result = 0; - RandomAccessFile in = null; - - if (closed) - throw new IOException("Stream Closed."); - - int[] nlen = {len}; // hack to make len mutable - in = chooseStream(pos, nlen); - - result = in.read(buff, off, nlen[0]); - if (result > 0) { - pos += result; - syncStreams(result); - } - - return result; - } - - /** - * TODO: use seek() insted of skipBytes(); skipBytes does I/O - */ - public void syncStreams (int bytes) throws IOException { - if ((hnts != null) && (hnts.get(0).isChunked()) && (fsInputStream != null)) - if (!this.lastActive) - fuseInputStream.skipBytes(bytes); - else - fsInputStream.skipBytes(bytes); - } - - public synchronized void close () throws IOException { - if (closed) - throw new IOException("Stream closed."); - - super.close(); - if (fsInputStream != null) - fsInputStream.close(); - fuseInputStream.close(); - - closed = true; - } - - // Not supported - mark () and reset () - - public boolean markSupported () { - return false; - } - - public void mark (int readLimit) {} - - public void reset () throws IOException { - throw new IOException("Mark/Reset not supported."); - } -} diff --git a/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEOutputStream.java b/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEOutputStream.java deleted file mode 100644 index 5192a0a5..00000000 --- a/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEOutputStream.java +++ /dev/null @@ -1,86 +0,0 @@ -/** - * - * Copyright (c) 2011 Gluster, Inc. - * This file is part of GlusterFS. - * - * Licensed 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.hadoop.fs.glusterfs; - -import java.io.*; - -import org.apache.hadoop.fs.FSOutputSummer; -import org.apache.hadoop.fs.FileSystem; - -public class GlusterFUSEOutputStream extends OutputStream { - File f; - long pos; - boolean closed; - OutputStream fuseOutputStream; - - public GlusterFUSEOutputStream (String file, boolean append) throws - IOException { - this.f = new File(file); /* not needed ? */ - this.pos = 0; - this.fuseOutputStream = new FileOutputStream(file, append); - this.closed = false; - } - - public long getPos () throws IOException { - return pos; - } - - public void write (int v) throws IOException { - if (closed) - throw new IOException("Stream closed."); - - byte[] b = new byte[1]; - b[0] = (byte) v; - - write(b, 0, 1); - } - - public void write (byte b[]) throws IOException { - if (closed) - throw new IOException("Stream closed."); - - fuseOutputStream.write(b, 0, b.length); - pos += (long) b.length; - } - - public void write (byte b[], int off, int len) throws IOException { - if (closed) - throw new IOException("Stream closed."); - - fuseOutputStream.write(b, off, len); - pos += (long) len; - } - - public void flush () throws IOException { - if (closed) - throw new IOException("Stream closed."); - - fuseOutputStream.flush(); - } - - public void close () throws IOException { - if (closed) - throw new IOException("Stream closed."); - - flush(); - fuseOutputStream.close(); - closed = true; - } -} diff --git a/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFileSystem.java b/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFileSystem.java deleted file mode 100644 index b0501cce..00000000 --- a/hdfs/0.20.2/src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFileSystem.java +++ /dev/null @@ -1,492 +0,0 @@ -/** - * - * Copyright (c) 2011 Gluster, Inc. - * This file is part of GlusterFS. - * - * Licensed 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. - * - */ - -/** - * Implements the Hadoop FileSystem Interface to allow applications to store - * files on GlusterFS and run Map/Reduce jobs on the data. - */ - -package org.apache.hadoop.fs.glusterfs; - -import java.io.*; -import java.net.*; - -import java.util.regex.*; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.BlockLocation; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.util.Progressable; - -import java.util.TreeMap; - -/* - * This package provides interface for hadoop jobs (incl. Map/Reduce) - * to access files in GlusterFS backed file system via FUSE mount - */ -public class GlusterFileSystem extends FileSystem { - - private FileSystem glusterFs = null; - private URI uri = null; - private Path workingDir = null; - private String glusterMount = null; - private boolean mounted = false; - - /* for quick IO */ - private boolean quickSlaveIO = false; - - /* extended attribute class */ - private GlusterFSXattr xattr = null; - - /* hostname of this machine */ - private static String hostname; - - public GlusterFileSystem () { - - } - - public URI getUri () { - return uri; - } - - public boolean FUSEMount (String volname, String server, String mount) - throws IOException, InterruptedException { - boolean ret = true; - int retVal = 0; - Process p = null; - String s = null; - String mountCmd = null; - - mountCmd = "mount -t glusterfs " + server + ":" + "/" + volname + " " + mount; - - try { - p = Runtime.getRuntime().exec(mountCmd); - - retVal = p.waitFor(); - if (retVal != 0) - ret = false; - - } catch (IOException e) { - System.out.println ("Problem mounting FUSE mount on: " + mount); - e.printStackTrace(); - System.exit(-1); - } - - return ret; - } - - public void initialize (URI uri, Configuration conf) throws IOException { - boolean ret = false; - String volName = null; - String remoteGFSServer = null; - String needQuickRead = null; - - if (this.mounted) - return; - - System.out.println("Initializing GlusterFS"); - - try { - volName = conf.get("fs.glusterfs.volname", ""); - glusterMount = conf.get("fs.glusterfs.mount", ""); - remoteGFSServer = conf.get("fs.glusterfs.server", ""); - needQuickRead = conf.get("quick.slave.io", ""); - - /* - * bail out if we do not have enough information to do a FUSE - * mount - */ - if ( (volName.length() == 0) || (remoteGFSServer.length() == 0) || - (glusterMount.length() == 0) ) - System.exit (-1); - - ret = FUSEMount(volName, remoteGFSServer, glusterMount); - if (!ret) { - System.out.println("Failed to initialize GlusterFS"); - System.exit(-1); - } - - if ((needQuickRead.length() != 0) - && (needQuickRead.equalsIgnoreCase("yes") - || needQuickRead.equalsIgnoreCase("on") - || needQuickRead.equals("1"))) - this.quickSlaveIO = true; - - this.mounted = true; - this.glusterFs = FileSystem.getLocal(conf); - this.workingDir = new Path(glusterMount); - this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority()); - - this.xattr = new GlusterFSXattr(); - - InetAddress addr = InetAddress.getLocalHost(); - this.hostname = addr.getHostName(); - - setConf(conf); - - } catch (Exception e) { - e.printStackTrace(); - System.out.println("Unable to initialize GlusterFS"); - System.exit(-1); - } - } - - @Deprecated - public String getName () { - return getUri().toString(); - } - - public Path getWorkingDirectory () { - return this.workingDir; - } - - public Path getHomeDirectory () { - return this.workingDir; - } - - public Path makeAbsolute (Path path) { - String pth = path.toUri().getPath(); - if (pth.startsWith(workingDir.toUri().getPath())) { - return path; - } - - return new Path(workingDir + "/" + pth); - } - - public void setWorkingDirectory (Path dir) { - this.workingDir = makeAbsolute(dir); - } - - public boolean exists (Path path) throws IOException { - Path absolute = makeAbsolute(path); - File f = new File(absolute.toUri().getPath()); - - return f.exists(); - } - - public boolean mkdirs (Path path, FsPermission permission - ) throws IOException { - boolean created = false; - Path absolute = makeAbsolute(path); - File f = new File(absolute.toUri().getPath()); - - if (f.exists()) { - System.out.println("Directory " + f.getPath() + " already exist"); - return true; - } - - return f.mkdirs(); - } - - @Deprecated - public boolean isDirectory (Path path) throws IOException { - Path absolute = makeAbsolute(path); - File f = new File(absolute.toUri().getPath()); - - return f.isDirectory(); - } - - public boolean isFile (Path path) throws IOException { - return !isDirectory(path); - } - - public Path[] listPaths (Path path) throws IOException { - Path absolute = makeAbsolute(path); - File f = new File (absolute.toUri().getPath()); - String relPath = path.toUri().getPath(); - String[] fileList = null; - Path[] filePath = null; - int fileCnt = 0; - - fileList = f.list(); - - filePath = new Path[fileList.length]; - - for (; fileCnt < fileList.length; fileCnt++) { - filePath[fileCnt] = new Path(relPath + "/" + fileList[fileCnt]); - } - - return filePath; - } - - public FileStatus[] listStatus (Path path) throws IOException { - int fileCnt = 0; - Path absolute = makeAbsolute(path); - String relpath = path.toUri().getPath(); - String[] strFileList = null; - FileStatus[] fileStatus = null; - File f = new File(absolute.toUri().getPath()); - - if (!f.exists()) { - return null; - } - - if (f.isFile()) - return new FileStatus[] { - getFileStatus(path) - }; - - if (relpath.charAt(relpath.length() - 1) != '/') - relpath += "/"; - - strFileList = f.list(); - - fileStatus = new FileStatus[strFileList.length]; - - for (; fileCnt < strFileList.length; fileCnt++) { - fileStatus[fileCnt] = getFileStatusFromFileString(relpath + strFileList[fileCnt]); - } - - return fileStatus; - } - - public FileStatus getFileStatusFromFileString (String path) - throws IOException { - Path nPath = new Path(path); - return getFileStatus(nPath); - } - - public FileStatus getFileStatus (Path path) throws IOException { - Path absolute = makeAbsolute(path); - File f = new File(absolute.toUri().getPath()); - - if (!f.exists ()) - throw new FileNotFoundException("File " + f.getPath() + " does not exist."); - - if (f.isDirectory ()) - return new FileStatus(0, true, 1, 0, f.lastModified(), path.makeQualified(this)); - else - return new FileStatus(f.length(), false, 0, getDefaultBlockSize(), - f.lastModified(), path.makeQualified(this)); - - } - - /* - * creates a new file in glusterfs namespace. internally the file - * descriptor is an instance of OutputStream class. - */ - public FSDataOutputStream create (Path path, FsPermission permission, - boolean overwrite, int bufferSize, - short replication, long blockSize, - Progressable progress) - throws IOException { - Path absolute = makeAbsolute(path); - Path parent = null; - File f = null; - File fParent = null; - FSDataOutputStream glusterFileStream = null; - - f = new File(absolute.toUri().getPath()); - - if (f.exists ()) { - if (overwrite) - f.delete (); - else - throw new IOException(f.getPath() + " already exist"); - } - - parent = path.getParent(); - fParent = new File ((makeAbsolute(parent)).toUri().getPath()); - if ((parent != null) && (fParent != null) && (!fParent.exists())) - if (!fParent.mkdirs()) - throw new IOException("cannot create parent directory: " + fParent.getPath()); - - glusterFileStream = new FSDataOutputStream(new GlusterFUSEOutputStream - (f.getPath(), false)); - - return glusterFileStream; - } - - /* - * open the file in read mode (internally the file descriptor is an - * instance of InputStream class). - * - * if quick read mode is set then read the file by by-passing FUSE - * if we are on same slave where the file exist - */ - public FSDataInputStream open (Path path) throws IOException { - Path absolute = makeAbsolute(path); - File f = new File(absolute.toUri().getPath()); - FSDataInputStream glusterFileStream = null; - TreeMap hnts = null; - - if (!f.exists()) - throw new IOException("File " + f.getPath() + " does not exist."); - - if (quickSlaveIO) - hnts = xattr.quickIOPossible(f.getPath(), 0, f.length()); - - glusterFileStream = new FSDataInputStream(new GlusterFUSEInputStream(f, hnts, hostname)); - return glusterFileStream; - } - - public FSDataInputStream open (Path path, int bufferSize) throws IOException { - return open(path); - } - - public FSDataOutputStream append (Path f, int bufferSize, Progressable progress) - throws IOException { - throw new IOException ("append not supported (as yet)."); - } - - public boolean rename (Path src, Path dst) throws IOException { - Path absoluteSrc = makeAbsolute(src); - Path absoluteDst = makeAbsolute(dst); - - File fSrc = new File(absoluteSrc.toUri().getPath()); - File fDst = new File(absoluteDst.toUri().getPath()); - - if (fDst.isDirectory()) { - fDst = null; - String newPath = absoluteDst.toUri().getPath() + "/" + fSrc.getName(); - fDst = new File(newPath); - } - return fSrc.renameTo(fDst); - } - - @Deprecated - public boolean delete (Path path) throws IOException { - return delete(path, true); - } - - public boolean delete (Path path, boolean recursive) throws IOException { - Path absolute = makeAbsolute(path); - File f = new File(absolute.toUri().getPath()); - - if (f.isFile()) - return f.delete(); - - FileStatus[] dirEntries = listStatus(absolute); - if ((!recursive) && (dirEntries != null) && (dirEntries.length != 0)) - throw new IOException ("Directory " + path.toString() + " is not empty"); - - if (dirEntries != null) - for (int i = 0; i < dirEntries.length; i++) - delete(new Path(absolute, dirEntries[i].getPath()), recursive); - - return f.delete(); - } - - @Deprecated - public long getLength (Path path) throws IOException { - Path absolute = makeAbsolute(path); - File f = new File(absolute.toUri().getPath()); - - if (!f.exists()) - throw new IOException(f.getPath() + " does not exist."); - - return f.length(); - } - - @Deprecated - public short getReplication (Path path) throws IOException { - Path absolute = makeAbsolute(path); - File f = new File(absolute.toUri().getPath()); - - if (!f.exists()) - throw new IOException(f.getPath() + " does not exist."); - - return xattr.getReplication(f.getPath()); - } - - public short getDefaultReplication (Path path) throws IOException { - return getReplication(path); - } - - public boolean setReplication (Path path, short replication) - throws IOException { - return true; - } - - public long getBlockSize (Path path) throws IOException { - long blkSz; - Path absolute = makeAbsolute(path); - File f = new File(absolute.toUri().getPath()); - - blkSz = xattr.getBlockSize(f.getPath()); - if (blkSz == 0) - blkSz = getLength(path); - - return blkSz; - } - - public long getDefaultBlockSize () { - return 1 << 26; /* default's from hdfs, kfs */ - } - - @Deprecated - public void lock (Path path, boolean shared) throws IOException { - } - - @Deprecated - public void release (Path path) throws IOException { - } - - public BlockLocation[] getFileBlockLocations (FileStatus file, long start, long len) - throws IOException { - - Path absolute = makeAbsolute(file.getPath()); - File f = new File(absolute.toUri().getPath()); - BlockLocation[] result = null; - - if (file == null) - return null; - - result = xattr.getPathInfo(f.getPath(), start, len); - if (result == null) { - System.out.println("Problem getting destination host for file " - + f.getPath()); - return null; - } - - return result; - } - - // getFileBlockLocations (FileStatus, long, long) is called by hadoop - public BlockLocation[] getFileBlockLocations (Path p, long start, long len) - throws IOException { - return null; - } - - public void copyFromLocalFile (boolean delSrc, Path src, Path dst) - throws IOException { - FileUtil.copy(glusterFs, src, this, dst, delSrc, getConf()); - } - - public void copyToLocalFile (boolean delSrc, Path src, Path dst) - throws IOException { - FileUtil.copy(this, src, glusterFs, dst, delSrc, getConf()); - } - - public Path startLocalOutput (Path fsOutputFile, Path tmpLocalFile) - throws IOException { - return tmpLocalFile; - } - - public void completeLocalOutput (Path fsOutputFile, Path tmpLocalFile) - throws IOException { - moveFromLocalFile(tmpLocalFile, fsOutputFile); - } -} diff --git a/hdfs/0.20.2/src/test/java/org/apache/hadoop/fs/glusterfs/AppTest.java b/hdfs/0.20.2/src/test/java/org/apache/hadoop/fs/glusterfs/AppTest.java deleted file mode 100644 index 21e188c5..00000000 --- a/hdfs/0.20.2/src/test/java/org/apache/hadoop/fs/glusterfs/AppTest.java +++ /dev/null @@ -1,38 +0,0 @@ -package org.apache.hadoop.fs.glusterfs; - -import junit.framework.Test; -import junit.framework.TestCase; -import junit.framework.TestSuite; - -/** - * Unit test for simple App. - */ -public class AppTest - extends TestCase -{ - /** - * Create the test case - * - * @param testName name of the test case - */ - public AppTest( String testName ) - { - super( testName ); - } - - /** - * @return the suite of tests being tested - */ - public static Test suite() - { - return new TestSuite( AppTest.class ); - } - - /** - * Rigourous Test :-) - */ - public void testApp() - { - assertTrue( true ); - } -} diff --git a/hdfs/0.20.2/tools/build-deploy-jar.py b/hdfs/0.20.2/tools/build-deploy-jar.py deleted file mode 100644 index 450e08fb..00000000 --- a/hdfs/0.20.2/tools/build-deploy-jar.py +++ /dev/null @@ -1,212 +0,0 @@ -#!/usr/bin/python - -## - # - # Copyright (c) 2011 Gluster, Inc. - # This file is part of GlusterFS. - # - # Licensed 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. - # - ## - -import getopt -import glob -import sys, os -import shutil -import subprocess, shlex - -def usage(): - print "usage: python build-deploy-jar.py [-b/--build] -d/--dir [-c/--core] [-m/--mapred] [-h/--henv]" - -def addSlash(s): - if not (s[-1] == '/'): - s = s + '/' - - return s - -def whereis(program): - abspath = None - for path in (os.environ.get('PATH', '')).split(':'): - abspath = os.path.join(path, program) - if os.path.exists(abspath) and not os.path.isdir(abspath): - return abspath - - return None - -def getLatestJar(targetdir): - latestJar = None - glusterfsJar = glob.glob(targetdir + "*.jar") - if len(glusterfsJar) == 0: - print "No GlusterFS jar file found in %s ... exiting" % (targetdir) - return None - - # pick up the latest jar file - just in case ... - stat = latestJar = None - ctime = 0 - - for jar in glusterfsJar: - stat = os.stat(jar) - if stat.st_ctime > ctime: - latestJar = jar - ctime = stat.st_ctime - - return latestJar - -# build the glusterfs hadoop plugin using maven -def build_jar(): - location = whereis('mvn') - - if location == None: - print "Cannot find maven to build glusterfs hadoop jar" - print "please install maven or if it's already installed then fix your PATH environ" - return None - - # do a clean packaging - targetdir = "./target/" - if os.path.exists(targetdir) and os.path.isdir(targetdir): - print "Cleaning up directories ... [ " + targetdir + " ]" - shutil.rmtree(targetdir) - - print "Building glusterfs jar ..." - process = subprocess.Popen(['package'], shell=True, - executable=location, stdout=subprocess.PIPE, stderr=subprocess.PIPE) - - try: - (pout, perr) = process.communicate() - except: - process.wait() - if not process.returncode == 0: - print "Building glusterfs jar failed" - return None - - latestJar = getLatestJar(targetdir) - return latestJar - -def rcopy(f, host, libdir): - print " * doing remote copy to host %s" % (host) - scpCmd = "scp %s %s:%s" % (f, host, libdir) - - os.system(scpCmd); - -def deployInSlave(f, confdir, libdir, cc, cm, he): - slavefile = confdir + "slaves" - - ccFile = confdir + "core-site.xml" - cmFile = confdir + "mapred-site.xml" - heFile = confdir + "hadoop-env.sh" - - sf = open(slavefile, 'r') - for host in sf: - host = host.rstrip('\n') - print " >>> Deploying %s on %s ..." % (os.path.basename(f), host) - rcopy(f, host, libdir) - - if cc: - print " >>> Deploying [%s] on %s ..." % (os.path.basename(ccFile), host) - rcopy(ccFile, host, confdir) - - if cm: - print " >>> Deploying [%s] on %s ..." % (os.path.basename(cmFile), host) - rcopy(cmFile, host, confdir) - - if he: - print " >>> Deploying [%s] on %s ..." % (os.path.basename(heFile), host) - rcopy(heFile, host, confdir); - - print "<<< Done\n" - - sf.close() - -def deployInMaster(f, confdir, libdir): - import socket - masterfile = confdir + "masters" - - mf = open(masterfile, 'r') - for host in mf: - host = host.rstrip('\n') - print " >>> Deploying %s on %s ..." % (os.path.basename(f), host) - h = host - try: - socket.inet_aton(host) - h = socket.getfqdn(host) - except socket.error: - # host is not a ip adddress - pass - - if h == socket.gethostname() or h == 'localhost': - # local cp - print " * doing local copy" - shutil.copy(f, libdir) - else: - # scp the file - rcopy(f, h, libdir) - - print "<<< Done\n" - - mf.close() - -if __name__ == '__main__': - opt = args = [] - try: - opt, args = getopt.getopt(sys.argv[1:], "bd:cmh", ["build", "dir=", "core", "mapred", "henv"]); - except getopt.GetoptError, err: - print str(err) - usage() - sys.exit(1) - - needbuild = hadoop_dir = copyCore = copyMapred = copyHadoopEnv = None - - for k, v in opt: - if k in ("-b", "--build"): - needbuild = True - elif k in ("-d", "--dir"): - hadoop_dir = v - elif k in ("-c", "--core"): - copyCore = True - elif k in ("-m", "--mapred"): - copyMapred = True - elif k in ("-h", "--henv"): - copyHadoopEnv = True - else: - assert False, "unhandled option" - - assert not hadoop_dir == None, "hadoop directory missing" - - if needbuild: - jar = build_jar() - if jar == None: - sys.exit(1) - else: - jar = getLatestJar('./target/') - if jar == None: - print "Maybe you want to build it ? -b option" - sys.exit(1) - - print "" - print "*** Deploying %s *** " % (jar) - - # copy jar to local hadoop distribution (master) - hadoop_home = addSlash(hadoop_dir) - if not (os.path.exists(hadoop_home) and os.path.isdir(hadoop_home)): - print "path " + hadoop_home + " does not exist or is not adiretory"; - sys.exit(1); - - hadoop_conf = hadoop_home + "conf/" - hadoop_lib = hadoop_home + "lib/" - - print " >>> Scanning hadoop master file for host(s) to deploy" - deployInMaster(jar, hadoop_conf, hadoop_lib) - - print "" - print " >>> Scanning hadoop slave file for host(s) to deploy" - deployInSlave(jar, hadoop_conf, hadoop_lib, copyCore, copyMapred, copyHadoopEnv) diff --git a/hdfs/COPYING b/hdfs/COPYING deleted file mode 100644 index d6456956..00000000 --- a/hdfs/COPYING +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/hdfs/README b/hdfs/README deleted file mode 100644 index 3026f11c..00000000 --- a/hdfs/README +++ /dev/null @@ -1,182 +0,0 @@ -GlusterFS Hadoop Plugin -======================= - -INTRODUCTION ------------- - -This document describes how to use GlusterFS (http://www.gluster.org/) as a backing store with Hadoop. - - -REQUIREMENTS ------------- - -* Supported OS is GNU/Linux -* GlusterFS and Hadoop installed on all machines in the cluster -* Java Runtime Environment (JRE) -* Maven (needed if you are building the plugin from source) -* JDK (needed if you are building the plugin from source) - -NOTE: Plugin relies on two *nix command line utilities to function properly. They are: - -* mount: Used to mount GlusterFS volumes. -* getfattr: Used to fetch Extended Attributes of a file - -Make sure they are installed on all hosts in the cluster and their locations are in $PATH -environment variable. - - -INSTALLATION ------------- - -** NOTE: Example below is for Hadoop version 0.20.2 ($GLUSTER_HOME/hdfs/0.20.2) ** - -* Building the plugin from source [Maven (http://maven.apache.org/) and JDK is required to build the plugin] - - Change to glusterfs-hadoop directory in the GlusterFS source tree and build the plugin. - - # cd $GLUSTER_HOME/hdfs/0.20.2 - # mvn package - - On a successful build the plugin will be present in the `target` directory. - (NOTE: version number will be a part of the plugin) - - # ls target/ - classes glusterfs-0.20.2-0.1.jar maven-archiver surefire-reports test-classes - ^^^^^^^^^^^^^^^^^^ - - Copy the plugin to lib/ directory in your $HADOOP_HOME dir. - - # cp target/glusterfs-0.20.2-0.1.jar $HADOOP_HOME/lib - - Copy the sample configuration file that ships with this source (conf/core-site.xml) to conf - directory in your $HADOOP_HOME dir. - - # cp conf/core-site.xml $HADOOP_HOME/conf - -* Installing the plugin from RPM - - See the plugin documentation for installing from RPM. - - -CLUSTER INSTALLATION --------------------- - - In case it is tedious to do the above steps(s) on all hosts in the cluster; use the build-and-deploy.py script to - build the plugin in one place and deploy it (along with the configuration file on all other hosts). - - This should be run on the host which is that hadoop master [Job Tracker]. - -* STEPS (You would have done Step 1 and 2 anyway while deploying Hadoop) - - 1. Edit conf/slaves file in your hadoop distribution; one line for each slave. - 2. Setup password-less ssh b/w hadoop master and slave(s). - 3. Edit conf/core-site.xml with all glusterfs related configurations (see CONFIGURATION) - 4. Run the following - # cd $GLUSTER_HOME/hdfs/0.20.2/tools - # python ./build-and-deploy.py -b -d /path/to/hadoop/home -c - - This will build the plugin and copy it (and the config file) to all slaves (mentioned in $HADOOP_HOME/conf/slaves). - - Script options: - -b : build the plugin - -d : location of hadoop directory - -c : deploy core-site.xml - -m : deploy mapred-site.xml - -h : deploy hadoop-env.sh - - -CONFIGURATION -------------- - - All plugin configuration is done in a single XML file (core-site.xml) with tags in each - block. - - Brief explanation of the tunables and the values they accept (change them where-ever needed) are mentioned below - - name: fs.glusterfs.impl - value: org.apache.hadoop.fs.glusterfs.GlusterFileSystem - - The default FileSystem API to use (there is little reason to modify this). - - name: fs.default.name - value: glusterfs://server:port - - The default name that hadoop uses to represent file as a URI (typically a server:port tuple). Use any host - in the cluster as the server and any port number. This option has to be in server:port format for hadoop - to create file URI; but is not used by plugin. - - name: fs.glusterfs.volname - value: volume-dist-rep - - The volume to mount. - - - name: fs.glusterfs.mount - value: /mnt/glusterfs - - This is the directory that the plugin will use to mount (FUSE mount) the volume. - - name: fs.glusterfs.server - value: 192.168.1.36, hackme.zugzug.org - - To mount a volume the plugin needs to know the hostname or the IP of a GlusterFS server in the cluster. - Mention it here. - - name: quick.slave.io - value: [On/Off], [Yes/No], [1/0] - - NOTE: This option is not tested as of now. - - This is a performance tunable option. Hadoop schedules jobs to hosts that contain the file data part. The job - then does I/O on the file (via FUSE in case of GlusterFS). When this option is set, the plugin will try to - do I/O directly from the backed filesystem (ext3, ext4 etc..) the file resides on. Hence read performance - will improve and job would run faster. - - -USAGE ------ - - Once configured, start Hadoop Map/Reduce daemons - - # cd $HADOOP_HOME - # ./bin/start-mapred.sh - - If the map/reduce job/task trackers are up, all I/O will be done to GlusterFS. - - -FOR HACKERS ------------ - -* Source Layout - -** version specific: hdfs/ ** -./src -./src/main -./src/main/java -./src/main/java/org -./src/main/java/org/apache -./src/main/java/org/apache/hadoop -./src/main/java/org/apache/hadoop/fs -./src/main/java/org/apache/hadoop/fs/glusterfs -./src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickClass.java -./src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSXattr.java <--- Fetch/Parse Extended Attributes of a file -./src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEInputStream.java <--- Input Stream (instantiated during open() calls; quick read from backed FS) -./src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFSBrickRepl.java -./src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFUSEOutputStream.java <--- Output Stream (instantiated during creat() calls) -./src/main/java/org/apache/hadoop/fs/glusterfs/GlusterFileSystem.java <--- Entry Point for the plugin (extends Hadoop FileSystem class) -./src/test -./src/test/java -./src/test/java/org -./src/test/java/org/apache -./src/test/java/org/apache/hadoop -./src/test/java/org/apache/hadoop/fs -./src/test/java/org/apache/hadoop/fs/glusterfs -./src/test/java/org/apache/hadoop/fs/glusterfs/AppTest.java <--- Your test cases go here (if any :-)) -./tools/build-deploy-jar.py <--- Build and Deployment Script -./conf -./conf/core-site.xml <--- Sample configuration file -./pom.xml <--- build XML file (used by maven) - -** toplevel: hdfs/ ** -./COPYING <--- License -./README <--- This file -- cgit