You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by om...@apache.org on 2014/03/26 19:19:59 UTC

svn commit: r1581982 [4/4] - in /hive/branches/branch-0.13: ./ common/src/java/org/apache/hadoop/hive/common/ itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/ metastore/src/java/org/apache/hadoop/hive/metastore/ metastore/src/java/org/a...

Modified: hive/branches/branch-0.13/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java?rev=1581982&r1=1581981&r2=1581982&view=diff
==============================================================================
--- hive/branches/branch-0.13/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java (original)
+++ hive/branches/branch-0.13/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java Wed Mar 26 18:19:58 2014
@@ -21,16 +21,19 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.MalformedURLException;
 import java.net.URL;
+import java.util.Arrays;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.net.URI;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -360,29 +363,11 @@ public class Hadoop20SShims extends Hado
   }
 
   @Override
-  public Iterator<FileStatus> listLocatedStatus(final FileSystem fs,
-                                                final Path path,
-                                                final PathFilter filter
-  ) throws IOException {
-    return new Iterator<FileStatus>() {
-      private final FileStatus[] result = fs.listStatus(path, filter);
-      private int current = 0;
-
-      @Override
-      public boolean hasNext() {
-        return current < result.length;
-      }
-
-      @Override
-      public FileStatus next() {
-        return result[current++];
-      }
-
-      @Override
-      public void remove() {
-        throw new IllegalArgumentException("Not supported");
-      }
-    };
+  public List<FileStatus> listLocatedStatus(final FileSystem fs,
+                                            final Path path,
+                                            final PathFilter filter
+                                            ) throws IOException {
+    return Arrays.asList(fs.listStatus(path, filter));
   }
 
   @Override
@@ -392,6 +377,11 @@ public class Hadoop20SShims extends Hado
   }
 
   @Override
+  public void hflush(FSDataOutputStream stream) throws IOException {
+    stream.sync();
+  }
+
+  @Override
   public FileSystem createProxyFileSystem(FileSystem fs, URI uri) {
     return new ProxyFileSystem(fs, uri);
   }

Modified: hive/branches/branch-0.13/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java?rev=1581982&r1=1581981&r2=1581982&view=diff
==============================================================================
--- hive/branches/branch-0.13/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java (original)
+++ hive/branches/branch-0.13/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java Wed Mar 26 18:19:58 2014
@@ -22,8 +22,10 @@ import java.lang.Integer;
 import java.net.InetSocketAddress;
 import java.net.MalformedURLException;
 import java.net.URL;
+import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.HashMap;
 import java.net.URI;
@@ -34,6 +36,7 @@ import org.apache.commons.lang.StringUti
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
@@ -465,51 +468,19 @@ public class Hadoop23Shims extends Hadoo
   }
 
   @Override
-  public Iterator<FileStatus> listLocatedStatus(final FileSystem fs,
-                                                final Path path,
-                                                final PathFilter filter
-  ) throws IOException {
-    return new Iterator<FileStatus>() {
-      private final RemoteIterator<LocatedFileStatus> inner =
-          fs.listLocatedStatus(path);
-      private FileStatus next;
-      {
-        next = null;
-        while (inner.hasNext() && next == null) {
-          next = inner.next();
-          if (filter != null && !filter.accept(next.getPath())) {
-            next = null;
-          }
-        }
-      }
-
-      @Override
-      public boolean hasNext() {
-        return next != null;
-      }
-
-      @Override
-      public FileStatus next() {
-        FileStatus result = next;
-        next = null;
-        try {
-          while (inner.hasNext() && next == null) {
-            next = inner.next();
-            if (filter != null && !filter.accept(next.getPath())) {
-              next = null;
-            }
-          }
-        } catch (IOException ioe) {
-          throw new IllegalArgumentException("Iterator exception", ioe);
-        }
-        return result;
+  public List<FileStatus> listLocatedStatus(final FileSystem fs,
+                                            final Path path,
+                                            final PathFilter filter
+                                           ) throws IOException {
+    RemoteIterator<LocatedFileStatus> itr = fs.listLocatedStatus(path);
+    List<FileStatus> result = new ArrayList<FileStatus>();
+    while(itr.hasNext()) {
+      FileStatus stat = itr.next();
+      if (filter == null || filter.accept(stat.getPath())) {
+        result.add(stat);
       }
-
-      @Override
-      public void remove() {
-        throw new IllegalArgumentException("Not supported");
-      }
-    };
+    }
+    return result;
   }
 
   @Override
@@ -522,6 +493,11 @@ public class Hadoop23Shims extends Hadoo
     }
   }
 
+  @Override
+  public void hflush(FSDataOutputStream stream) throws IOException {
+    stream.hflush();
+  }
+
   class ProxyFileSystem23 extends ProxyFileSystem {
     public ProxyFileSystem23(FileSystem fs) {
       super(fs);

Modified: hive/branches/branch-0.13/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java?rev=1581982&r1=1581981&r2=1581982&view=diff
==============================================================================
--- hive/branches/branch-0.13/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java (original)
+++ hive/branches/branch-0.13/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java Wed Mar 26 18:19:58 2014
@@ -38,6 +38,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -445,11 +446,11 @@ public interface HadoopShims {
    * @param fs the file system
    * @param path the directory name to get the status and block locations
    * @param filter a filter that needs to accept the file (or null)
-   * @return an iterator for the located file status objects
+   * @return an list for the located file status objects
    * @throws IOException
    */
-  Iterator<FileStatus> listLocatedStatus(FileSystem fs, Path path,
-      PathFilter filter) throws IOException;
+  List<FileStatus> listLocatedStatus(FileSystem fs, Path path,
+                                     PathFilter filter) throws IOException;
 
   /**
    * For file status returned by listLocatedStatus, convert them into a list
@@ -462,6 +463,13 @@ public interface HadoopShims {
   BlockLocation[] getLocations(FileSystem fs,
       FileStatus status) throws IOException;
 
+  /**
+   * Flush and make visible to other users the changes to the given stream.
+   * @param stream the stream to hflush.
+   * @throws IOException
+   */
+  public void hflush(FSDataOutputStream stream) throws IOException;
+
   public HCatHadoopShims getHCatShim();
   public interface HCatHadoopShims {