You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metamodel.apache.org by to...@apache.org on 2015/08/10 16:56:53 UTC

[1/4] metamodel git commit: Adds ability to read folders from FileResource and HdfsResource

Repository: metamodel
Updated Branches:
  refs/heads/master 93d9745cc -> 9424c312a


Adds ability to read folders from FileResource and HdfsResource

This will allow FileResource and HdfsResource to use a folder as a complete resource. It will skip any subfolder and only use files in the folder. Before opening files, they will be sorted them alphabetically by name (actually it uses the natural ordering of the File/FileStatus object, but that is the pathname/url sorted alphabetically).


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

Branch: refs/heads/master
Commit: f11f9607055ce931601cf417d9de009461efddcf
Parents: 34d9d58
Author: Dennis Du Krøger <de...@humaninference.com>
Authored: Fri Aug 7 09:45:15 2015 +0200
Committer: Dennis Du Krøger <de...@humaninference.com>
Committed: Fri Aug 7 09:45:15 2015 +0200

----------------------------------------------------------------------
 .../util/AbstractDirectoryInputStream.java      |  97 +++++++++++
 .../org/apache/metamodel/util/FileResource.java |  30 ++++
 .../apache/metamodel/util/FileResourceTest.java |  56 +++++++
 .../org/apache/metamodel/util/HdfsResource.java | 168 +++++++++++++------
 .../util/HdfsResourceIntegrationTest.java       | 119 ++++++++++---
 5 files changed, 390 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/f11f9607/core/src/main/java/org/apache/metamodel/util/AbstractDirectoryInputStream.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/metamodel/util/AbstractDirectoryInputStream.java b/core/src/main/java/org/apache/metamodel/util/AbstractDirectoryInputStream.java
new file mode 100644
index 0000000..5b34527
--- /dev/null
+++ b/core/src/main/java/org/apache/metamodel/util/AbstractDirectoryInputStream.java
@@ -0,0 +1,97 @@
+package org.apache.metamodel.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+public abstract class AbstractDirectoryInputStream<T> extends InputStream {
+    protected T[] _files;
+    private int _currentFileIndex = -1;
+    private InputStream _currentInputStream;
+
+
+    @Override
+    public int read(final byte[] b, final int off, final int len) throws IOException {
+        if (_currentInputStream != null) {
+            final int byteCount = _currentInputStream.read(b, off, len);
+            if (byteCount > 0) {
+                return byteCount;
+            }
+        }
+
+        if (!openNextFile()){
+            return -1; // No more files.
+        }
+
+        return read(b, off, len);
+    }
+
+    @Override
+    public int read(final byte[] b) throws IOException {
+        return read(b, 0, b.length);
+    }
+
+    @Override
+    public int read() throws IOException {
+        final byte[] b = new byte[1];
+        int count = read(b, 0 , 1);
+        if(count < 0){
+            return -1;
+        }
+        return (int) b[0];
+    }
+
+    @Override
+    public int available() throws IOException {
+        if(_currentInputStream != null){
+            return _currentInputStream.available();
+        } else {
+            return 0;
+        }
+    }
+
+    private boolean openNextFile() throws IOException {
+        if(_currentInputStream != null){
+            FileHelper.safeClose(_currentInputStream);
+            _currentInputStream = null;
+        }
+        _currentFileIndex++;
+        if (_currentFileIndex >= _files.length) {
+            return false;
+        }
+
+        _currentInputStream = openStream(_currentFileIndex);
+        return true;
+    }
+
+    abstract InputStream openStream(int index) throws IOException;
+
+    @Override
+    public boolean markSupported() {
+        return false;
+    }
+
+    @Override
+    public void close() throws IOException {
+        if(_currentInputStream != null){
+            FileHelper.safeClose(_currentInputStream);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/metamodel/blob/f11f9607/core/src/main/java/org/apache/metamodel/util/FileResource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/metamodel/util/FileResource.java b/core/src/main/java/org/apache/metamodel/util/FileResource.java
index 94b7293..20dea50 100644
--- a/core/src/main/java/org/apache/metamodel/util/FileResource.java
+++ b/core/src/main/java/org/apache/metamodel/util/FileResource.java
@@ -19,16 +19,43 @@
 package org.apache.metamodel.util;
 
 import java.io.File;
+import java.io.FileFilter;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Comparator;
 
 /**
  * {@link File} based {@link Resource} implementation.
  */
 public class FileResource implements Resource, Serializable {
 
+    private class DirectoryInputStream extends AbstractDirectoryInputStream<File> {
+
+        public DirectoryInputStream(){
+            final File[] unsortedFiles = _file.listFiles(new FileFilter() {
+                @Override
+                public boolean accept(final File pathname) {
+                    return pathname.isFile();
+                }
+            });
+
+            if(unsortedFiles == null){
+                _files = new File[0];
+            } else {
+                Arrays.sort(unsortedFiles);
+                _files = unsortedFiles;
+            }
+        }
+
+        @Override
+        InputStream openStream(final int index) throws IOException {
+            return FileHelper.getInputStream(_files[index]);
+        }
+    }
+
     private static final long serialVersionUID = 1L;
     private final File _file;
 
@@ -142,6 +169,9 @@ public class FileResource implements Resource, Serializable {
 
     @Override
     public InputStream read() throws ResourceException {
+        if(_file.isDirectory()){
+            return new DirectoryInputStream();
+        }
         final InputStream in = FileHelper.getInputStream(_file);
         return in;
     }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/f11f9607/core/src/test/java/org/apache/metamodel/util/FileResourceTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/metamodel/util/FileResourceTest.java b/core/src/test/java/org/apache/metamodel/util/FileResourceTest.java
new file mode 100644
index 0000000..910ae5b
--- /dev/null
+++ b/core/src/test/java/org/apache/metamodel/util/FileResourceTest.java
@@ -0,0 +1,56 @@
+package org.apache.metamodel.util;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class FileResourceTest {
+
+    @Test
+    public void testReadDirectory() throws Exception {
+        final String contentString = "fun and games with Apache MetaModel and Hadoop is what we do";
+        final String[] contents = new String[] { "fun ", "and ", "games ", "with ", "Apache ", "MetaModel ", "and ", "Hadoop ", "is ", "what ", "we ", "do" };
+
+        Path path = Files.createTempDirectory("test");
+
+        // Reverse both filename and contents to make sure it is the name and not the creation order that is sorted on.
+        int i = contents.length;
+        Collections.reverse(Arrays.asList(contents));
+        for(final String contentPart : contents){
+            final FileResource partResource = new FileResource(path + "/part-" + String.format("%02d", i--));
+            partResource.write(new Action<OutputStream>() {
+                @Override
+                public void run(OutputStream out) throws Exception {
+                    out.write(contentPart.getBytes());
+                }
+            });
+        }
+
+
+        final FileResource res1 = new FileResource(path.toFile());
+
+        final String str1 = res1.read(new Func<InputStream, String>() {
+            @Override
+            public String eval(InputStream in) {
+                return FileHelper.readInputStreamAsString(in, "UTF8");
+            }
+        });
+
+        Assert.assertEquals(contentString, str1);
+
+        final String str2 = res1.read(new Func<InputStream, String>() {
+            @Override
+            public String eval(InputStream in) {
+                return FileHelper.readInputStreamAsString(in, "UTF8");
+            }
+        });
+        Assert.assertEquals(str1, str2);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/metamodel/blob/f11f9607/hadoop/src/main/java/org/apache/metamodel/util/HdfsResource.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/metamodel/util/HdfsResource.java b/hadoop/src/main/java/org/apache/metamodel/util/HdfsResource.java
index f66b8c9..eab9597 100644
--- a/hadoop/src/main/java/org/apache/metamodel/util/HdfsResource.java
+++ b/hadoop/src/main/java/org/apache/metamodel/util/HdfsResource.java
@@ -18,18 +18,22 @@
  */
 package org.apache.metamodel.util;
 
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
 import org.apache.metamodel.MetaModelException;
 
 /**
@@ -37,6 +41,104 @@ import org.apache.metamodel.MetaModelException;
  * distributed file system.
  */
 public class HdfsResource implements Resource, Serializable {
+    private static class HdfsFileInputStream extends InputStream {
+        private final InputStream _in;
+        private final FileSystem _fs;
+
+        public HdfsFileInputStream(final InputStream in, final FileSystem fs) {
+            _in = in;
+            _fs = fs;
+        }
+
+        @Override
+        public int read() throws IOException {
+            return _in.read();
+        }
+
+        @Override
+        public int read(byte[] b, int off, int len) throws IOException {
+            return _in.read(b, off, len);
+        }
+
+        @Override
+        public int read(byte[] b) throws IOException {
+            return _in.read(b);
+        }
+
+        @Override
+        public boolean markSupported() {
+            return _in.markSupported();
+        }
+
+        @Override
+        public synchronized void mark(int readLimit) {
+            _in.mark(readLimit);
+        }
+
+        @Override
+        public int available() throws IOException {
+            return _in.available();
+        }
+
+        @Override
+        public synchronized void reset() throws IOException {
+            _in.reset();
+        }
+
+        @Override
+        public long skip(long n) throws IOException {
+            return _in.skip(n);
+        }
+
+        @Override
+        public void close() throws IOException {
+            super.close();
+            // need to close 'fs' when input stream is closed
+            FileHelper.safeClose(_fs);
+        }
+    }
+
+    private class HdfsDirectoryInputStream extends AbstractDirectoryInputStream<FileStatus> {
+        private final Path _hadoopPath;
+        private final FileSystem _fs;
+
+        public HdfsDirectoryInputStream(final Path hadoopPath,
+                final FileSystem fs) {
+            _hadoopPath = hadoopPath;
+            _fs = fs;
+            FileStatus[] fileStatuses;
+            try {
+                fileStatuses = _fs.listStatus(_hadoopPath, new PathFilter() {
+                    @Override
+                    public boolean accept(final Path path) {
+                        try {
+                            return _fs.isFile(path);
+                        } catch (IOException e) {
+                            return false;
+                        }
+                    }
+                });
+                // Natural ordering is the URL
+                Arrays.sort(fileStatuses);
+
+            } catch (IOException e) {
+                fileStatuses = new FileStatus[0];
+            }
+            _files = fileStatuses;
+        }
+
+        @Override
+        InputStream openStream(final int index) throws IOException {
+            final Path nextPath = _files[index].getPath();
+            return _fs.open(nextPath);
+        }
+
+        @Override
+        public void close() throws IOException {
+            super.close();
+            FileHelper.safeClose(_fs);
+        }
+    }
 
     private static final long serialVersionUID = 1L;
 
@@ -49,7 +151,7 @@ public class HdfsResource implements Resource, Serializable {
 
     /**
      * Creates a {@link HdfsResource}
-     * 
+     *
      * @param url
      *            a URL of the form: hdfs://hostname:port/path/to/file
      */
@@ -69,7 +171,7 @@ public class HdfsResource implements Resource, Serializable {
 
     /**
      * Creates a {@link HdfsResource}
-     * 
+     *
      * @param hostname
      *            the HDFS (namenode) hostname
      * @param port
@@ -190,69 +292,27 @@ public class HdfsResource implements Resource, Serializable {
         final FileSystem fs = getHadoopFileSystem();
         final InputStream in;
         try {
-            in = fs.open(getHadoopPath());
+            final Path hadoopPath = getHadoopPath();
+            // return a wrapper InputStream which manages the 'fs' closeable
+            if (fs.isFile(hadoopPath)) {
+                in = fs.open(hadoopPath);
+                return new HdfsFileInputStream(in, fs);
+            } else {
+                return new HdfsDirectoryInputStream(hadoopPath, fs);
+            }
         } catch (Exception e) {
             // we can close 'fs' in case of an exception
             FileHelper.safeClose(fs);
             throw wrapException(e);
         }
 
-        // return a wrappper InputStream which manages the 'fs' closeable
-        return new InputStream() {
-            @Override
-            public int read() throws IOException {
-                return in.read();
-            }
-
-            @Override
-            public int read(byte[] b, int off, int len) throws IOException {
-                return in.read(b, off, len);
-            }
-
-            @Override
-            public int read(byte[] b) throws IOException {
-                return in.read(b);
-            }
-
-            @Override
-            public boolean markSupported() {
-                return in.markSupported();
-            }
-
-            @Override
-            public synchronized void mark(int readlimit) {
-                in.mark(readlimit);
-            }
-
-            @Override
-            public int available() throws IOException {
-                return in.available();
-            }
-
-            @Override
-            public synchronized void reset() throws IOException {
-                in.reset();
-            }
-
-            @Override
-            public long skip(long n) throws IOException {
-                return in.skip(n);
-            }
-
-            @Override
-            public void close() throws IOException {
-                super.close();
-                // need to close 'fs' when input stream is closed
-                FileHelper.safeClose(fs);
-            }
-        };
     }
 
     @Override
     public void read(Action<InputStream> readCallback) throws ResourceException {
         final FileSystem fs = getHadoopFileSystem();
         try {
-            final InputStream in = fs.open(getHadoopPath());
+            final InputStream in = read();
             try {
                 readCallback.run(in);
             } finally {
@@ -269,7 +329,7 @@ public class HdfsResource implements Resource, Serializable {
     public <E> E read(Func<InputStream, E> readCallback) throws ResourceException {
         final FileSystem fs = getHadoopFileSystem();
         try {
-            final InputStream in = fs.open(getHadoopPath());
+            final InputStream in = read();
             try {
                 return readCallback.eval(in);
             } finally {

http://git-wip-us.apache.org/repos/asf/metamodel/blob/f11f9607/hadoop/src/test/java/org/apache/metamodel/util/HdfsResourceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/metamodel/util/HdfsResourceIntegrationTest.java b/hadoop/src/test/java/org/apache/metamodel/util/HdfsResourceIntegrationTest.java
index 3f37379..03a5610 100644
--- a/hadoop/src/test/java/org/apache/metamodel/util/HdfsResourceIntegrationTest.java
+++ b/hadoop/src/test/java/org/apache/metamodel/util/HdfsResourceIntegrationTest.java
@@ -22,44 +22,50 @@ import java.io.File;
 import java.io.FileReader;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Stopwatch;
 
-import junit.framework.TestCase;
 
-public class HdfsResourceIntegrationTest extends TestCase {
+public class HdfsResourceIntegrationTest {
     
     private static final Logger logger = LoggerFactory.getLogger(HdfsResourceIntegrationTest.class);
 
-    private boolean _configured;
-    private Properties _properties;
     private String _filePath;
     private String _hostname;
     private int _port;
 
-    @Override
-    protected final void setUp() throws Exception {
-        super.setUp();
-
-        _properties = new Properties();
+    @Before
+    public void setUp() throws Exception {
         final File file = new File(getPropertyFilePath());
+        final boolean configured;
         if (file.exists()) {
-            _properties.load(new FileReader(file));
-            _filePath = _properties.getProperty("hadoop.hdfs.file.path");
-            _hostname = _properties.getProperty("hadoop.hdfs.hostname");
-            final String portString = _properties.getProperty("hadoop.hdfs.port");
-            _configured = _filePath != null && _hostname != null && portString != null;
-            if (_configured) {
+            final Properties properties = new Properties();
+            properties.load(new FileReader(file));
+            _filePath = properties.getProperty("hadoop.hdfs.file.path");
+            _hostname = properties.getProperty("hadoop.hdfs.hostname");
+            final String portString = properties.getProperty("hadoop.hdfs.port");
+            configured = _filePath != null && _hostname != null && portString != null;
+            if (configured) {
                 _port = Integer.parseInt(portString);
             }
         } else {
-            _configured = false;
+            configured = false;
         }
+        Assume.assumeTrue(configured);
     }
 
     private String getPropertyFilePath() {
@@ -67,13 +73,73 @@ public class HdfsResourceIntegrationTest extends TestCase {
         return userHome + "/metamodel-integrationtest-configuration.properties";
     }
 
-    public void testReadOnRealHdfsInstall() throws Exception {
-        if (!_configured) {
-            System.err.println("!!! WARN !!! Hadoop HDFS integration test ignored\r\n"
-                    + "Please configure Hadoop HDFS test-properties (" + getPropertyFilePath()
-                    + "), to run integration tests");
-            return;
+    @Test
+    public void testReadDirectory() throws Exception {
+        final String contentString = "fun and games with Apache MetaModel and Hadoop is what we do";
+        final String[] contents = new String[] { "fun ", "and ", "games ", "with ", "Apache ", "MetaModel ", "and ", "Hadoop ", "is ", "what ", "we ", "do" };
+
+        final Configuration conf = new Configuration();
+        conf.set("fs.defaultFS", "hdfs://" + _hostname + ":" + _port);
+        final FileSystem fileSystem = FileSystem.get(conf);
+        final Path path = new Path(_filePath);
+        final boolean exists = fileSystem.exists(path);
+
+        if(exists){
+            fileSystem.delete(path, true);
         }
+
+        fileSystem.mkdirs(path);
+
+
+        // Reverse both filename and contents to make sure it is the name and not the creation order that is sorted on.
+        int i = contents.length;
+        Collections.reverse(Arrays.asList(contents));
+        for(final String contentPart : contents){
+            final HdfsResource partResource = new HdfsResource(_hostname, _port, _filePath + "/part-" + String.format("%02d", i--));
+            partResource.write(new Action<OutputStream>() {
+                @Override
+                public void run(OutputStream out) throws Exception {
+                    out.write(contentPart.getBytes());
+                }
+            });
+        }
+
+        final Stopwatch stopwatch = Stopwatch.createStarted();
+
+        final HdfsResource res1 = new HdfsResource(_hostname, _port, _filePath);
+
+        logger.info(stopwatch.elapsed(TimeUnit.MILLISECONDS) + " - start");
+
+        final String str1 = res1.read(new Func<InputStream, String>() {
+            @Override
+            public String eval(InputStream in) {
+                return FileHelper.readInputStreamAsString(in, "UTF8");
+            }
+        });
+
+        Assert.assertEquals(contentString, str1);
+        logger.info(stopwatch.elapsed(TimeUnit.MILLISECONDS) + " - read1");
+
+        final String str2 = res1.read(new Func<InputStream, String>() {
+            @Override
+            public String eval(InputStream in) {
+                return FileHelper.readInputStreamAsString(in, "UTF8");
+            }
+        });
+        Assert.assertEquals(str1, str2);
+        logger.info(stopwatch.elapsed(TimeUnit.MILLISECONDS) + " - read2");
+
+        res1.getHadoopFileSystem().delete(res1.getHadoopPath(), true);
+        logger.info(stopwatch.elapsed(TimeUnit.MILLISECONDS) + " - deleted");
+
+        Assert.assertFalse(res1.isExists());
+
+        logger.info(stopwatch.elapsed(TimeUnit.MILLISECONDS) + " - done");
+        stopwatch.stop();
+    }
+
+    @Test
+    public void testReadOnRealHdfsInstall() throws Exception {
         final String contentString = "fun and games with Apache MetaModel and Hadoop is what we do";
 
         final Stopwatch stopwatch = Stopwatch.createStarted();
@@ -89,7 +155,7 @@ public class HdfsResourceIntegrationTest extends TestCase {
 
         logger.info(stopwatch.elapsed(TimeUnit.MILLISECONDS) + " - written");
 
-        assertTrue(res1.isExists());
+        Assert.assertTrue(res1.isExists());
 
         final String str1 = res1.read(new Func<InputStream, String>() {
             @Override
@@ -97,7 +163,7 @@ public class HdfsResourceIntegrationTest extends TestCase {
                 return FileHelper.readInputStreamAsString(in, "UTF8");
             }
         });
-        assertEquals(contentString, str1);
+        Assert.assertEquals(contentString, str1);
         logger.info(stopwatch.elapsed(TimeUnit.MILLISECONDS) + " - read1");
 
         final String str2 = res1.read(new Func<InputStream, String>() {
@@ -106,15 +172,16 @@ public class HdfsResourceIntegrationTest extends TestCase {
                 return FileHelper.readInputStreamAsString(in, "UTF8");
             }
         });
-        assertEquals(str1, str2);
+        Assert.assertEquals(str1, str2);
         logger.info(stopwatch.elapsed(TimeUnit.MILLISECONDS) + " - read2");
 
         res1.getHadoopFileSystem().delete(res1.getHadoopPath(), false);
         logger.info(stopwatch.elapsed(TimeUnit.MILLISECONDS) + " - deleted");
 
-        assertFalse(res1.isExists());
+        Assert.assertFalse(res1.isExists());
 
         logger.info(stopwatch.elapsed(TimeUnit.MILLISECONDS) + " - done");
         stopwatch.stop();
     }
+
 }


[4/4] metamodel git commit: Merge branch 'feature/METAMODEL-163-folder-resource' of github.com:LosD/metamodel

Posted by to...@apache.org.
Merge branch 'feature/METAMODEL-163-folder-resource' of github.com:LosD/metamodel


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

Branch: refs/heads/master
Commit: 9424c312a9383b1fb7244e60ce58bce8522e2cdd
Parents: 93d9745 85f955a
Author: tomaszguzialek <to...@humaninference.com>
Authored: Mon Aug 10 16:56:25 2015 +0200
Committer: tomaszguzialek <to...@humaninference.com>
Committed: Mon Aug 10 16:56:25 2015 +0200

----------------------------------------------------------------------
 .../util/AbstractDirectoryInputStream.java      |  97 +++++++++++
 .../org/apache/metamodel/util/FileResource.java |  31 +++-
 .../apache/metamodel/util/FileResourceTest.java |  73 ++++++++
 .../org/apache/metamodel/util/HdfsResource.java | 166 +++++++++++++------
 .../util/HdfsResourceIntegrationTest.java       | 119 ++++++++++---
 5 files changed, 405 insertions(+), 81 deletions(-)
----------------------------------------------------------------------



[3/4] metamodel git commit: Use JUnit 4 TemporaryFolder rule for temp folder.

Posted by to...@apache.org.
Use JUnit 4 TemporaryFolder rule for temp folder.

This removes the Java 7 dependency in the FileResourceTest by using
JUnit 4's TemporaryFolder rule. This also ensures that the folder
will always be deleted after running the test.

The license header in AbstractDirectoryInputStream has also been put
where it is supposed to be, and minor cleanups of imports and code
style has been done.


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

Branch: refs/heads/master
Commit: 85f955a58544cc1d9da34aacd317e7bfe5fde0dd
Parents: 66915e7
Author: Dennis Du Krøger <de...@humaninference.com>
Authored: Mon Aug 10 10:01:00 2015 +0200
Committer: Dennis Du Krøger <de...@humaninference.com>
Committed: Mon Aug 10 10:01:00 2015 +0200

----------------------------------------------------------------------
 .../util/AbstractDirectoryInputStream.java      | 22 ++++++++++----------
 .../org/apache/metamodel/util/FileResource.java |  9 ++++----
 .../apache/metamodel/util/FileResourceTest.java | 13 ++++++------
 .../org/apache/metamodel/util/HdfsResource.java |  2 --
 4 files changed, 21 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/85f955a5/core/src/main/java/org/apache/metamodel/util/AbstractDirectoryInputStream.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/metamodel/util/AbstractDirectoryInputStream.java b/core/src/main/java/org/apache/metamodel/util/AbstractDirectoryInputStream.java
index 5b34527..c5619ec 100644
--- a/core/src/main/java/org/apache/metamodel/util/AbstractDirectoryInputStream.java
+++ b/core/src/main/java/org/apache/metamodel/util/AbstractDirectoryInputStream.java
@@ -1,8 +1,3 @@
-package org.apache.metamodel.util;
-
-import java.io.IOException;
-import java.io.InputStream;
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -21,6 +16,11 @@ import java.io.InputStream;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.metamodel.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+
 public abstract class AbstractDirectoryInputStream<T> extends InputStream {
     protected T[] _files;
     private int _currentFileIndex = -1;
@@ -36,7 +36,7 @@ public abstract class AbstractDirectoryInputStream<T> extends InputStream {
             }
         }
 
-        if (!openNextFile()){
+        if (!openNextFile()) {
             return -1; // No more files.
         }
 
@@ -51,8 +51,8 @@ public abstract class AbstractDirectoryInputStream<T> extends InputStream {
     @Override
     public int read() throws IOException {
         final byte[] b = new byte[1];
-        int count = read(b, 0 , 1);
-        if(count < 0){
+        int count = read(b, 0, 1);
+        if (count < 0) {
             return -1;
         }
         return (int) b[0];
@@ -60,7 +60,7 @@ public abstract class AbstractDirectoryInputStream<T> extends InputStream {
 
     @Override
     public int available() throws IOException {
-        if(_currentInputStream != null){
+        if (_currentInputStream != null) {
             return _currentInputStream.available();
         } else {
             return 0;
@@ -68,7 +68,7 @@ public abstract class AbstractDirectoryInputStream<T> extends InputStream {
     }
 
     private boolean openNextFile() throws IOException {
-        if(_currentInputStream != null){
+        if (_currentInputStream != null) {
             FileHelper.safeClose(_currentInputStream);
             _currentInputStream = null;
         }
@@ -90,7 +90,7 @@ public abstract class AbstractDirectoryInputStream<T> extends InputStream {
 
     @Override
     public void close() throws IOException {
-        if(_currentInputStream != null){
+        if (_currentInputStream != null) {
             FileHelper.safeClose(_currentInputStream);
         }
     }

http://git-wip-us.apache.org/repos/asf/metamodel/blob/85f955a5/core/src/main/java/org/apache/metamodel/util/FileResource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/metamodel/util/FileResource.java b/core/src/main/java/org/apache/metamodel/util/FileResource.java
index 20dea50..ab2d8c3 100644
--- a/core/src/main/java/org/apache/metamodel/util/FileResource.java
+++ b/core/src/main/java/org/apache/metamodel/util/FileResource.java
@@ -25,7 +25,6 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
 import java.util.Arrays;
-import java.util.Comparator;
 
 /**
  * {@link File} based {@link Resource} implementation.
@@ -34,7 +33,7 @@ public class FileResource implements Resource, Serializable {
 
     private class DirectoryInputStream extends AbstractDirectoryInputStream<File> {
 
-        public DirectoryInputStream(){
+        public DirectoryInputStream() {
             final File[] unsortedFiles = _file.listFiles(new FileFilter() {
                 @Override
                 public boolean accept(final File pathname) {
@@ -42,7 +41,7 @@ public class FileResource implements Resource, Serializable {
                 }
             });
 
-            if(unsortedFiles == null){
+            if (unsortedFiles == null) {
                 _files = new File[0];
             } else {
                 Arrays.sort(unsortedFiles);
@@ -66,7 +65,7 @@ public class FileResource implements Resource, Serializable {
     public FileResource(File file) {
         _file = file;
     }
-    
+
     @Override
     public String toString() {
         return "FileResource[" + _file.getPath() + "]";
@@ -169,7 +168,7 @@ public class FileResource implements Resource, Serializable {
 
     @Override
     public InputStream read() throws ResourceException {
-        if(_file.isDirectory()){
+        if (_file.isDirectory()) {
             return new DirectoryInputStream();
         }
         final InputStream in = FileHelper.getInputStream(_file);

http://git-wip-us.apache.org/repos/asf/metamodel/blob/85f955a5/core/src/test/java/org/apache/metamodel/util/FileResourceTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/metamodel/util/FileResourceTest.java b/core/src/test/java/org/apache/metamodel/util/FileResourceTest.java
index 1316db9..c085225 100644
--- a/core/src/test/java/org/apache/metamodel/util/FileResourceTest.java
+++ b/core/src/test/java/org/apache/metamodel/util/FileResourceTest.java
@@ -20,28 +20,28 @@ package org.apache.metamodel.util;
 
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.Collections;
 
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 
 public class FileResourceTest {
+    @Rule
+    public TemporaryFolder folder = new TemporaryFolder();
 
     @Test
     public void testReadDirectory() throws Exception {
         final String contentString = "fun and games with Apache MetaModel and Hadoop is what we do";
         final String[] contents = new String[] { "fun ", "and ", "games ", "with ", "Apache ", "MetaModel ", "and ", "Hadoop ", "is ", "what ", "we ", "do" };
 
-        Path path = Files.createTempDirectory("test");
-
         // Reverse both filename and contents to make sure it is the name and not the creation order that is sorted on.
         int i = contents.length;
         Collections.reverse(Arrays.asList(contents));
         for(final String contentPart : contents){
-            final FileResource partResource = new FileResource(path + "/part-" + String.format("%02d", i--));
+            final FileResource partResource = new FileResource(folder.newFile("/part-" + String.format("%02d", i--)));
             partResource.write(new Action<OutputStream>() {
                 @Override
                 public void run(OutputStream out) throws Exception {
@@ -50,8 +50,7 @@ public class FileResourceTest {
             });
         }
 
-
-        final FileResource res1 = new FileResource(path.toFile());
+        final FileResource res1 = new FileResource(folder.getRoot());
 
         final String str1 = res1.read(new Func<InputStream, String>() {
             @Override

http://git-wip-us.apache.org/repos/asf/metamodel/blob/85f955a5/hadoop/src/main/java/org/apache/metamodel/util/HdfsResource.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/metamodel/util/HdfsResource.java b/hadoop/src/main/java/org/apache/metamodel/util/HdfsResource.java
index eab9597..263ac1d 100644
--- a/hadoop/src/main/java/org/apache/metamodel/util/HdfsResource.java
+++ b/hadoop/src/main/java/org/apache/metamodel/util/HdfsResource.java
@@ -18,13 +18,11 @@
  */
 package org.apache.metamodel.util;
 
-import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
 import java.util.Arrays;
-import java.util.Comparator;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 


[2/4] metamodel git commit: Missing license header

Posted by to...@apache.org.
Missing license header


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

Branch: refs/heads/master
Commit: 66915e745587fe5d74323898f43c08b37baf4840
Parents: f11f960
Author: Dennis Du Krøger <de...@humaninference.com>
Authored: Fri Aug 7 10:23:30 2015 +0200
Committer: Dennis Du Krøger <de...@humaninference.com>
Committed: Fri Aug 7 10:23:30 2015 +0200

----------------------------------------------------------------------
 .../apache/metamodel/util/FileResourceTest.java   | 18 ++++++++++++++++++
 1 file changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metamodel/blob/66915e74/core/src/test/java/org/apache/metamodel/util/FileResourceTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/metamodel/util/FileResourceTest.java b/core/src/test/java/org/apache/metamodel/util/FileResourceTest.java
index 910ae5b..1316db9 100644
--- a/core/src/test/java/org/apache/metamodel/util/FileResourceTest.java
+++ b/core/src/test/java/org/apache/metamodel/util/FileResourceTest.java
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.metamodel.util;
 
 import java.io.InputStream;