You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by am...@apache.org on 2016/01/30 00:52:14 UTC

[1/5] incubator-asterixdb git commit: Add flush() to IFrameWriter

Repository: incubator-asterixdb
Updated Branches:
  refs/heads/master e2439b443 -> ee387c12b


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
index fb2688f..3d7f60b 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
@@ -40,9 +40,13 @@ public class ExternalDataConstants {
     public static final String KEY_HDFS_URL = "hdfs";
     // specify the path when reading from a file system
     public static final String KEY_PATH = "path";
+    // specify the hdfs input format when reading data from HDFS
     public static final String KEY_INPUT_FORMAT = "input-format";
+    // specifies the filesystem (localfs or hdfs) when using a filesystem data source
     public static final String KEY_FILESYSTEM = "fs";
+    // specifies the address of the hdfs name node
     public static final String KEY_HADOOP_FILESYSTEM_URI = "fs.defaultFS";
+    // specifies the class implementation of the accessed instance of HDFS
     public static final String KEY_HADOOP_FILESYSTEM_CLASS = "fs.hdfs.impl";
     public static final String KEY_HADOOP_INPUT_DIR = "mapred.input.dir";
     public static final String KEY_HADOOP_INPUT_FORMAT = "mapred.input.format.class";
@@ -73,6 +77,20 @@ public class ExternalDataConstants {
     public static final String KEY_IS_FEED = "is-feed";
     public static final String KEY_WAIT_FOR_DATA = "wait-for-data";
     public static final String KEY_FEED_NAME = "feed";
+    // a string representing external bucket name
+    public static final String KEY_BUCKET = "bucket";
+    // a comma delimited list of nodes
+    public static final String KEY_NODES = "nodes";
+    // a string representing the password used to authenticate with the external data source
+    public static final String KEY_PASSWORD = "password";
+    // an integer representing the number of raw records that can be bufferred in the parsing queue 
+    public static final String KEY_QUEUE_SIZE = "queue-size";
+    // a comma delimited integers representing the indexes of the meta fields in the raw record (i,e: "3,1,0,2" denotes that the first meta field is in index 3 in the actual record)
+    public static final String KEY_META_INDEXES = "meta-indexes";
+    // an integer representing the index of the value field in the data type
+    public static final String KEY_VALUE_INDEX = "value-index";
+    // a string representing the format of the raw record in the value field in the data type
+    public static final String KEY_VALUE_FORMAT = "value-format";
     /**
      * HDFS class names
      */
@@ -95,6 +113,7 @@ public class ExternalDataConstants {
      */
     public static final String READER_HDFS = "hdfs";
     public static final String READER_ADM = "adm";
+    public static final String READER_COUCHBASE = "couchbase";
     public static final String READER_SEMISTRUCTURED = "semi-structured";
     public static final String READER_DELIMITED = "delimited-text";
     public static final String READER_TWITTER_PUSH = "twitter-push";
@@ -120,6 +139,7 @@ public class ExternalDataConstants {
     public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
     public static final String FORMAT_TWEET = "tweet";
     public static final String FORMAT_RSS = "rss";
+    public static final String FORMAT_RECORD_WITH_META = "record-with-meta";
 
     /**
      * input streams
@@ -178,6 +198,7 @@ public class ExternalDataConstants {
      */
     public static final int DEFAULT_BUFFER_SIZE = 4096;
     public static final int DEFAULT_BUFFER_INCREMENT = 4096;
+    public static final int DEFAULT_QUEUE_SIZE = 64;
 
     /**
      * Expected parameter values

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index c9be872..7c03e4d 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -171,15 +171,19 @@ public class ExternalDataUtils {
             if (tag == null) {
                 throw new NotImplementedException("Failed to get the type information for field " + i + ".");
             }
-            IValueParserFactory vpf = valueParserFactoryMap.get(tag);
-            if (vpf == null) {
-                throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
-            }
-            fieldParserFactories[i] = vpf;
+            fieldParserFactories[i] = getParserFactory(tag);
         }
         return fieldParserFactories;
     }
 
+    public static IValueParserFactory getParserFactory(ATypeTag tag) {
+        IValueParserFactory vpf = valueParserFactoryMap.get(tag);
+        if (vpf == null) {
+            throw new NotImplementedException("No value parser factory for fields of type " + tag);
+        }
+        return vpf;
+    }
+
     public static String getRecordReaderStreamName(Map<String, String> configuration) {
         return configuration.get(ExternalDataConstants.KEY_READER_STREAM);
     }
@@ -254,4 +258,10 @@ public class ExternalDataUtils {
     public static String getFeedName(Map<String, String> configuration) {
         return configuration.get(ExternalDataConstants.KEY_FEED_NAME);
     }
+
+    public static int getQueueSize(Map<String, String> configuration) {
+        return configuration.containsKey(ExternalDataConstants.KEY_QUEUE_SIZE)
+                ? Integer.parseInt(configuration.get(ExternalDataConstants.KEY_QUEUE_SIZE))
+                : ExternalDataConstants.DEFAULT_QUEUE_SIZE;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/util/FileSystemWatcher.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/FileSystemWatcher.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/FileSystemWatcher.java
index 4bb9d92..631eef4 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/util/FileSystemWatcher.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/FileSystemWatcher.java
@@ -34,6 +34,8 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
 
+import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
@@ -50,6 +52,7 @@ public class FileSystemWatcher {
     private final boolean isFeed;
     private boolean done;
     private File current;
+    private AbstractFeedDataFlowController controller;
 
     public FileSystemWatcher(FeedLogManager logManager, Path inputResource, String expression, boolean isFeed)
             throws IOException {
@@ -199,7 +202,7 @@ public class FileSystemWatcher {
         return false;
     }
 
-    public boolean hasNext() {
+    public boolean hasNext() throws HyracksDataException {
         if (it.hasNext()) {
             return true;
         }
@@ -218,6 +221,9 @@ public class FileSystemWatcher {
             key = watcher.poll();
         }
         // No file was found, wait for the filesystem to push events
+        if (controller != null) {
+            controller.flush();
+        }
         while (files.isEmpty()) {
             try {
                 key = watcher.take();
@@ -241,4 +247,8 @@ public class FileSystemWatcher {
         it = files.iterator();
         return it.hasNext();
     }
+
+    public void setController(AbstractFeedDataFlowController controller) {
+        this.controller = controller;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
index de6737a..7ac0428 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
@@ -28,7 +28,7 @@ import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.IndexingScheduler;
 import org.apache.asterix.external.indexing.RecordId.RecordIdType;
-import org.apache.asterix.external.input.stream.HDFSInputStreamProvider;
+import org.apache.asterix.external.input.stream.provider.HDFSInputStreamProvider;
 import org.apache.asterix.om.util.AsterixAppContextInfo;
 import org.apache.asterix.om.util.AsterixClusterProperties;
 import org.apache.hadoop.fs.BlockLocation;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
----------------------------------------------------------------------
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
index cff5c6b..f15540a 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
@@ -2155,7 +2155,7 @@ public class AqlMetadataProvider implements IMetadataProvider<AqlSourceId, Strin
         }
     }
 
-    public AlgebricksPartitionConstraint getClusterLocations() {
+    public AlgebricksAbsolutePartitionConstraint getClusterLocations() {
         return AsterixClusterProperties.INSTANCE.getClusterLocations();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
----------------------------------------------------------------------
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
index 769b883..c69cd16 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
@@ -37,6 +37,7 @@ import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.api.IDataSourceAdapter;
+import org.apache.asterix.external.feed.api.IFeed;
 import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
 import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
@@ -510,7 +511,7 @@ public class FeedMetadataUtil {
         return feedProps;
     }
 
-    private static ARecordType getOutputType(Feed feed, Map<String, String> configuration) throws Exception {
+    public static ARecordType getOutputType(IFeed feed, Map<String, String> configuration) throws Exception {
         ARecordType outputType = null;
         String fqOutputType = configuration.get(ExternalDataConstants.KEY_TYPE_NAME);
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixClusterProperties.java
----------------------------------------------------------------------
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixClusterProperties.java b/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixClusterProperties.java
index c2c28df..2744630 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixClusterProperties.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixClusterProperties.java
@@ -44,7 +44,6 @@ import org.apache.asterix.common.messaging.api.ICCMessageBroker;
 import org.apache.asterix.event.schema.cluster.Cluster;
 import org.apache.asterix.event.schema.cluster.Node;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 
 /**
  * A holder class for properties related to the Asterix cluster.
@@ -229,7 +228,7 @@ public class AsterixClusterProperties {
         return participantNodes;
     }
 
-    public synchronized AlgebricksPartitionConstraint getClusterLocations() {
+    public synchronized AlgebricksAbsolutePartitionConstraint getClusterLocations() {
         if (clusterPartitionConstraint == null) {
             resetClusterPartitionConstraint();
         }


[2/5] incubator-asterixdb git commit: Add flush() to IFrameWriter

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterRecordReaderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterRecordReaderFactory.java
new file mode 100644
index 0000000..f38c2cb
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterRecordReaderFactory.java
@@ -0,0 +1,134 @@
+/*
+ * 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.asterix.external.input.record.reader.twitter;
+
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.api.IRecordReaderFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.TwitterUtil;
+import org.apache.asterix.external.util.TwitterUtil.AuthenticationConstants;
+import org.apache.asterix.external.util.TwitterUtil.SearchAPIConstants;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+import twitter4j.Status;
+
+public class TwitterRecordReaderFactory implements IRecordReaderFactory<Status> {
+
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOGGER = Logger.getLogger(TwitterRecordReaderFactory.class.getName());
+
+    private static final String DEFAULT_INTERVAL = "10"; // 10 seconds
+    private static final int INTAKE_CARDINALITY = 1; // degree of parallelism at intake stage
+
+    private Map<String, String> configuration;
+    private boolean pull;
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return new AlgebricksCountPartitionConstraint(INTAKE_CARDINALITY);
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+        TwitterUtil.initializeConfigurationWithAuthInfo(configuration);
+        if (!validateConfiguration(configuration)) {
+            StringBuilder builder = new StringBuilder();
+            builder.append("One or more parameters are missing from adapter configuration\n");
+            builder.append(AuthenticationConstants.OAUTH_CONSUMER_KEY + "\n");
+            builder.append(AuthenticationConstants.OAUTH_CONSUMER_SECRET + "\n");
+            builder.append(AuthenticationConstants.OAUTH_ACCESS_TOKEN + "\n");
+            builder.append(AuthenticationConstants.OAUTH_ACCESS_TOKEN_SECRET + "\n");
+            throw new Exception(builder.toString());
+        }
+        if (ExternalDataUtils.isPull(configuration)) {
+            pull = true;
+            if (configuration.get(SearchAPIConstants.QUERY) == null) {
+                throw new AsterixException(
+                        "parameter " + SearchAPIConstants.QUERY + " not specified as part of adaptor configuration");
+            }
+            String interval = configuration.get(SearchAPIConstants.INTERVAL);
+            if (interval != null) {
+                try {
+                    Integer.parseInt(interval);
+                } catch (NumberFormatException nfe) {
+                    throw new IllegalArgumentException(
+                            "parameter " + SearchAPIConstants.INTERVAL + " is defined incorrectly, expecting a number");
+                }
+            } else {
+                configuration.put(SearchAPIConstants.INTERVAL, DEFAULT_INTERVAL);
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.warning(" Parameter " + SearchAPIConstants.INTERVAL + " not defined, using default ("
+                            + DEFAULT_INTERVAL + ")");
+                }
+            }
+        } else if (ExternalDataUtils.isPush(configuration)) {
+            pull = false;
+        } else {
+            throw new AsterixException("One of boolean parameters " + ExternalDataConstants.KEY_PULL + " and "
+                    + ExternalDataConstants.KEY_PUSH + " must be specified as part of adaptor configuration");
+        }
+    }
+
+    @Override
+    public boolean isIndexible() {
+        return false;
+    }
+
+    @Override
+    public IRecordReader<? extends Status> createRecordReader(IHyracksTaskContext ctx, int partition) throws Exception {
+        IRecordReader<Status> reader;
+        if (pull) {
+            reader = new TwitterPullRecordReader();
+        } else {
+            reader = new TwitterPushRecordReader();
+        }
+        reader.configure(configuration);
+        return reader;
+    }
+
+    @Override
+    public Class<? extends Status> getRecordClass() {
+        return Status.class;
+    }
+
+    private boolean validateConfiguration(Map<String, String> configuration) {
+        String consumerKey = configuration.get(AuthenticationConstants.OAUTH_CONSUMER_KEY);
+        String consumerSecret = configuration.get(AuthenticationConstants.OAUTH_CONSUMER_SECRET);
+        String accessToken = configuration.get(AuthenticationConstants.OAUTH_ACCESS_TOKEN);
+        String tokenSecret = configuration.get(AuthenticationConstants.OAUTH_ACCESS_TOKEN_SECRET);
+        if (consumerKey == null || consumerSecret == null || accessToken == null || tokenSecret == null) {
+            return false;
+        }
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStream.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStream.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStream.java
index 73f6195..ce65249 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStream.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStream.java
@@ -20,9 +20,14 @@ package org.apache.asterix.external.input.stream;
 
 import java.io.InputStream;
 
+import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
+
 public abstract class AInputStream extends InputStream {
     public abstract boolean skipError() throws Exception;
 
     public abstract boolean stop() throws Exception;
 
+    public void setController(AbstractFeedDataFlowController controller) throws UnsupportedOperationException {
+        throw new UnsupportedOperationException();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStreamReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStreamReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStreamReader.java
index 7ba6032..25418b0 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStreamReader.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AInputStreamReader.java
@@ -21,6 +21,8 @@ package org.apache.asterix.external.input.stream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 
+import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
+
 public class AInputStreamReader extends InputStreamReader {
     private AInputStream in;
 
@@ -40,4 +42,8 @@ public class AInputStreamReader extends InputStreamReader {
             throw new IOException(e);
         }
     }
+
+    public void setController(AbstractFeedDataFlowController controller) {
+        in.setController(controller);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStreamProvider.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStreamProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStreamProvider.java
deleted file mode 100644
index 8f4c094..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/HDFSInputStreamProvider.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * 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.asterix.external.input.stream;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.external.api.IInputStreamProvider;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.input.record.reader.HDFSRecordReader;
-import org.apache.asterix.external.provider.ExternalIndexerProvider;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-
-public class HDFSInputStreamProvider<K> extends HDFSRecordReader<K, Text> implements IInputStreamProvider {
-
-    public HDFSInputStreamProvider(boolean read[], InputSplit[] inputSplits, String[] readSchedule, String nodeName,
-            JobConf conf, Map<String, String> configuration, List<ExternalFile> snapshot) throws Exception {
-        super(read, inputSplits, readSchedule, nodeName, conf);
-        value = new Text();
-        configure(configuration);
-        if (snapshot != null) {
-            setSnapshot(snapshot);
-            setIndexer(ExternalIndexerProvider.getIndexer(configuration));
-            if (currentSplitIndex < snapshot.size()) {
-                indexer.reset(this);
-            }
-        }
-    }
-
-    @Override
-    public AInputStream getInputStream() throws Exception {
-        return new HDFSInputStream();
-    }
-
-    private class HDFSInputStream extends AInputStream {
-        int pos = 0;
-
-        @Override
-        public int read() throws IOException {
-            if (value.getLength() < pos) {
-                if (!readMore()) {
-                    return -1;
-                }
-            } else if (value.getLength() == pos) {
-                pos++;
-                return ExternalDataConstants.BYTE_LF;
-            }
-            return value.getBytes()[pos++];
-        }
-
-        private int readRecord(byte[] buffer, int offset, int len) {
-            int actualLength = value.getLength() + 1;
-            if ((actualLength - pos) > len) {
-                //copy partial record
-                System.arraycopy(value.getBytes(), pos, buffer, offset, len);
-                pos += len;
-                return len;
-            } else {
-                int numBytes = value.getLength() - pos;
-                System.arraycopy(value.getBytes(), pos, buffer, offset, numBytes);
-                buffer[offset + numBytes] = ExternalDataConstants.LF;
-                pos += numBytes;
-                numBytes++;
-                return numBytes;
-            }
-        }
-
-        @Override
-        public int read(byte[] buffer, int offset, int len) throws IOException {
-            if (value.getLength() > pos) {
-                return readRecord(buffer, offset, len);
-            }
-            if (!readMore()) {
-                return -1;
-            }
-            return readRecord(buffer, offset, len);
-        }
-
-        private boolean readMore() throws IOException {
-            try {
-                pos = 0;
-                return HDFSInputStreamProvider.this.hasNext();
-            } catch (Exception e) {
-                throw new IOException(e);
-            }
-        }
-
-        @Override
-        public boolean skipError() throws Exception {
-            return true;
-        }
-
-        @Override
-        public boolean stop() throws Exception {
-            return false;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStreamProvider.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStreamProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStreamProvider.java
deleted file mode 100644
index 22d0a87..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStreamProvider.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.asterix.external.input.stream;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Path;
-import java.util.Map;
-
-import org.apache.asterix.external.api.IInputStreamProvider;
-import org.apache.asterix.external.util.FeedLogManager;
-import org.apache.asterix.external.util.FeedUtils;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.dataflow.std.file.FileSplit;
-
-public class LocalFSInputStreamProvider implements IInputStreamProvider {
-
-    private String expression;
-    private boolean isFeed;
-    private Path path;
-    private File feedLogFile;
-
-    public LocalFSInputStreamProvider(FileSplit[] fileSplits, IHyracksTaskContext ctx,
-            Map<String, String> configuration, int partition, String expression, boolean isFeed,
-            FileSplit[] feedLogFileSplits) {
-        this.expression = expression;
-        this.isFeed = isFeed;
-        this.path = fileSplits[partition].getLocalFile().getFile().toPath();
-        if (feedLogFileSplits != null) {
-            this.feedLogFile = FeedUtils
-                    .getAbsoluteFileRef(feedLogFileSplits[partition].getLocalFile().getFile().getPath(),
-                            feedLogFileSplits[partition].getIODeviceId(), ctx.getIOManager())
-                    .getFile();
-
-        }
-    }
-
-    @Override
-    public AInputStream getInputStream() throws IOException {
-        FeedLogManager feedLogManager = null;
-        if (isFeed && feedLogFile != null) {
-            feedLogManager = new FeedLogManager(feedLogFile);
-        }
-        return new LocalFileSystemInputStream(path, expression, feedLogManager, isFeed);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFileSystemInputStream.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFileSystemInputStream.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFileSystemInputStream.java
index 7eebe4c..7b7cd8b 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFileSystemInputStream.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFileSystemInputStream.java
@@ -22,6 +22,7 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.nio.file.Path;
 
+import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.FeedLogManager;
 import org.apache.asterix.external.util.FileSystemWatcher;
@@ -39,6 +40,11 @@ public class LocalFileSystemInputStream extends AInputStream {
     }
 
     @Override
+    public void setController(AbstractFeedDataFlowController controller) {
+        watcher.setController(controller);
+    }
+
+    @Override
     public void close() throws IOException {
         IOException ioe = null;
         if (in != null) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketInputStreamProvider.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketInputStreamProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketInputStreamProvider.java
deleted file mode 100644
index 1f920e9..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketInputStreamProvider.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 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.asterix.external.input.stream;
-
-import java.net.ServerSocket;
-
-import org.apache.asterix.external.api.IInputStreamProvider;
-
-public class SocketInputStreamProvider implements IInputStreamProvider {
-    private ServerSocket server;
-
-    public SocketInputStreamProvider(ServerSocket server) {
-        this.server = server;
-    }
-
-    @Override
-    public AInputStream getInputStream() throws Exception {
-        return new SocketInputStream(server);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/TwitterFirehoseInputStreamProvider.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/TwitterFirehoseInputStreamProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/TwitterFirehoseInputStreamProvider.java
deleted file mode 100644
index 7c64aa3..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/TwitterFirehoseInputStreamProvider.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * 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.asterix.external.input.stream;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.PipedInputStream;
-import java.io.PipedOutputStream;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.api.IInputStreamProvider;
-import org.apache.asterix.external.util.TweetGenerator;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-public class TwitterFirehoseInputStreamProvider implements IInputStreamProvider {
-
-    private static final Logger LOGGER = Logger.getLogger(TwitterFirehoseInputStreamProvider.class.getName());
-
-    private ExecutorService executorService;
-
-    private PipedOutputStream outputStream;
-
-    private PipedInputStream inputStream;
-
-    private TwitterServer twitterServer;
-
-    public TwitterFirehoseInputStreamProvider(Map<String, String> configuration, IHyracksTaskContext ctx, int partition)
-            throws Exception {
-        executorService = Executors.newCachedThreadPool();
-        outputStream = new PipedOutputStream();
-        inputStream = new PipedInputStream(outputStream);
-        twitterServer = new TwitterServer(configuration, partition, outputStream, executorService, inputStream);
-    }
-
-    @Override
-    public AInputStream getInputStream() throws Exception {
-        twitterServer.start();
-        return twitterServer;
-    }
-
-    private static class TwitterServer extends AInputStream {
-        private final DataProvider dataProvider;
-        private final ExecutorService executorService;
-        private InputStream in;
-        private boolean started;
-
-        public TwitterServer(Map<String, String> configuration, int partition, OutputStream os,
-                ExecutorService executorService, InputStream in) throws Exception {
-            dataProvider = new DataProvider(configuration, partition, os);
-            this.executorService = executorService;
-            this.in = in;
-            this.started = false;
-        }
-
-        @Override
-        public boolean stop() throws IOException {
-            dataProvider.stop();
-            return true;
-        }
-
-        public void start() {
-            executorService.execute(dataProvider);
-        }
-
-        @Override
-        public boolean skipError() throws Exception {
-            return false;
-        }
-
-        @Override
-        public int read() throws IOException {
-            if (!started) {
-                start();
-                started = true;
-            }
-            return in.read();
-        }
-
-        @Override
-        public int read(byte b[], int off, int len) throws IOException {
-            if (!started) {
-                start();
-                started = true;
-            }
-            return in.read(b, off, len);
-        }
-    }
-
-    private static class DataProvider implements Runnable {
-
-        public static final String KEY_MODE = "mode";
-
-        private TweetGenerator tweetGenerator;
-        private boolean continuePush = true;
-        private int batchSize;
-        private final Mode mode;
-        private final OutputStream os;
-
-        public static enum Mode {
-            AGGRESSIVE,
-            CONTROLLED
-        }
-
-        public DataProvider(Map<String, String> configuration, int partition, OutputStream os) throws Exception {
-            this.tweetGenerator = new TweetGenerator(configuration, partition);
-            this.tweetGenerator.registerSubscriber(os);
-            this.os = os;
-            mode = configuration.get(KEY_MODE) != null ? Mode.valueOf(configuration.get(KEY_MODE).toUpperCase())
-                    : Mode.AGGRESSIVE;
-            switch (mode) {
-                case CONTROLLED:
-                    String tpsValue = configuration.get(TweetGenerator.KEY_TPS);
-                    if (tpsValue == null) {
-                        throw new IllegalArgumentException("TPS value not configured. use tps=<value>");
-                    }
-                    batchSize = Integer.parseInt(tpsValue);
-                    break;
-                case AGGRESSIVE:
-                    batchSize = 5000;
-                    break;
-            }
-        }
-
-        @Override
-        public void run() {
-            boolean moreData = true;
-            long startBatch;
-            long endBatch;
-
-            while (true) {
-                try {
-                    while (moreData && continuePush) {
-                        switch (mode) {
-                            case AGGRESSIVE:
-                                moreData = tweetGenerator.generateNextBatch(batchSize);
-                                break;
-                            case CONTROLLED:
-                                startBatch = System.currentTimeMillis();
-                                moreData = tweetGenerator.generateNextBatch(batchSize);
-                                endBatch = System.currentTimeMillis();
-                                if (endBatch - startBatch < 1000) {
-                                    Thread.sleep(1000 - (endBatch - startBatch));
-                                }
-                                break;
-                        }
-                    }
-                    os.close();
-                    break;
-                } catch (Exception e) {
-                    if (LOGGER.isLoggable(Level.WARNING)) {
-                        LOGGER.warning("Exception in adaptor " + e.getMessage());
-                    }
-                }
-            }
-        }
-
-        public void stop() {
-            continuePush = false;
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/LocalFSInputStreamProviderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/LocalFSInputStreamProviderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/LocalFSInputStreamProviderFactory.java
index ab1f8a0..06833af 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/LocalFSInputStreamProviderFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/LocalFSInputStreamProviderFactory.java
@@ -28,7 +28,7 @@ import org.apache.asterix.external.api.IInputStreamProvider;
 import org.apache.asterix.external.api.IInputStreamProviderFactory;
 import org.apache.asterix.external.api.INodeResolver;
 import org.apache.asterix.external.api.INodeResolverFactory;
-import org.apache.asterix.external.input.stream.LocalFSInputStreamProvider;
+import org.apache.asterix.external.input.stream.provider.LocalFSInputStreamProvider;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.external.util.FeedUtils;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketInputStreamProviderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketInputStreamProviderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketInputStreamProviderFactory.java
index 37afa53..ea60f43 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketInputStreamProviderFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketInputStreamProviderFactory.java
@@ -30,7 +30,7 @@ import java.util.Set;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.external.api.IInputStreamProvider;
 import org.apache.asterix.external.api.IInputStreamProviderFactory;
-import org.apache.asterix.external.input.stream.SocketInputStreamProvider;
+import org.apache.asterix.external.input.stream.provider.SocketInputStreamProvider;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.om.util.AsterixRuntimeUtil;
 import org.apache.commons.lang3.StringUtils;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/HDFSInputStreamProvider.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/HDFSInputStreamProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/HDFSInputStreamProvider.java
new file mode 100644
index 0000000..93a1685
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/HDFSInputStreamProvider.java
@@ -0,0 +1,119 @@
+/*
+ * 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.asterix.external.input.stream.provider;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IInputStreamProvider;
+import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.input.record.reader.hdfs.HDFSRecordReader;
+import org.apache.asterix.external.input.stream.AInputStream;
+import org.apache.asterix.external.provider.ExternalIndexerProvider;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+
+public class HDFSInputStreamProvider<K> extends HDFSRecordReader<K, Text> implements IInputStreamProvider {
+
+    public HDFSInputStreamProvider(boolean read[], InputSplit[] inputSplits, String[] readSchedule, String nodeName,
+            JobConf conf, Map<String, String> configuration, List<ExternalFile> snapshot) throws Exception {
+        super(read, inputSplits, readSchedule, nodeName, conf);
+        value = new Text();
+        configure(configuration);
+        if (snapshot != null) {
+            setSnapshot(snapshot);
+            setIndexer(ExternalIndexerProvider.getIndexer(configuration));
+            if (currentSplitIndex < snapshot.size()) {
+                indexer.reset(this);
+            }
+        }
+    }
+
+    @Override
+    public AInputStream getInputStream() throws Exception {
+        return new HDFSInputStream();
+    }
+
+    private class HDFSInputStream extends AInputStream {
+        int pos = 0;
+
+        @Override
+        public int read() throws IOException {
+            if (value.getLength() < pos) {
+                if (!readMore()) {
+                    return -1;
+                }
+            } else if (value.getLength() == pos) {
+                pos++;
+                return ExternalDataConstants.BYTE_LF;
+            }
+            return value.getBytes()[pos++];
+        }
+
+        private int readRecord(byte[] buffer, int offset, int len) {
+            int actualLength = value.getLength() + 1;
+            if ((actualLength - pos) > len) {
+                //copy partial record
+                System.arraycopy(value.getBytes(), pos, buffer, offset, len);
+                pos += len;
+                return len;
+            } else {
+                int numBytes = value.getLength() - pos;
+                System.arraycopy(value.getBytes(), pos, buffer, offset, numBytes);
+                buffer[offset + numBytes] = ExternalDataConstants.LF;
+                pos += numBytes;
+                numBytes++;
+                return numBytes;
+            }
+        }
+
+        @Override
+        public int read(byte[] buffer, int offset, int len) throws IOException {
+            if (value.getLength() > pos) {
+                return readRecord(buffer, offset, len);
+            }
+            if (!readMore()) {
+                return -1;
+            }
+            return readRecord(buffer, offset, len);
+        }
+
+        private boolean readMore() throws IOException {
+            try {
+                pos = 0;
+                return HDFSInputStreamProvider.this.hasNext();
+            } catch (Exception e) {
+                throw new IOException(e);
+            }
+        }
+
+        @Override
+        public boolean skipError() throws Exception {
+            return true;
+        }
+
+        @Override
+        public boolean stop() throws Exception {
+            return false;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/LocalFSInputStreamProvider.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/LocalFSInputStreamProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/LocalFSInputStreamProvider.java
new file mode 100644
index 0000000..4c4edd3
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/LocalFSInputStreamProvider.java
@@ -0,0 +1,64 @@
+/*
+ * 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.asterix.external.input.stream.provider;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IInputStreamProvider;
+import org.apache.asterix.external.input.stream.AInputStream;
+import org.apache.asterix.external.input.stream.LocalFileSystemInputStream;
+import org.apache.asterix.external.util.FeedLogManager;
+import org.apache.asterix.external.util.FeedUtils;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.dataflow.std.file.FileSplit;
+
+public class LocalFSInputStreamProvider implements IInputStreamProvider {
+
+    private String expression;
+    private boolean isFeed;
+    private Path path;
+    private File feedLogFile;
+
+    public LocalFSInputStreamProvider(FileSplit[] fileSplits, IHyracksTaskContext ctx,
+            Map<String, String> configuration, int partition, String expression, boolean isFeed,
+            FileSplit[] feedLogFileSplits) {
+        this.expression = expression;
+        this.isFeed = isFeed;
+        this.path = fileSplits[partition].getLocalFile().getFile().toPath();
+        if (feedLogFileSplits != null) {
+            this.feedLogFile = FeedUtils
+                    .getAbsoluteFileRef(feedLogFileSplits[partition].getLocalFile().getFile().getPath(),
+                            feedLogFileSplits[partition].getIODeviceId(), ctx.getIOManager())
+                    .getFile();
+
+        }
+    }
+
+    @Override
+    public AInputStream getInputStream() throws IOException {
+        FeedLogManager feedLogManager = null;
+        if (isFeed && feedLogFile != null) {
+            feedLogManager = new FeedLogManager(feedLogFile);
+        }
+        return new LocalFileSystemInputStream(path, expression, feedLogManager, isFeed);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/SocketInputStreamProvider.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/SocketInputStreamProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/SocketInputStreamProvider.java
new file mode 100644
index 0000000..2b12675
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/SocketInputStreamProvider.java
@@ -0,0 +1,38 @@
+/*
+ * 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.asterix.external.input.stream.provider;
+
+import java.net.ServerSocket;
+
+import org.apache.asterix.external.api.IInputStreamProvider;
+import org.apache.asterix.external.input.stream.AInputStream;
+import org.apache.asterix.external.input.stream.SocketInputStream;
+
+public class SocketInputStreamProvider implements IInputStreamProvider {
+    private ServerSocket server;
+
+    public SocketInputStreamProvider(ServerSocket server) {
+        this.server = server;
+    }
+
+    @Override
+    public AInputStream getInputStream() throws Exception {
+        return new SocketInputStream(server);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/TwitterFirehoseInputStreamProvider.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/TwitterFirehoseInputStreamProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/TwitterFirehoseInputStreamProvider.java
new file mode 100644
index 0000000..06f7e72
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/provider/TwitterFirehoseInputStreamProvider.java
@@ -0,0 +1,184 @@
+/*
+ * 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.asterix.external.input.stream.provider;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PipedInputStream;
+import java.io.PipedOutputStream;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.external.api.IInputStreamProvider;
+import org.apache.asterix.external.input.stream.AInputStream;
+import org.apache.asterix.external.util.TweetGenerator;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class TwitterFirehoseInputStreamProvider implements IInputStreamProvider {
+
+    private static final Logger LOGGER = Logger.getLogger(TwitterFirehoseInputStreamProvider.class.getName());
+
+    private ExecutorService executorService;
+
+    private PipedOutputStream outputStream;
+
+    private PipedInputStream inputStream;
+
+    private TwitterServer twitterServer;
+
+    public TwitterFirehoseInputStreamProvider(Map<String, String> configuration, IHyracksTaskContext ctx, int partition)
+            throws Exception {
+        executorService = Executors.newCachedThreadPool();
+        outputStream = new PipedOutputStream();
+        inputStream = new PipedInputStream(outputStream);
+        twitterServer = new TwitterServer(configuration, partition, outputStream, executorService, inputStream);
+    }
+
+    @Override
+    public AInputStream getInputStream() throws Exception {
+        twitterServer.start();
+        return twitterServer;
+    }
+
+    private static class TwitterServer extends AInputStream {
+        private final DataProvider dataProvider;
+        private final ExecutorService executorService;
+        private InputStream in;
+        private boolean started;
+
+        public TwitterServer(Map<String, String> configuration, int partition, OutputStream os,
+                ExecutorService executorService, InputStream in) throws Exception {
+            dataProvider = new DataProvider(configuration, partition, os);
+            this.executorService = executorService;
+            this.in = in;
+            this.started = false;
+        }
+
+        @Override
+        public boolean stop() throws IOException {
+            dataProvider.stop();
+            return true;
+        }
+
+        public void start() {
+            executorService.execute(dataProvider);
+        }
+
+        @Override
+        public boolean skipError() throws Exception {
+            return false;
+        }
+
+        @Override
+        public int read() throws IOException {
+            if (!started) {
+                start();
+                started = true;
+            }
+            return in.read();
+        }
+
+        @Override
+        public int read(byte b[], int off, int len) throws IOException {
+            if (!started) {
+                start();
+                started = true;
+            }
+            return in.read(b, off, len);
+        }
+    }
+
+    private static class DataProvider implements Runnable {
+
+        public static final String KEY_MODE = "mode";
+
+        private TweetGenerator tweetGenerator;
+        private boolean continuePush = true;
+        private int batchSize;
+        private final Mode mode;
+        private final OutputStream os;
+
+        public static enum Mode {
+            AGGRESSIVE,
+            CONTROLLED
+        }
+
+        public DataProvider(Map<String, String> configuration, int partition, OutputStream os) throws Exception {
+            this.tweetGenerator = new TweetGenerator(configuration, partition);
+            this.tweetGenerator.registerSubscriber(os);
+            this.os = os;
+            mode = configuration.get(KEY_MODE) != null ? Mode.valueOf(configuration.get(KEY_MODE).toUpperCase())
+                    : Mode.AGGRESSIVE;
+            switch (mode) {
+                case CONTROLLED:
+                    String tpsValue = configuration.get(TweetGenerator.KEY_TPS);
+                    if (tpsValue == null) {
+                        throw new IllegalArgumentException("TPS value not configured. use tps=<value>");
+                    }
+                    batchSize = Integer.parseInt(tpsValue);
+                    break;
+                case AGGRESSIVE:
+                    batchSize = 5000;
+                    break;
+            }
+        }
+
+        @Override
+        public void run() {
+            boolean moreData = true;
+            long startBatch;
+            long endBatch;
+
+            while (true) {
+                try {
+                    while (moreData && continuePush) {
+                        switch (mode) {
+                            case AGGRESSIVE:
+                                moreData = tweetGenerator.generateNextBatch(batchSize);
+                                break;
+                            case CONTROLLED:
+                                startBatch = System.currentTimeMillis();
+                                moreData = tweetGenerator.generateNextBatch(batchSize);
+                                endBatch = System.currentTimeMillis();
+                                if (endBatch - startBatch < 1000) {
+                                    Thread.sleep(1000 - (endBatch - startBatch));
+                                }
+                                break;
+                        }
+                    }
+                    os.close();
+                    break;
+                } catch (Exception e) {
+                    if (LOGGER.isLoggable(Level.WARNING)) {
+                        LOGGER.warning("Exception in adaptor " + e.getMessage());
+                    }
+                }
+            }
+        }
+
+        public void stop() {
+            continuePush = false;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalLookupOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalLookupOperatorDescriptor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalLookupOperatorDescriptor.java
index 9e35617..2cab3a7 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalLookupOperatorDescriptor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalLookupOperatorDescriptor.java
@@ -80,7 +80,7 @@ public class ExternalLookupOperatorDescriptor extends AbstractTreeIndexOperatorD
                 try {
                     adapter = adapterFactory.createAdapter(ctx, partition,
                             recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), snapshotAccessor, writer);
-                    //Open the file index accessor here
+                    // Open the file index accessor here
                     snapshotAccessor.open();
                     indexOpen = true;
                     adapter.open();
@@ -127,6 +127,11 @@ public class ExternalLookupOperatorDescriptor extends AbstractTreeIndexOperatorD
                     throw new HyracksDataException(th);
                 }
             }
+
+            @Override
+            public void flush() throws HyracksDataException {
+                adapter.flush();
+            }
         };
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java
index 80a54be..fa2e513 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java
@@ -25,9 +25,9 @@ import java.util.logging.Logger;
 
 import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
 import org.apache.asterix.external.feed.api.IFeedManager;
-import org.apache.asterix.external.feed.api.ISubscribableRuntime;
 import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
 import org.apache.asterix.external.feed.api.IFeedRuntime.Mode;
+import org.apache.asterix.external.feed.api.ISubscribableRuntime;
 import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
 import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
@@ -183,6 +183,9 @@ public class FeedMetaComputeNodePushable extends AbstractUnaryInputUnaryOutputOp
                     inputSideHandler.nextFrame(null); // signal end of data
                     while (!inputSideHandler.isFinished()) {
                         synchronized (coreOperator) {
+                            if (inputSideHandler.isFinished()) {
+                                break;
+                            }
                             coreOperator.wait();
                         }
                     }
@@ -192,8 +195,8 @@ public class FeedMetaComputeNodePushable extends AbstractUnaryInputUnaryOutputOp
             }
             coreOperator.close();
             System.out.println("CLOSED " + coreOperator + " STALLED ?" + stalled + " ENDED " + end);
-        } catch (Exception e) {
-            e.printStackTrace();
+        } catch (InterruptedException e) {
+            throw new HyracksDataException(e);
         } finally {
             if (!stalled) {
                 deregister();
@@ -221,4 +224,9 @@ public class FeedMetaComputeNodePushable extends AbstractUnaryInputUnaryOutputOp
         }
     }
 
+    @Override
+    public void flush() throws HyracksDataException {
+        inputSideHandler.flush();
+    }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaNodePushable.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaNodePushable.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaNodePushable.java
index 4dae72d..b09504a 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaNodePushable.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaNodePushable.java
@@ -181,4 +181,9 @@ public class FeedMetaNodePushable extends AbstractUnaryInputUnaryOutputOperatorN
         }
     }
 
+    @Override
+    public void flush() throws HyracksDataException {
+        inputSideHandler.flush();
+    }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
index f75b3eb..3c4c9ad 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
@@ -188,6 +188,9 @@ public class FeedMetaStoreNodePushable extends AbstractUnaryInputUnaryOutputOper
                 inputSideHandler.nextFrame(null); // signal end of data
                 while (!inputSideHandler.isFinished()) {
                     synchronized (coreOperator) {
+                        if (inputSideHandler.isFinished()) {
+                            break;
+                        }
                         coreOperator.wait();
                     }
                 }
@@ -195,8 +198,7 @@ public class FeedMetaStoreNodePushable extends AbstractUnaryInputUnaryOutputOper
             }
             coreOperator.close();
         } catch (Exception e) {
-            e.printStackTrace();
-            // ignore
+            throw new HyracksDataException(e);
         } finally {
             if (!stalled) {
                 deregister();
@@ -217,4 +219,9 @@ public class FeedMetaStoreNodePushable extends AbstractUnaryInputUnaryOutputOper
         }
     }
 
+    @Override
+    public void flush() throws HyracksDataException {
+        inputSideHandler.flush();
+    }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
index 129b62f..14a3e2a 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
@@ -39,7 +39,6 @@ import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType
 import org.apache.asterix.external.api.IRawRecord;
 import org.apache.asterix.external.api.IRecordDataParser;
 import org.apache.asterix.external.api.IStreamDataParser;
-import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ANull;
@@ -67,7 +66,6 @@ public class ADMDataParser extends AbstractDataParser implements IStreamDataPars
 
     private AdmLexer admLexer;
     private ARecordType recordType;
-    private boolean datasetRec;
     private boolean isStreamParser = true;
 
     private int nullableFieldId = 0;
@@ -142,7 +140,7 @@ public class ADMDataParser extends AbstractDataParser implements IStreamDataPars
     public boolean parse(DataOutput out) throws AsterixException {
         try {
             resetPools();
-            return parseAdmInstance(recordType, datasetRec, out);
+            return parseAdmInstance(recordType, out);
         } catch (IOException e) {
             throw new ParseException(e, filename, admLexer.getLine(), admLexer.getColumn());
         } catch (AdmLexerException e) {
@@ -163,12 +161,6 @@ public class ADMDataParser extends AbstractDataParser implements IStreamDataPars
     public void configure(Map<String, String> configuration, ARecordType recordType) throws IOException {
         this.recordType = recordType;
         this.configuration = configuration;
-        String isDatasetRecordString = configuration.get(ExternalDataConstants.KEY_DATASET_RECORD);
-        if (isDatasetRecordString == null) {
-            this.datasetRec = true;
-        } else {
-            this.datasetRec = Boolean.parseBoolean(isDatasetRecordString);
-        }
         this.isStreamParser = ExternalDataUtils.isDataSourceStreamProvider(configuration);
         if (!isStreamParser) {
             this.admLexer = new AdmLexer();
@@ -180,7 +172,7 @@ public class ADMDataParser extends AbstractDataParser implements IStreamDataPars
         try {
             resetPools();
             admLexer.setBuffer(record.get());
-            parseAdmInstance(recordType, datasetRec, out);
+            parseAdmInstance(recordType, out);
         } catch (IOException e) {
             throw new ParseException(e, filename, admLexer.getLine(), admLexer.getColumn());
         } catch (AdmLexerException e) {
@@ -201,18 +193,18 @@ public class ADMDataParser extends AbstractDataParser implements IStreamDataPars
         admLexer = new AdmLexer(new java.io.InputStreamReader(in));
     }
 
-    protected boolean parseAdmInstance(IAType objectType, boolean datasetRec, DataOutput out)
+    protected boolean parseAdmInstance(IAType objectType, DataOutput out)
             throws AsterixException, IOException, AdmLexerException {
         int token = admLexer.next();
         if (token == AdmLexer.TOKEN_EOF) {
             return false;
         } else {
-            admFromLexerStream(token, objectType, out, datasetRec);
+            admFromLexerStream(token, objectType, out);
             return true;
         }
     }
 
-    private void admFromLexerStream(int token, IAType objectType, DataOutput out, Boolean datasetRec)
+    private void admFromLexerStream(int token, IAType objectType, DataOutput out)
             throws AsterixException, IOException, AdmLexerException {
 
         switch (token) {
@@ -441,7 +433,7 @@ public class ADMDataParser extends AbstractDataParser implements IStreamDataPars
             case AdmLexer.TOKEN_START_RECORD: {
                 if (checkType(ATypeTag.RECORD, objectType)) {
                     objectType = getComplexType(objectType, ATypeTag.RECORD);
-                    parseRecord((ARecordType) objectType, out, datasetRec);
+                    parseRecord((ARecordType) objectType, out);
                 } else {
                     throw new ParseException(mismatchErrorMessage + objectType.getTypeTag());
                 }
@@ -567,7 +559,7 @@ public class ADMDataParser extends AbstractDataParser implements IStreamDataPars
         return getTargetTypeTag(expectedTypeTag, aObjectType) != null;
     }
 
-    private void parseRecord(ARecordType recType, DataOutput out, Boolean datasetRec)
+    private void parseRecord(ARecordType recType, DataOutput out)
             throws IOException, AsterixException, AdmLexerException {
 
         ArrayBackedValueStorage fieldValueBuffer = getTempBuffer();
@@ -575,16 +567,8 @@ public class ADMDataParser extends AbstractDataParser implements IStreamDataPars
         IARecordBuilder recBuilder = getRecordBuilder();
 
         BitSet nulls = null;
-        if (datasetRec) {
-
-            if (recType != null) {
-                nulls = new BitSet(recType.getFieldNames().length);
-                recBuilder.reset(recType);
-            } else {
-                recBuilder.reset(null);
-            }
-
-        } else if (recType != null) {
+        if (recType != null) {
+            //TODO: use BitSet Pool
             nulls = new BitSet(recType.getFieldNames().length);
             recBuilder.reset(recType);
         } else {
@@ -650,7 +634,7 @@ public class ADMDataParser extends AbstractDataParser implements IStreamDataPars
                     }
 
                     token = admLexer.next();
-                    this.admFromLexerStream(token, fieldType, fieldValueBuffer.getDataOutput(), false);
+                    this.admFromLexerStream(token, fieldType, fieldValueBuffer.getDataOutput());
                     if (openRecordField) {
                         if (fieldValueBuffer.getByteArray()[0] != ATypeTag.NULL.serialize()) {
                             recBuilder.addField(fieldNameBuffer, fieldValueBuffer);
@@ -752,7 +736,7 @@ public class ADMDataParser extends AbstractDataParser implements IStreamDataPars
                 expectingListItem = false;
                 itemBuffer.reset();
 
-                admFromLexerStream(token, itemType, itemBuffer.getDataOutput(), false);
+                admFromLexerStream(token, itemType, itemBuffer.getDataOutput());
                 orderedListBuilder.addItem(itemBuffer);
             }
             first = false;
@@ -799,7 +783,7 @@ public class ADMDataParser extends AbstractDataParser implements IStreamDataPars
             } else {
                 expectingListItem = false;
                 itemBuffer.reset();
-                admFromLexerStream(token, itemType, itemBuffer.getDataOutput(), false);
+                admFromLexerStream(token, itemType, itemBuffer.getDataOutput());
                 unorderedListBuilder.addItem(itemBuffer);
             }
             first = false;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java
index 6c399c3..2f0fc86 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java
@@ -28,6 +28,7 @@ import org.apache.asterix.builders.IARecordBuilder;
 import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.dataflow.data.nontagged.serde.ANullSerializerDeserializer;
+import org.apache.asterix.external.api.IDataParser;
 import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
 import org.apache.asterix.external.api.IRawRecord;
 import org.apache.asterix.external.api.IRecordDataParser;
@@ -124,18 +125,6 @@ public class DelimitedDataParser extends AbstractDataParser implements IStreamDa
         }
     }
 
-    protected void fieldNameToBytes(String fieldName, AMutableString str, ArrayBackedValueStorage buffer)
-            throws HyracksDataException {
-        buffer.reset();
-        DataOutput out = buffer.getDataOutput();
-        str.setValue(fieldName);
-        try {
-            stringSerde.serialize(str, out);
-        } catch (IOException e) {
-            throw new HyracksDataException(e);
-        }
-    }
-
     @Override
     public DataSourceType getDataSourceType() {
         return isStreamParser ? DataSourceType.STREAM : DataSourceType.RECORDS;
@@ -173,7 +162,8 @@ public class DelimitedDataParser extends AbstractDataParser implements IStreamDa
                     throw new HyracksDataException("Illegal field " + name + " in closed type " + recordType);
                 } else {
                     nameBuffers[i] = new ArrayBackedValueStorage();
-                    fieldNameToBytes(name, str, nameBuffers[i]);
+                    str.setValue(name);
+                    IDataParser.toBytes(str, nameBuffers[i], stringSerde);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RecordWithMetadataParser.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RecordWithMetadataParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RecordWithMetadataParser.java
new file mode 100644
index 0000000..ecdb03d
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RecordWithMetadataParser.java
@@ -0,0 +1,107 @@
+/*
+ * 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.asterix.external.parser;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.external.api.IDataParser;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.input.record.RecordWithMetadata;
+import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableString;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public class RecordWithMetadataParser<T> implements IRecordDataParser<RecordWithMetadata<T>> {
+
+    private final Class<? extends RecordWithMetadata<T>> clazz;
+    private final int[] metaIndexes;
+    private final int valueIndex;
+    private ARecordType recordType;
+    private IRecordDataParser<T> valueParser;
+    private RecordBuilder recBuilder;
+    private ArrayBackedValueStorage[] nameBuffers;
+    private int numberOfFields;
+    private ArrayBackedValueStorage valueBuffer = new ArrayBackedValueStorage();
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ASTRING);
+
+    public RecordWithMetadataParser(Class<? extends RecordWithMetadata<T>> clazz, int[] metaIndexes,
+            IRecordDataParser<T> valueParser, int valueIndex) {
+        this.clazz = clazz;
+        this.metaIndexes = metaIndexes;
+        this.valueParser = valueParser;
+        this.valueIndex = valueIndex;
+    }
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration, ARecordType recordType)
+            throws HyracksDataException, IOException {
+        this.recordType = recordType;
+        this.numberOfFields = recordType.getFieldNames().length;
+        recBuilder = new RecordBuilder();
+        recBuilder.reset(recordType);
+        recBuilder.init();
+        nameBuffers = new ArrayBackedValueStorage[numberOfFields];
+        AMutableString str = new AMutableString(null);
+        for (int i = 0; i < numberOfFields; i++) {
+            String name = recordType.getFieldNames()[i];
+            nameBuffers[i] = new ArrayBackedValueStorage();
+            str.setValue(name);
+            IDataParser.toBytes(str, nameBuffers[i], stringSerde);
+        }
+    }
+
+    @Override
+    public Class<? extends RecordWithMetadata<T>> getRecordClass() {
+        return clazz;
+    }
+
+    @Override
+    public void parse(IRawRecord<? extends RecordWithMetadata<T>> record, DataOutput out) throws Exception {
+        recBuilder.reset(recordType);
+        valueBuffer.reset();
+        recBuilder.init();
+        RecordWithMetadata<T> rwm = record.get();
+        for (int i = 0; i < numberOfFields; i++) {
+            if (i == valueIndex) {
+                valueParser.parse(rwm.getRecord(), valueBuffer.getDataOutput());
+                recBuilder.addField(i, valueBuffer);
+            } else {
+                recBuilder.addField(i, rwm.getMetadata(metaIndexes[i]));
+            }
+        }
+        recBuilder.write(out, true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/RecordWithMetadataParserFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/RecordWithMetadataParserFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/RecordWithMetadataParserFactory.java
new file mode 100644
index 0000000..88a0683
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/RecordWithMetadataParserFactory.java
@@ -0,0 +1,100 @@
+/*
+ * 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.asterix.external.parser.factory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.api.IRecordDataParserFactory;
+import org.apache.asterix.external.input.record.RecordWithMetadata;
+import org.apache.asterix.external.parser.RecordWithMetadataParser;
+import org.apache.asterix.external.provider.ParserFactoryProvider;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class RecordWithMetadataParserFactory<T> implements IRecordDataParserFactory<RecordWithMetadata<T>> {
+
+    private static final long serialVersionUID = 1L;
+    private Class<? extends RecordWithMetadata<T>> recordClass;
+    private ARecordType recordType;
+    private int[] metaIndexes;
+    private IRecordDataParserFactory<T> valueParserFactory;
+    private int valueIndex;
+
+    @Override
+    public DataSourceType getDataSourceType() throws AsterixException {
+        return DataSourceType.RECORDS;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        // validation first
+        if (!configuration.containsKey(ExternalDataConstants.KEY_META_INDEXES)) {
+            throw new HyracksDataException(
+                    "the parser parameter (" + ExternalDataConstants.KEY_META_INDEXES + ") is missing");
+        }
+        if (!configuration.containsKey(ExternalDataConstants.KEY_VALUE_INDEX)) {
+            throw new HyracksDataException(
+                    "the parser parameter (" + ExternalDataConstants.KEY_VALUE_INDEX + ") is missing");
+        }
+        if (!configuration.containsKey(ExternalDataConstants.KEY_VALUE_FORMAT)) {
+            throw new HyracksDataException(
+                    "the parser parameter (" + ExternalDataConstants.KEY_VALUE_FORMAT + ") is missing");
+        }
+        // get meta field indexes
+        String[] stringMetaIndexes = configuration.get(ExternalDataConstants.KEY_META_INDEXES).split(",");
+        metaIndexes = new int[stringMetaIndexes.length];
+        for (int i = 0; i < stringMetaIndexes.length; i++) {
+            metaIndexes[i] = Integer.parseInt(stringMetaIndexes[i].trim());
+        }
+        // get value index
+        valueIndex = Integer.parseInt(configuration.get(ExternalDataConstants.KEY_VALUE_INDEX).trim());
+        // get value format
+        configuration.put(ExternalDataConstants.KEY_DATA_PARSER,
+                configuration.get(ExternalDataConstants.KEY_VALUE_FORMAT));
+        valueParserFactory = (IRecordDataParserFactory<T>) ParserFactoryProvider.getDataParserFactory(configuration);
+        valueParserFactory.setRecordType((ARecordType) recordType.getFieldTypes()[valueIndex]);
+        valueParserFactory.configure(configuration);
+        recordClass = (Class<? extends RecordWithMetadata<T>>) (new RecordWithMetadata<T>(
+                valueParserFactory.getRecordClass())).getClass();
+    }
+
+    @Override
+    public void setRecordType(ARecordType recordType) {
+        this.recordType = recordType;
+    }
+
+    @Override
+    public IRecordDataParser<RecordWithMetadata<T>> createRecordParser(IHyracksTaskContext ctx)
+            throws HyracksDataException, AsterixException, IOException {
+        IRecordDataParser<T> valueParser = valueParserFactory.createRecordParser(ctx);
+        return new RecordWithMetadataParser<T>(recordClass, metaIndexes, valueParser, valueIndex);
+    }
+
+    @Override
+    public Class<? extends RecordWithMetadata<T>> getRecordClass() {
+        return recordClass;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
index a7ab062..745c653 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
@@ -25,9 +25,10 @@ import org.apache.asterix.external.api.IExternalDataSourceFactory;
 import org.apache.asterix.external.api.IInputStreamProviderFactory;
 import org.apache.asterix.external.api.IRecordReaderFactory;
 import org.apache.asterix.external.input.HDFSDataSourceFactory;
-import org.apache.asterix.external.input.record.reader.factory.LineRecordReaderFactory;
-import org.apache.asterix.external.input.record.reader.factory.SemiStructuredRecordReaderFactory;
-import org.apache.asterix.external.input.record.reader.factory.TwitterRecordReaderFactory;
+import org.apache.asterix.external.input.record.reader.couchbase.CouchbaseReaderFactory;
+import org.apache.asterix.external.input.record.reader.stream.LineRecordReaderFactory;
+import org.apache.asterix.external.input.record.reader.stream.SemiStructuredRecordReaderFactory;
+import org.apache.asterix.external.input.record.reader.twitter.TwitterRecordReaderFactory;
 import org.apache.asterix.external.input.stream.factory.LocalFSInputStreamProviderFactory;
 import org.apache.asterix.external.input.stream.factory.SocketInputStreamProviderFactory;
 import org.apache.asterix.external.util.ExternalDataConstants;
@@ -97,8 +98,11 @@ public class DatasourceFactoryProvider {
                 case ExternalDataConstants.READER_TWITTER_PUSH:
                     readerFactory = new TwitterRecordReaderFactory();
                     break;
+                case ExternalDataConstants.READER_COUCHBASE:
+                    readerFactory = new CouchbaseReaderFactory();
+                    break;
                 default:
-                    throw new AsterixException("unknown record reader factory");
+                    throw new AsterixException("unknown record reader factory: " + reader);
             }
         }
         return readerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.java
new file mode 100644
index 0000000..18b9cb5
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.java
@@ -0,0 +1,44 @@
+/*
+ * 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.asterix.external.provider;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.ILookupReaderFactory;
+import org.apache.asterix.external.input.record.reader.hdfs.HDFSLookupReaderFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+
+public class LookupReaderFactoryProvider {
+
+    @SuppressWarnings("rawtypes")
+    public static ILookupReaderFactory getLookupReaderFactory(Map<String, String> configuration) throws Exception {
+        String inputFormat = HDFSUtils.getInputFormatClassName(configuration);
+        if (inputFormat.equals(ExternalDataConstants.CLASS_NAME_TEXT_INPUT_FORMAT)
+                || inputFormat.equals(ExternalDataConstants.CLASS_NAME_SEQUENCE_INPUT_FORMAT)
+                || inputFormat.equals(ExternalDataConstants.CLASS_NAME_RC_INPUT_FORMAT)) {
+            HDFSLookupReaderFactory<Object> readerFactory = new HDFSLookupReaderFactory<Object>();
+            readerFactory.configure(configuration);
+            return readerFactory;
+        } else {
+            throw new AsterixException("Unrecognized external format");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java
index f5a0512..30595db 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java
@@ -22,10 +22,12 @@ import java.util.Map;
 
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.external.api.IDataParserFactory;
+import org.apache.asterix.external.input.record.RecordWithMetadata;
 import org.apache.asterix.external.parser.factory.ADMDataParserFactory;
 import org.apache.asterix.external.parser.factory.DelimitedDataParserFactory;
 import org.apache.asterix.external.parser.factory.HiveDataParserFactory;
 import org.apache.asterix.external.parser.factory.RSSParserFactory;
+import org.apache.asterix.external.parser.factory.RecordWithMetadataParserFactory;
 import org.apache.asterix.external.parser.factory.TweetParserFactory;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.ExternalDataUtils;
@@ -39,13 +41,12 @@ public class ParserFactoryProvider {
             return ExternalDataUtils.createExternalParserFactory(ExternalDataUtils.getDataverse(configuration),
                     parserFactoryName);
         } else {
-            parserFactory = ParserFactoryProvider.getParserFactory(configuration);
+            parserFactory = ParserFactoryProvider.getParserFactory(ExternalDataUtils.getRecordFormat(configuration));
         }
         return parserFactory;
     }
 
-    private static IDataParserFactory getParserFactory(Map<String, String> configuration) throws AsterixException {
-        String recordFormat = ExternalDataUtils.getRecordFormat(configuration);
+    private static IDataParserFactory getParserFactory(String recordFormat) throws AsterixException {
         switch (recordFormat) {
             case ExternalDataConstants.FORMAT_ADM:
             case ExternalDataConstants.FORMAT_JSON:
@@ -58,6 +59,8 @@ public class ParserFactoryProvider {
                 return new TweetParserFactory();
             case ExternalDataConstants.FORMAT_RSS:
                 return new RSSParserFactory();
+            case ExternalDataConstants.FORMAT_RECORD_WITH_META:
+                return new RecordWithMetadataParserFactory<RecordWithMetadata<?>>();
             default:
                 throw new AsterixException("Unknown data format");
         }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/util/DataflowUtils.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/util/DataflowUtils.java b/asterix-external-data/src/main/java/org/apache/asterix/external/util/DataflowUtils.java
index 90c74e1..27a1d0e 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/util/DataflowUtils.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/util/DataflowUtils.java
@@ -36,7 +36,7 @@ public class DataflowUtils {
     public static void addTupleToFrame(FrameTupleAppender appender, ArrayTupleBuilder tb, IFrameWriter writer)
             throws HyracksDataException {
         if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            appender.flush(writer, true);
+            appender.write(writer, true);
             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                 throw new HyracksDataException("Tuple is too large for a frame");
             }


[3/5] incubator-asterixdb git commit: Add flush() to IFrameWriter

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractHDFSLookupRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractHDFSLookupRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractHDFSLookupRecordReader.java
new file mode 100644
index 0000000..28abddb
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractHDFSLookupRecordReader.java
@@ -0,0 +1,113 @@
+/*
+ * 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.asterix.external.input.record.reader.hdfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.external.api.ILookupRecordReader;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
+import org.apache.asterix.external.indexing.RecordId;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class AbstractHDFSLookupRecordReader<T> implements ILookupRecordReader<T> {
+
+    protected int fileId;
+    private ExternalFileIndexAccessor snapshotAccessor;
+    protected ExternalFile file;
+    protected FileSystem fs;
+    protected Configuration conf;
+    protected boolean replaced;
+
+    public AbstractHDFSLookupRecordReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs,
+            Configuration conf) {
+        this.snapshotAccessor = snapshotAccessor;
+        this.fs = fs;
+        this.conf = conf;
+        this.fileId = -1;
+        this.file = new ExternalFile();
+    }
+
+    @Override
+    public void configure(Map<String, String> configurations) throws Exception {
+    }
+
+    @Override
+    public IRawRecord<T> read(RecordId rid) throws Exception {
+        if (rid.getFileId() != fileId) {
+            // close current file
+            closeFile();
+            // lookup new file
+            snapshotAccessor.lookup(rid.getFileId(), file);
+            fileId = rid.getFileId();
+            try {
+                validate();
+                if (!replaced) {
+                    openFile();
+                    validate();
+                    if (replaced) {
+                        closeFile();
+                    }
+                }
+            } catch (FileNotFoundException e) {
+                replaced = true;
+            }
+        }
+        if (replaced) {
+            return null;
+        }
+        return lookup(rid);
+    }
+
+    protected abstract IRawRecord<T> lookup(RecordId rid) throws IOException;
+
+    private void validate() throws IllegalArgumentException, IOException {
+        FileStatus fileStatus = fs.getFileStatus(new Path(file.getFileName()));
+        replaced = fileStatus.getModificationTime() != file.getLastModefiedTime().getTime();
+    }
+
+    protected abstract void closeFile();
+
+    protected abstract void openFile() throws IllegalArgumentException, IOException;
+
+    @Override
+    public final void open() throws HyracksDataException {
+        snapshotAccessor.open();
+    }
+
+    @Override
+    public void close() throws IOException {
+        try {
+            closeFile();
+        } finally {
+            snapshotAccessor.close();
+        }
+    }
+
+    @Override
+    public void fail() {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.java
new file mode 100644
index 0000000..c302b9b
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.java
@@ -0,0 +1,87 @@
+/*
+ * 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.asterix.external.input.record.reader.hdfs;
+
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.ILookupReaderFactory;
+import org.apache.asterix.external.api.ILookupRecordReader;
+import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.hdfs.dataflow.ConfFactory;
+
+public class HDFSLookupReaderFactory<T> implements ILookupReaderFactory<T> {
+
+    protected static final long serialVersionUID = 1L;
+    protected transient AlgebricksPartitionConstraint clusterLocations;
+    protected ConfFactory confFactory;
+    protected Map<String, String> configuration;
+
+    public HDFSLookupReaderFactory() {
+    }
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        clusterLocations = HDFSUtils.getPartitionConstraints(clusterLocations);
+        return clusterLocations;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+        JobConf conf = HDFSUtils.configureHDFSJobConf(configuration);
+        confFactory = new ConfFactory(conf);
+
+    }
+
+    @Override
+    public boolean isIndexible() {
+        return false;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public ILookupRecordReader<? extends T> createRecordReader(IHyracksTaskContext ctx, int partition,
+            ExternalFileIndexAccessor snapshotAccessor) throws Exception {
+        String inputFormatParameter = configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT).trim();
+        JobConf conf = confFactory.getConf();
+        FileSystem fs = FileSystem.get(conf);
+        switch (inputFormatParameter) {
+            case ExternalDataConstants.INPUT_FORMAT_TEXT:
+                return (ILookupRecordReader<? extends T>) new TextLookupReader(snapshotAccessor, fs, conf);
+            case ExternalDataConstants.INPUT_FORMAT_SEQUENCE:
+                return (ILookupRecordReader<? extends T>) new SequenceLookupReader(snapshotAccessor, fs, conf);
+            case ExternalDataConstants.INPUT_FORMAT_RC:
+                return (ILookupRecordReader<? extends T>) new RCLookupReader(snapshotAccessor, fs, conf);
+            default:
+                throw new AsterixException("Unrecognised input format: " + inputFormatParameter);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java
new file mode 100644
index 0000000..564d55a
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java
@@ -0,0 +1,185 @@
+/*
+ * 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.asterix.external.input.record.reader.hdfs;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IExternalIndexer;
+import org.apache.asterix.external.api.IIndexingDatasource;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.external.input.record.GenericRecord;
+import org.apache.asterix.external.input.record.reader.EmptyRecordReader;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class HDFSRecordReader<K, V extends Writable> implements IRecordReader<Writable>, IIndexingDatasource {
+
+    protected RecordReader<K, Writable> reader;
+    protected V value = null;
+    protected K key = null;
+    protected int currentSplitIndex = 0;
+    protected boolean read[];
+    protected InputFormat<?, ?> inputFormat;
+    protected InputSplit[] inputSplits;
+    protected String[] readSchedule;
+    protected String nodeName;
+    protected JobConf conf;
+    protected GenericRecord<Writable> record;
+    // Indexing variables
+    protected IExternalIndexer indexer;
+    protected List<ExternalFile> snapshot;
+    protected FileSystem hdfs;
+
+    public HDFSRecordReader(boolean read[], InputSplit[] inputSplits, String[] readSchedule, String nodeName,
+            JobConf conf) {
+        this.read = read;
+        this.inputSplits = inputSplits;
+        this.readSchedule = readSchedule;
+        this.nodeName = nodeName;
+        this.conf = conf;
+        this.inputFormat = conf.getInputFormat();
+        this.reader = new EmptyRecordReader<K, Writable>();
+    }
+
+    @Override
+    public void close() throws IOException {
+        reader.close();
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        record = new GenericRecord<Writable>();
+        nextInputSplit();
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        if (reader.next(key, value)) {
+            return true;
+        }
+        while (nextInputSplit()) {
+            if (reader.next(key, value)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public IRawRecord<Writable> next() throws IOException {
+        record.set(value);
+        return record;
+    }
+
+    private boolean nextInputSplit() throws IOException {
+        for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+            /**
+             * read all the partitions scheduled to the current node
+             */
+            if (readSchedule[currentSplitIndex].equals(nodeName)) {
+                /**
+                 * pick an unread split to read synchronize among
+                 * simultaneous partitions in the same machine
+                 */
+                synchronized (read) {
+                    if (read[currentSplitIndex] == false) {
+                        read[currentSplitIndex] = true;
+                    } else {
+                        continue;
+                    }
+                }
+                if (snapshot != null) {
+                    String fileName = ((FileSplit) (inputSplits[currentSplitIndex])).getPath().toUri().getPath();
+                    FileStatus fileStatus = hdfs.getFileStatus(new Path(fileName));
+                    // Skip if not the same file stored in the files snapshot
+                    if (fileStatus.getModificationTime() != snapshot.get(currentSplitIndex).getLastModefiedTime()
+                            .getTime())
+                        continue;
+                }
+
+                reader.close();
+                reader = getRecordReader(currentSplitIndex);
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @SuppressWarnings("unchecked")
+    private RecordReader<K, Writable> getRecordReader(int splitIndex) throws IOException {
+        reader = (RecordReader<K, Writable>) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
+        if (key == null) {
+            key = reader.createKey();
+            value = (V) reader.createValue();
+        }
+        if (indexer != null) {
+            try {
+                indexer.reset(this);
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        return reader;
+    }
+
+    @Override
+    public boolean stop() {
+        return false;
+    }
+
+    @Override
+    public IExternalIndexer getIndexer() {
+        return indexer;
+    }
+
+    @Override
+    public void setIndexer(IExternalIndexer indexer) {
+        this.indexer = indexer;
+    }
+
+    public List<ExternalFile> getSnapshot() {
+        return snapshot;
+    }
+
+    public void setSnapshot(List<ExternalFile> snapshot) throws IOException {
+        this.snapshot = snapshot;
+        hdfs = FileSystem.get(conf);
+    }
+
+    public int getCurrentSplitIndex() {
+        return currentSplitIndex;
+    }
+
+    public RecordReader<K, Writable> getReader() {
+        return reader;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSTextLineReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSTextLineReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSTextLineReader.java
new file mode 100644
index 0000000..90f1558
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSTextLineReader.java
@@ -0,0 +1,234 @@
+/*
+ * 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.asterix.external.input.record.reader.hdfs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.io.Text;
+
+public class HDFSTextLineReader {
+    private static final int DEFAULT_BUFFER_SIZE = 32 * 1024;
+    private int bufferSize = DEFAULT_BUFFER_SIZE;
+    private FSDataInputStream reader;
+
+    private byte[] buffer;
+    // the number of bytes of real data in the buffer
+    private int bufferLength = 0;
+    // the current position in the buffer
+    private int bufferPosn = 0;
+
+    private long currentFilePos = 0L;
+
+    private static final byte CR = '\r';
+    private static final byte LF = '\n';
+
+    public static final String KEY_BUFFER_SIZE = "io.file.buffer.size";
+
+    /**
+     * Create a line reader that reads from the given stream using the
+     * default buffer-size (32k).
+     *
+     * @param in
+     *            The input stream
+     * @throws IOException
+     */
+    public HDFSTextLineReader(FSDataInputStream in) throws IOException {
+        this(in, DEFAULT_BUFFER_SIZE);
+    }
+
+    /**
+     * Create a line reader that reads from the given stream using the
+     * given buffer-size.
+     *
+     * @param in
+     *            The input stream
+     * @param bufferSize
+     *            Size of the read buffer
+     * @throws IOException
+     */
+    public HDFSTextLineReader(FSDataInputStream in, int bufferSize) throws IOException {
+        this.reader = in;
+        this.bufferSize = bufferSize;
+        this.buffer = new byte[this.bufferSize];
+        currentFilePos = in.getPos();
+    }
+
+    public HDFSTextLineReader() throws IOException {
+        this.bufferSize = DEFAULT_BUFFER_SIZE;
+        this.buffer = new byte[this.bufferSize];
+    }
+
+    /**
+     * Create a line reader that reads from the given stream using the <code>io.file.buffer.size</code> specified in the given <code>Configuration</code>.
+     *
+     * @param in
+     *            input stream
+     * @param conf
+     *            configuration
+     * @throws IOException
+     */
+    public HDFSTextLineReader(FSDataInputStream in, Configuration conf) throws IOException {
+        this(in, conf.getInt(KEY_BUFFER_SIZE, DEFAULT_BUFFER_SIZE));
+    }
+
+    /**
+     * Read one line from the InputStream into the given Text. A line
+     * can be terminated by one of the following: '\n' (LF) , '\r' (CR),
+     * or '\r\n' (CR+LF). EOF also terminates an otherwise unterminated
+     * line.
+     *
+     * @param str
+     *            the object to store the given line (without newline)
+     * @param maxLineLength
+     *            the maximum number of bytes to store into str;
+     *            the rest of the line is silently discarded.
+     * @param maxBytesToConsume
+     *            the maximum number of bytes to consume
+     *            in this call. This is only a hint, because if the line cross
+     *            this threshold, we allow it to happen. It can overshoot
+     *            potentially by as much as one buffer length.
+     * @return the number of bytes read including the (longest) newline
+     *         found.
+     * @throws IOException
+     *             if the underlying stream throws
+     */
+    public int readLine(Text str, int maxLineLength, int maxBytesToConsume) throws IOException {
+        /* We're reading data from in, but the head of the stream may be
+         * already buffered in buffer, so we have several cases:
+         * 1. No newline characters are in the buffer, so we need to copy
+         *    everything and read another buffer from the stream.
+         * 2. An unambiguously terminated line is in buffer, so we just
+         *    copy to str.
+         * 3. Ambiguously terminated line is in buffer, i.e. buffer ends
+         *    in CR.  In this case we copy everything up to CR to str, but
+         *    we also need to see what follows CR: if it's LF, then we
+         *    need consume LF as well, so next call to readLine will read
+         *    from after that.
+         * We use a flag prevCharCR to signal if previous character was CR
+         * and, if it happens to be at the end of the buffer, delay
+         * consuming it until we have a chance to look at the char that
+         * follows.
+         */
+        str.clear();
+        int txtLength = 0; //tracks str.getLength(), as an optimization
+        int newlineLength = 0; //length of terminating newline
+        boolean prevCharCR = false; //true of prev char was CR
+        long bytesConsumed = 0;
+        do {
+            int startPosn = bufferPosn; //starting from where we left off the last time
+            if (bufferPosn >= bufferLength) {
+                startPosn = bufferPosn = 0;
+                if (prevCharCR)
+                    ++bytesConsumed; //account for CR from previous read
+                bufferLength = reader.read(buffer);
+                if (bufferLength <= 0)
+                    break; // EOF
+            }
+            for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline
+                if (buffer[bufferPosn] == LF) {
+                    newlineLength = (prevCharCR) ? 2 : 1;
+                    ++bufferPosn; // at next invocation proceed from following byte
+                    break;
+                }
+                if (prevCharCR) { //CR + notLF, we are at notLF
+                    newlineLength = 1;
+                    break;
+                }
+                prevCharCR = (buffer[bufferPosn] == CR);
+            }
+            int readLength = bufferPosn - startPosn;
+            if (prevCharCR && newlineLength == 0)
+                --readLength; //CR at the end of the buffer
+            bytesConsumed += readLength;
+            int appendLength = readLength - newlineLength;
+            if (appendLength > maxLineLength - txtLength) {
+                appendLength = maxLineLength - txtLength;
+            }
+            if (appendLength > 0) {
+                str.append(buffer, startPosn, appendLength);
+                txtLength += appendLength;
+            }
+        } while (newlineLength == 0 && bytesConsumed < maxBytesToConsume);
+
+        if (bytesConsumed > Integer.MAX_VALUE)
+            throw new IOException("Too many bytes before newline: " + bytesConsumed);
+        currentFilePos = reader.getPos() - bufferLength + bufferPosn;
+        return (int) bytesConsumed;
+    }
+
+    /**
+     * Read from the InputStream into the given Text.
+     *
+     * @param str
+     *            the object to store the given line
+     * @param maxLineLength
+     *            the maximum number of bytes to store into str.
+     * @return the number of bytes read including the newline
+     * @throws IOException
+     *             if the underlying stream throws
+     */
+    public int readLine(Text str, int maxLineLength) throws IOException {
+        return readLine(str, maxLineLength, Integer.MAX_VALUE);
+    }
+
+    /**
+     * Read from the InputStream into the given Text.
+     *
+     * @param str
+     *            the object to store the given line
+     * @return the number of bytes read including the newline
+     * @throws IOException
+     *             if the underlying stream throws
+     */
+    public int readLine(Text str) throws IOException {
+        return readLine(str, Integer.MAX_VALUE, Integer.MAX_VALUE);
+    }
+
+    public void seek(long desired) throws IOException {
+        if (reader.getPos() <= desired || currentFilePos > desired) {
+            // desired position is ahead of stream or before the current position, seek to position
+            reader.seek(desired);
+            bufferLength = 0;
+            bufferPosn = 0;
+            currentFilePos = desired;
+        } else if (currentFilePos < desired) {
+            // desired position is in the buffer
+            int difference = (int) (desired - currentFilePos);
+            bufferPosn += difference;
+            currentFilePos = desired;
+        }
+    }
+
+    public FSDataInputStream getReader() {
+        return reader;
+    }
+
+    public void resetReader(FSDataInputStream reader) throws IOException {
+        this.reader = reader;
+        bufferLength = 0;
+        bufferPosn = 0;
+        currentFilePos = reader.getPos();
+    }
+
+    public void close() throws IOException {
+        reader.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/RCLookupReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/RCLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/RCLookupReader.java
new file mode 100644
index 0000000..95d76ba
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/RCLookupReader.java
@@ -0,0 +1,92 @@
+/*
+ * 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.asterix.external.input.record.reader.hdfs;
+
+import java.io.IOException;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
+import org.apache.asterix.external.indexing.RecordId;
+import org.apache.asterix.external.input.record.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.RCFile.Reader;
+import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.log4j.Logger;
+
+public class RCLookupReader extends AbstractHDFSLookupRecordReader<BytesRefArrayWritable> {
+    public RCLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs, Configuration conf) {
+        super(snapshotAccessor, fs, conf);
+    }
+
+    private static final Logger LOGGER = Logger.getLogger(RCLookupReader.class.getName());
+    private Reader reader;
+    private LongWritable key = new LongWritable();
+    private BytesRefArrayWritable value = new BytesRefArrayWritable();
+    private GenericRecord<BytesRefArrayWritable> record = new GenericRecord<BytesRefArrayWritable>();
+    private long offset;
+    private int row;
+
+    @Override
+    public Class<?> getRecordClass() throws IOException {
+        return Writable.class;
+    }
+
+    @Override
+    protected IRawRecord<BytesRefArrayWritable> lookup(RecordId rid) throws IOException {
+        if (rid.getOffset() != offset) {
+            offset = rid.getOffset();
+            if (reader.getPosition() != offset)
+                reader.seek(offset);
+            reader.resetBuffer();
+            row = -1;
+        }
+
+        // skip rows to the record row
+        while (row < rid.getRow()) {
+            reader.next(key);
+            reader.getCurrentRow(value);
+            row++;
+        }
+        record.set(value);
+        return record;
+    }
+
+    @Override
+    protected void closeFile() {
+        if (reader == null) {
+            return;
+        }
+        try {
+            reader.close();
+        } catch (Exception e) {
+            LOGGER.warn("Error closing HDFS file", e);
+        }
+    }
+
+    @Override
+    protected void openFile() throws IllegalArgumentException, IOException {
+        reader = new Reader(fs, new Path(file.getFileName()), conf);
+        offset = -1;
+        row = -1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/SequenceLookupReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/SequenceLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/SequenceLookupReader.java
new file mode 100644
index 0000000..23e647f
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/SequenceLookupReader.java
@@ -0,0 +1,71 @@
+/*
+ * 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.asterix.external.input.record.reader.hdfs;
+
+import java.io.IOException;
+
+import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
+import org.apache.asterix.external.indexing.RecordId;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.log4j.Logger;
+
+public class SequenceLookupReader extends AbstractCharRecordLookupReader {
+
+    public SequenceLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs, Configuration conf) {
+        super(snapshotAccessor, fs, conf);
+    }
+
+    private static final Logger LOGGER = Logger.getLogger(SequenceLookupReader.class.getName());
+    private Reader reader;
+    private Writable key;
+
+    @Override
+    protected void readRecord(RecordId rid) throws IOException {
+        reader.seek(rid.getOffset());
+        reader.next(key, value);
+    }
+
+    @Override
+    protected void closeFile() {
+        if (reader == null) {
+            return;
+        }
+        try {
+            reader.close();
+        } catch (Exception e) {
+            LOGGER.warn("Error closing HDFS file ", e);
+        }
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    protected void openFile() throws IllegalArgumentException, IOException {
+        reader = new SequenceFile.Reader(fs, new Path(file.getFileName()), conf);
+        key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
+        value = (Text) ReflectionUtils.newInstance(reader.getValueClass(), conf);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/TextLookupReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/TextLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/TextLookupReader.java
new file mode 100644
index 0000000..2e1a11a
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/TextLookupReader.java
@@ -0,0 +1,64 @@
+/*
+ * 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.asterix.external.input.record.reader.hdfs;
+
+import java.io.IOException;
+
+import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
+import org.apache.asterix.external.indexing.RecordId;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.Logger;
+
+public class TextLookupReader extends AbstractCharRecordLookupReader {
+
+    public TextLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs, Configuration conf) {
+        super(snapshotAccessor, fs, conf);
+    }
+
+    private static final Logger LOGGER = Logger.getLogger(TextLookupReader.class.getName());
+    private HDFSTextLineReader reader;
+
+    @Override
+    protected void readRecord(RecordId rid) throws IOException {
+        reader.seek(rid.getOffset());
+        reader.readLine(value);
+    }
+
+    @Override
+    protected void closeFile() {
+        if (reader == null) {
+            return;
+        }
+        try {
+            reader.close();
+        } catch (Exception e) {
+            LOGGER.warn("Error closing HDFS file ", e);
+        }
+    }
+
+    @Override
+    protected void openFile() throws IllegalArgumentException, IOException {
+        if (reader == null) {
+            reader = new HDFSTextLineReader();
+        }
+        reader.resetReader(fs.open(new Path(file.getFileName())));;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/rss/RSSRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/rss/RSSRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/rss/RSSRecordReader.java
new file mode 100644
index 0000000..13cd26a
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/rss/RSSRecordReader.java
@@ -0,0 +1,172 @@
+/*
+ * 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.asterix.external.input.record.reader.rss;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.input.record.GenericRecord;
+import org.apache.log4j.Logger;
+
+import com.sun.syndication.feed.synd.SyndEntryImpl;
+import com.sun.syndication.feed.synd.SyndFeed;
+import com.sun.syndication.fetcher.FeedFetcher;
+import com.sun.syndication.fetcher.FetcherEvent;
+import com.sun.syndication.fetcher.FetcherException;
+import com.sun.syndication.fetcher.FetcherListener;
+import com.sun.syndication.fetcher.impl.FeedFetcherCache;
+import com.sun.syndication.fetcher.impl.HashMapFeedInfoCache;
+import com.sun.syndication.fetcher.impl.HttpURLFeedFetcher;
+import com.sun.syndication.io.FeedException;
+
+public class RSSRecordReader implements IRecordReader<SyndEntryImpl> {
+
+    private static final Logger LOGGER = Logger.getLogger(RSSRecordReader.class.getName());
+    private boolean modified = false;
+    private Queue<SyndEntryImpl> rssFeedBuffer = new LinkedList<SyndEntryImpl>();
+    private FeedFetcherCache feedInfoCache;
+    private FeedFetcher fetcher;
+    private FetcherEventListenerImpl listener;
+    private URL feedUrl;
+    private GenericRecord<SyndEntryImpl> record = new GenericRecord<SyndEntryImpl>();
+    private boolean done = false;
+
+    public RSSRecordReader(String url) throws MalformedURLException {
+        feedUrl = new URL(url);
+        feedInfoCache = HashMapFeedInfoCache.getInstance();
+        fetcher = new HttpURLFeedFetcher(feedInfoCache);
+        listener = new FetcherEventListenerImpl(this, LOGGER);
+        fetcher.addFetcherEventListener(listener);
+    }
+
+    public boolean isModified() {
+        return modified;
+    }
+
+    @Override
+    public void close() throws IOException {
+        fetcher.removeFetcherEventListener(listener);
+    }
+
+    @Override
+    public void configure(Map<String, String> configurations) throws Exception {
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        return !done;
+    }
+
+    @Override
+    public IRawRecord<SyndEntryImpl> next() throws IOException {
+        if (done) {
+            return null;
+        }
+        try {
+            SyndEntryImpl feedEntry;
+            feedEntry = getNextRSSFeed();
+            if (feedEntry == null) {
+                return null;
+            }
+            record.set(feedEntry);
+            return record;
+        } catch (Exception e) {
+            throw new IOException(e);
+        }
+    }
+
+    @Override
+    public boolean stop() {
+        done = true;
+        return true;
+    }
+
+    public void setModified(boolean modified) {
+        this.modified = modified;
+    }
+
+    private SyndEntryImpl getNextRSSFeed() throws Exception {
+        if (rssFeedBuffer.isEmpty()) {
+            fetchFeed();
+        }
+        if (rssFeedBuffer.isEmpty()) {
+            return null;
+        } else {
+            return rssFeedBuffer.remove();
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private void fetchFeed() throws IllegalArgumentException, IOException, FeedException, FetcherException {
+        // Retrieve the feed.
+        // We will get a Feed Polled Event and then a
+        // Feed Retrieved event (assuming the feed is valid)
+        SyndFeed feed = fetcher.retrieveFeed(feedUrl);
+        if (modified) {
+            if (LOGGER.isInfoEnabled()) {
+                LOGGER.info(feedUrl + " retrieved");
+                LOGGER.info(feedUrl + " has a title: " + feed.getTitle() + " and contains " + feed.getEntries().size()
+                        + " entries.");
+            }
+            List<? extends SyndEntryImpl> fetchedFeeds = feed.getEntries();
+            rssFeedBuffer.addAll(fetchedFeeds);
+        }
+    }
+}
+
+class FetcherEventListenerImpl implements FetcherListener {
+
+    private RSSRecordReader reader;
+    private Logger LOGGER;
+
+    public FetcherEventListenerImpl(RSSRecordReader reader, Logger LOGGER) {
+        this.reader = reader;
+        this.LOGGER = LOGGER;
+    }
+
+    /**
+     * @see com.sun.syndication.fetcher.FetcherListener#fetcherEvent(com.sun.syndication.fetcher.FetcherEvent)
+     */
+    @Override
+    public void fetcherEvent(FetcherEvent event) {
+        String eventType = event.getEventType();
+        if (FetcherEvent.EVENT_TYPE_FEED_POLLED.equals(eventType)) {
+            if (LOGGER.isInfoEnabled()) {
+                LOGGER.info("\tEVENT: Feed Polled. URL = " + event.getUrlString());
+            }
+        } else if (FetcherEvent.EVENT_TYPE_FEED_RETRIEVED.equals(eventType)) {
+            if (LOGGER.isInfoEnabled()) {
+                LOGGER.info("\tEVENT: Feed Retrieved. URL = " + event.getUrlString());
+            }
+            (reader).setModified(true);
+        } else if (FetcherEvent.EVENT_TYPE_FEED_UNCHANGED.equals(eventType)) {
+            if (LOGGER.isInfoEnabled()) {
+                LOGGER.info("\tEVENT: Feed Unchanged. URL = " + event.getUrlString());
+            }
+            (reader).setModified(true);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/rss/RSSRecordReaderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/rss/RSSRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/rss/RSSRecordReaderFactory.java
new file mode 100644
index 0000000..bbe485c
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/rss/RSSRecordReaderFactory.java
@@ -0,0 +1,86 @@
+/*
+ * 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.asterix.external.input.record.reader.rss;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.api.IRecordReaderFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+import com.sun.syndication.feed.synd.SyndEntryImpl;
+
+public class RSSRecordReaderFactory implements IRecordReaderFactory<SyndEntryImpl> {
+
+    private static final long serialVersionUID = 1L;
+    private Map<String, String> configuration;
+    private List<String> urls = new ArrayList<String>();
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return new AlgebricksCountPartitionConstraint(urls.size());
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+        String url = configuration.get(ExternalDataConstants.KEY_RSS_URL);
+        if (url == null) {
+            throw new IllegalArgumentException("no RSS URL provided");
+        }
+        initializeURLs(url);
+    }
+
+    private void initializeURLs(String url) {
+        urls.clear();
+        String[] rssURLs = url.split(",");
+        for (String rssURL : rssURLs) {
+            urls.add(rssURL);
+        }
+    }
+
+    @Override
+    public boolean isIndexible() {
+        return false;
+    }
+
+    @Override
+    public IRecordReader<? extends SyndEntryImpl> createRecordReader(IHyracksTaskContext ctx, int partition)
+            throws Exception {
+        RSSRecordReader reader = new RSSRecordReader(urls.get(partition));
+        reader.configure(configuration);
+        return reader;
+    }
+
+    @Override
+    public Class<? extends SyndEntryImpl> getRecordClass() {
+        return SyndEntryImpl.class;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/AbstractStreamRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/AbstractStreamRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/AbstractStreamRecordReader.java
new file mode 100644
index 0000000..6225b82
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/AbstractStreamRecordReader.java
@@ -0,0 +1,92 @@
+/*
+ * 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.asterix.external.input.record.reader.stream;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IDataFlowController;
+import org.apache.asterix.external.api.IExternalIndexer;
+import org.apache.asterix.external.api.IIndexingDatasource;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
+import org.apache.asterix.external.input.record.CharArrayRecord;
+import org.apache.asterix.external.input.stream.AInputStream;
+import org.apache.asterix.external.input.stream.AInputStreamReader;
+import org.apache.asterix.external.util.ExternalDataConstants;
+
+public abstract class AbstractStreamRecordReader implements IRecordReader<char[]>, IIndexingDatasource {
+    protected AInputStreamReader reader;
+    protected CharArrayRecord record;
+    protected char[] inputBuffer;
+    protected int bufferLength = 0;
+    protected int bufferPosn = 0;
+    protected IExternalIndexer indexer;
+    protected boolean done = false;
+
+    @Override
+    public IRawRecord<char[]> next() throws IOException {
+        return record;
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (!done) {
+            reader.close();
+        }
+        done = true;
+    }
+
+    public void setInputStream(AInputStream inputStream) throws IOException {
+        this.reader = new AInputStreamReader(inputStream);
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        record = new CharArrayRecord();
+        inputBuffer = new char[ExternalDataConstants.DEFAULT_BUFFER_SIZE];
+    }
+
+    @Override
+    public IExternalIndexer getIndexer() {
+        return indexer;
+    }
+
+    @Override
+    public void setIndexer(IExternalIndexer indexer) {
+        this.indexer = indexer;
+    }
+
+    @Override
+    public boolean stop() {
+        try {
+            reader.stop();
+            return true;
+        } catch (Exception e) {
+            e.printStackTrace();
+            return false;
+        }
+    }
+
+    @Override
+    public void setController(IDataFlowController controller) {
+        reader.setController((AbstractFeedDataFlowController) controller);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/AbstractStreamRecordReaderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/AbstractStreamRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/AbstractStreamRecordReaderFactory.java
new file mode 100644
index 0000000..f02bd93
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/AbstractStreamRecordReaderFactory.java
@@ -0,0 +1,99 @@
+/*
+ * 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.asterix.external.input.record.reader.stream;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IExternalIndexer;
+import org.apache.asterix.external.api.IIndexibleExternalDataSource;
+import org.apache.asterix.external.api.IIndexingDatasource;
+import org.apache.asterix.external.api.IInputStreamProvider;
+import org.apache.asterix.external.api.IInputStreamProviderFactory;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.api.IRecordReaderFactory;
+import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public abstract class AbstractStreamRecordReaderFactory<T>
+        implements IRecordReaderFactory<T>, IIndexibleExternalDataSource {
+
+    private static final long serialVersionUID = 1L;
+    protected IInputStreamProviderFactory inputStreamFactory;
+    protected Map<String, String> configuration;
+
+    public AbstractStreamRecordReaderFactory<T> setInputStreamFactoryProvider(
+            IInputStreamProviderFactory inputStreamFactory) {
+        this.inputStreamFactory = inputStreamFactory;
+        return this;
+    }
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return inputStreamFactory.getPartitionConstraint();
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.configuration = configuration;
+        inputStreamFactory.configure(configuration);
+        configureStreamReaderFactory(configuration);
+    }
+
+    protected abstract void configureStreamReaderFactory(Map<String, String> configuration) throws Exception;
+
+    @Override
+    public boolean isIndexible() {
+        return inputStreamFactory.isIndexible();
+    }
+
+    @Override
+    public void setSnapshot(List<ExternalFile> files, boolean indexingOp) throws Exception {
+        ((IIndexibleExternalDataSource) inputStreamFactory).setSnapshot(files, indexingOp);
+    }
+
+    @Override
+    public boolean isIndexingOp() {
+        if (inputStreamFactory.isIndexible()) {
+            return ((IIndexibleExternalDataSource) inputStreamFactory).isIndexingOp();
+        }
+        return false;
+    }
+
+    protected IRecordReader<char[]> configureReader(AbstractStreamRecordReader recordReader, IHyracksTaskContext ctx,
+            int partition) throws Exception {
+        IInputStreamProvider inputStreamProvider = inputStreamFactory.createInputStreamProvider(ctx, partition);
+        IExternalIndexer indexer = null;
+        if (inputStreamFactory.isIndexible()) {
+            if (((IIndexibleExternalDataSource) inputStreamFactory).isIndexingOp()) {
+                indexer = ((IIndexingDatasource) inputStreamProvider).getIndexer();
+            }
+        }
+        recordReader.setInputStream(inputStreamProvider.getInputStream());
+        recordReader.setIndexer(indexer);
+        recordReader.configure(configuration);
+        return recordReader;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReader.java
new file mode 100644
index 0000000..f8f7cda
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReader.java
@@ -0,0 +1,106 @@
+/*
+ * 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.asterix.external.input.record.reader.stream;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+
+public class LineRecordReader extends AbstractStreamRecordReader {
+
+    protected boolean prevCharCR;
+    protected int newlineLength;
+    protected int recordNumber = 0;
+
+    @Override
+    public boolean hasNext() throws IOException {
+        if (done) {
+            return false;
+        }
+        /* We're reading data from in, but the head of the stream may be
+         * already buffered in buffer, so we have several cases:
+         * 1. No newline characters are in the buffer, so we need to copy
+         *    everything and read another buffer from the stream.
+         * 2. An unambiguously terminated line is in buffer, so we just
+         *    copy to record.
+         * 3. Ambiguously terminated line is in buffer, i.e. buffer ends
+         *    in CR.  In this case we copy everything up to CR to record, but
+         *    we also need to see what follows CR: if it's LF, then we
+         *    need consume LF as well, so next call to readLine will read
+         *    from after that.
+         * We use a flag prevCharCR to signal if previous character was CR
+         * and, if it happens to be at the end of the buffer, delay
+         * consuming it until we have a chance to look at the char that
+         * follows.
+         */
+        newlineLength = 0; //length of terminating newline
+        prevCharCR = false; //true of prev char was CR
+        record.reset();
+        int readLength = 0;
+        do {
+            int startPosn = bufferPosn; //starting from where we left off the last time
+            if (bufferPosn >= bufferLength) {
+                startPosn = bufferPosn = 0;
+                bufferLength = reader.read(inputBuffer);
+                if (bufferLength <= 0) {
+                    if (readLength > 0) {
+                        record.endRecord();
+                        recordNumber++;
+                        return true;
+                    }
+                    close();
+                    return false; //EOF
+                }
+            }
+            for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline
+                if (inputBuffer[bufferPosn] == ExternalDataConstants.LF) {
+                    newlineLength = (prevCharCR) ? 2 : 1;
+                    ++bufferPosn; // at next invocation proceed from following byte
+                    break;
+                }
+                if (prevCharCR) { //CR + notLF, we are at notLF
+                    newlineLength = 1;
+                    break;
+                }
+                prevCharCR = (inputBuffer[bufferPosn] == ExternalDataConstants.CR);
+            }
+            readLength = bufferPosn - startPosn;
+            if (prevCharCR && newlineLength == 0) {
+                --readLength; //CR at the end of the buffer
+            }
+            if (readLength > 0) {
+                record.append(inputBuffer, startPosn, readLength);
+            }
+        } while (newlineLength == 0);
+        recordNumber++;
+        return true;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        super.configure(configuration);
+        if (ExternalDataUtils.hasHeader(configuration)) {
+            if (hasNext()) {
+                next();
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReaderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReaderFactory.java
new file mode 100644
index 0000000..f0867d3
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReaderFactory.java
@@ -0,0 +1,52 @@
+/*
+ * 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.asterix.external.input.record.reader.stream;
+
+import java.util.Map;
+
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class LineRecordReaderFactory extends AbstractStreamRecordReaderFactory<char[]> {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IRecordReader<? extends char[]> createRecordReader(IHyracksTaskContext ctx, int partition) throws Exception {
+        String quoteString = configuration.get(ExternalDataConstants.KEY_QUOTE);
+        LineRecordReader recordReader;
+        if (quoteString != null) {
+            recordReader = new QuotedLineRecordReader();
+        } else {
+            recordReader = new LineRecordReader();
+        }
+        return configureReader(recordReader, ctx, partition);
+    }
+
+    @Override
+    public Class<? extends char[]> getRecordClass() {
+        return char[].class;
+    }
+
+    @Override
+    protected void configureStreamReaderFactory(Map<String, String> configuration) throws Exception {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/QuotedLineRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/QuotedLineRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/QuotedLineRecordReader.java
new file mode 100644
index 0000000..a8eb07b
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/QuotedLineRecordReader.java
@@ -0,0 +1,119 @@
+/*
+ * 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.asterix.external.input.record.reader.stream;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataExceptionUtils;
+
+public class QuotedLineRecordReader extends LineRecordReader {
+
+    private char quote;
+    private boolean prevCharEscape;
+    private boolean inQuote;
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        super.configure(configuration);
+        String quoteString = configuration.get(ExternalDataConstants.KEY_QUOTE);
+        if (quoteString == null || quoteString.length() != 1) {
+            throw new AsterixException(ExternalDataExceptionUtils.incorrectParameterMessage(
+                    ExternalDataConstants.KEY_QUOTE, ExternalDataConstants.PARAMETER_OF_SIZE_ONE, quoteString));
+        }
+        this.quote = quoteString.charAt(0);
+    }
+
+    @Override
+    public boolean hasNext() throws IOException {
+        if (done) {
+            return false;
+        }
+        newlineLength = 0;
+        prevCharCR = false;
+        prevCharEscape = false;
+        record.reset();
+        int readLength = 0;
+        inQuote = false;
+        do {
+            int startPosn = bufferPosn;
+            if (bufferPosn >= bufferLength) {
+                startPosn = bufferPosn = 0;
+                bufferLength = reader.read(inputBuffer);
+                if (bufferLength <= 0) {
+                    {
+                        if (readLength > 0) {
+                            if (inQuote) {
+                                throw new IOException("malformed input record ended inside quote");
+                            }
+                            record.endRecord();
+                            recordNumber++;
+                            return true;
+                        }
+                        close();
+                        return false;
+                    }
+                }
+            }
+            for (; bufferPosn < bufferLength; ++bufferPosn) {
+                if (!inQuote) {
+                    if (inputBuffer[bufferPosn] == ExternalDataConstants.LF) {
+                        newlineLength = (prevCharCR) ? 2 : 1;
+                        ++bufferPosn;
+                        break;
+                    }
+                    if (prevCharCR) {
+                        newlineLength = 1;
+                        break;
+                    }
+                    prevCharCR = (inputBuffer[bufferPosn] == ExternalDataConstants.CR);
+                    if (inputBuffer[bufferPosn] == quote) {
+                        if (!prevCharEscape) {
+                            inQuote = true;
+                        }
+                    }
+                    if (prevCharEscape) {
+                        prevCharEscape = false;
+                    } else {
+                        prevCharEscape = inputBuffer[bufferPosn] == ExternalDataConstants.ESCAPE;
+                    }
+                } else {
+                    // only look for next quote
+                    if (inputBuffer[bufferPosn] == quote) {
+                        if (!prevCharEscape) {
+                            inQuote = false;
+                        }
+                    }
+                    prevCharEscape = inputBuffer[bufferPosn] == ExternalDataConstants.ESCAPE;
+                }
+            }
+            readLength = bufferPosn - startPosn;
+            if (prevCharCR && newlineLength == 0) {
+                --readLength;
+            }
+            if (readLength > 0) {
+                record.append(inputBuffer, startPosn, readLength);
+            }
+        } while (newlineLength == 0);
+        recordNumber++;
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReader.java
new file mode 100644
index 0000000..d469cb3
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReader.java
@@ -0,0 +1,157 @@
+/*
+ * 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.asterix.external.input.record.reader.stream;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataExceptionUtils;
+
+public class SemiStructuredRecordReader extends AbstractStreamRecordReader {
+
+    private int depth;
+    private boolean prevCharEscape;
+    private boolean inString;
+    private char recordStart;
+    private char recordEnd;
+    private int recordNumber = 0;
+
+    public int getRecordNumber() {
+        return recordNumber;
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        super.configure(configuration);
+        String recStartString = configuration.get(ExternalDataConstants.KEY_RECORD_START);
+        String recEndString = configuration.get(ExternalDataConstants.KEY_RECORD_END);
+        if (recStartString != null) {
+            if (recStartString.length() != 1) {
+                throw new AsterixException(
+                        ExternalDataExceptionUtils.incorrectParameterMessage(ExternalDataConstants.KEY_RECORD_START,
+                                ExternalDataConstants.PARAMETER_OF_SIZE_ONE, recStartString));
+            }
+            recordStart = recStartString.charAt(0);
+        } else {
+            recordStart = ExternalDataConstants.DEFAULT_RECORD_START;
+        }
+        if (recEndString != null) {
+            if (recEndString.length() != 1) {
+                throw new AsterixException(
+                        ExternalDataExceptionUtils.incorrectParameterMessage(ExternalDataConstants.KEY_RECORD_END,
+                                ExternalDataConstants.PARAMETER_OF_SIZE_ONE, recEndString));
+            }
+            recordEnd = recEndString.charAt(0);
+        } else {
+            recordEnd = ExternalDataConstants.DEFAULT_RECORD_END;
+        }
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        if (done) {
+            return false;
+        }
+        record.reset();
+        boolean hasStarted = false;
+        boolean hasFinished = false;
+        prevCharEscape = false;
+        inString = false;
+        depth = 0;
+        do {
+            int startPosn = bufferPosn; //starting from where we left off the last time
+            if (bufferPosn >= bufferLength) {
+                startPosn = bufferPosn = 0;
+                bufferLength = reader.read(inputBuffer);
+                if (bufferLength <= 0) {
+                    close();
+                    return false; // EOF
+                }
+            }
+            if (!hasStarted) {
+                for (; bufferPosn < bufferLength; ++bufferPosn) { //search for record begin
+                    if (inputBuffer[bufferPosn] == recordStart) {
+                        startPosn = bufferPosn;
+                        hasStarted = true;
+                        depth = 1;
+                        ++bufferPosn; // at next invocation proceed from following byte
+                        break;
+                    } else if (inputBuffer[bufferPosn] != ExternalDataConstants.SPACE
+                            && inputBuffer[bufferPosn] != ExternalDataConstants.TAB
+                            && inputBuffer[bufferPosn] != ExternalDataConstants.LF
+                            && inputBuffer[bufferPosn] != ExternalDataConstants.CR) {
+                        // corrupted file. clear the buffer and stop reading
+                        reader.skipError();
+                        bufferPosn = bufferLength = 0;
+                        throw new IOException("Malformed input stream");
+                    }
+                }
+            }
+            if (hasStarted) {
+                for (; bufferPosn < bufferLength; ++bufferPosn) { //search for record begin
+                    if (inString) {
+                        // we are in a string, we only care about the string end
+                        if (inputBuffer[bufferPosn] == ExternalDataConstants.QUOTE && !prevCharEscape) {
+                            inString = false;
+                        }
+                        if (prevCharEscape) {
+                            prevCharEscape = false;
+                        } else {
+                            prevCharEscape = inputBuffer[bufferPosn] == ExternalDataConstants.ESCAPE;
+                        }
+                    } else {
+                        if (inputBuffer[bufferPosn] == ExternalDataConstants.QUOTE) {
+                            inString = true;
+                        } else if (inputBuffer[bufferPosn] == recordStart) {
+                            depth += 1;
+                        } else if (inputBuffer[bufferPosn] == recordEnd) {
+                            depth -= 1;
+                            if (depth == 0) {
+                                hasFinished = true;
+                                ++bufferPosn; // at next invocation proceed from following byte
+                                break;
+                            }
+                        }
+                    }
+                }
+            }
+
+            int appendLength = bufferPosn - startPosn;
+            if (appendLength > 0) {
+                record.append(inputBuffer, startPosn, appendLength);
+            }
+        } while (!hasFinished);
+        record.endRecord();
+        recordNumber++;
+        return true;
+    }
+
+    @Override
+    public boolean stop() {
+        try {
+            reader.stop();
+        } catch (Exception e) {
+            e.printStackTrace();
+            return false;
+        }
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReaderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReaderFactory.java
new file mode 100644
index 0000000..ec8eac9
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReaderFactory.java
@@ -0,0 +1,44 @@
+/*
+ * 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.asterix.external.input.record.reader.stream;
+
+import java.util.Map;
+
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class SemiStructuredRecordReaderFactory extends AbstractStreamRecordReaderFactory<char[]> {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IRecordReader<? extends char[]> createRecordReader(IHyracksTaskContext ctx, int partition) throws Exception {
+        SemiStructuredRecordReader recordReader = new SemiStructuredRecordReader();
+        return configureReader(recordReader, ctx, partition);
+    }
+
+    @Override
+    public Class<? extends char[]> getRecordClass() {
+        return char[].class;
+    }
+
+    @Override
+    protected void configureStreamReaderFactory(Map<String, String> configuration) throws Exception {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterPullRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterPullRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterPullRecordReader.java
new file mode 100644
index 0000000..084d6d0
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterPullRecordReader.java
@@ -0,0 +1,100 @@
+/*
+ * 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.asterix.external.input.record.reader.twitter;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.input.record.GenericRecord;
+import org.apache.asterix.external.util.TwitterUtil;
+import org.apache.asterix.external.util.TwitterUtil.SearchAPIConstants;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import twitter4j.Query;
+import twitter4j.QueryResult;
+import twitter4j.Status;
+import twitter4j.Twitter;
+import twitter4j.TwitterException;
+
+public class TwitterPullRecordReader implements IRecordReader<Status> {
+
+    private String keywords;
+    private Query query;
+    private Twitter twitter;
+    private int requestInterval = 5; // seconds
+    private QueryResult result;
+    private int nextTweetIndex = 0;
+    private long lastTweetIdReceived = 0;
+    private GenericRecord<Status> record;
+
+    public TwitterPullRecordReader() {
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        twitter = TwitterUtil.getTwitterService(configuration);
+        keywords = configuration.get(SearchAPIConstants.QUERY);
+        requestInterval = Integer.parseInt(configuration.get(SearchAPIConstants.INTERVAL));
+        query = new Query(keywords);
+        query.setCount(100);
+        record = new GenericRecord<Status>();
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        return true;
+    }
+
+    @Override
+    public IRawRecord<Status> next() throws IOException, InterruptedException {
+        if (result == null || nextTweetIndex >= result.getTweets().size()) {
+            Thread.sleep(1000 * requestInterval);
+            query.setSinceId(lastTweetIdReceived);
+            try {
+                result = twitter.search(query);
+            } catch (TwitterException e) {
+                throw new HyracksDataException(e);
+            }
+            nextTweetIndex = 0;
+        }
+        if (result != null && !result.getTweets().isEmpty()) {
+            List<Status> tw = result.getTweets();
+            Status tweet = tw.get(nextTweetIndex++);
+            if (lastTweetIdReceived < tweet.getId()) {
+                lastTweetIdReceived = tweet.getId();
+            }
+            record.set(tweet);
+            return record;
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public boolean stop() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterPushRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterPushRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterPushRecordReader.java
new file mode 100644
index 0000000..764ac1d
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterPushRecordReader.java
@@ -0,0 +1,126 @@
+/*
+ * 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.asterix.external.input.record.reader.twitter;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.input.record.GenericRecord;
+import org.apache.asterix.external.util.TwitterUtil;
+
+import twitter4j.FilterQuery;
+import twitter4j.StallWarning;
+import twitter4j.Status;
+import twitter4j.StatusDeletionNotice;
+import twitter4j.StatusListener;
+import twitter4j.TwitterStream;
+
+public class TwitterPushRecordReader implements IRecordReader<Status> {
+    private LinkedBlockingQueue<Status> inputQ;
+    private TwitterStream twitterStream;
+    private GenericRecord<Status> record;
+    private boolean closed = false;
+
+    @Override
+    public void close() throws IOException {
+        if (!closed) {
+            twitterStream.clearListeners();
+            twitterStream.cleanUp();
+            twitterStream = null;
+            closed = true;
+        }
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        record = new GenericRecord<Status>();
+        inputQ = new LinkedBlockingQueue<Status>();
+        twitterStream = TwitterUtil.getTwitterStream(configuration);
+        twitterStream.addListener(new TweetListener(inputQ));
+        FilterQuery query = TwitterUtil.getFilterQuery(configuration);
+        if (query != null) {
+            twitterStream.filter(query);
+        } else {
+            twitterStream.sample();
+        }
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        return !closed;
+    }
+
+    @Override
+    public IRawRecord<Status> next() throws IOException, InterruptedException {
+        Status tweet = inputQ.poll();
+        if (tweet == null) {
+            return null;
+        }
+        record.set(tweet);
+        return record;
+    }
+
+    @Override
+    public boolean stop() {
+        try {
+            close();
+        } catch (Exception e) {
+            return false;
+        }
+        return true;
+    }
+
+    private class TweetListener implements StatusListener {
+
+        private LinkedBlockingQueue<Status> inputQ;
+
+        public TweetListener(LinkedBlockingQueue<Status> inputQ) {
+            this.inputQ = inputQ;
+        }
+
+        @Override
+        public void onStatus(Status tweet) {
+            inputQ.add(tweet);
+        }
+
+        @Override
+        public void onException(Exception arg0) {
+
+        }
+
+        @Override
+        public void onDeletionNotice(StatusDeletionNotice arg0) {
+        }
+
+        @Override
+        public void onScrubGeo(long arg0, long arg1) {
+        }
+
+        @Override
+        public void onStallWarning(StallWarning arg0) {
+        }
+
+        @Override
+        public void onTrackLimitationNotice(int arg0) {
+        }
+    }
+}


[4/5] incubator-asterixdb git commit: Add flush() to IFrameWriter

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/LookupReaderFactoryProvider.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/LookupReaderFactoryProvider.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/LookupReaderFactoryProvider.java
deleted file mode 100644
index 3a82a68..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/LookupReaderFactoryProvider.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.util.Map;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.api.ILookupReaderFactory;
-import org.apache.asterix.external.input.record.reader.factory.HDFSLookupReaderFactory;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.HDFSUtils;
-
-public class LookupReaderFactoryProvider {
-
-    @SuppressWarnings("rawtypes")
-    public static ILookupReaderFactory getLookupReaderFactory(Map<String, String> configuration) throws Exception {
-        String inputFormat = HDFSUtils.getInputFormatClassName(configuration);
-        if (inputFormat.equals(ExternalDataConstants.CLASS_NAME_TEXT_INPUT_FORMAT)
-                || inputFormat.equals(ExternalDataConstants.CLASS_NAME_SEQUENCE_INPUT_FORMAT)
-                || inputFormat.equals(ExternalDataConstants.CLASS_NAME_RC_INPUT_FORMAT)) {
-            HDFSLookupReaderFactory<Object> readerFactory = new HDFSLookupReaderFactory<Object>();
-            readerFactory.configure(configuration);
-            return readerFactory;
-        } else {
-            throw new AsterixException("Unrecognized external format");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/QuotedLineRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/QuotedLineRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/QuotedLineRecordReader.java
deleted file mode 100644
index 49e67e9..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/QuotedLineRecordReader.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataExceptionUtils;
-
-public class QuotedLineRecordReader extends LineRecordReader {
-
-    private char quote;
-    private boolean prevCharEscape;
-    private boolean inQuote;
-
-    @Override
-    public void configure(Map<String, String> configuration) throws Exception {
-        super.configure(configuration);
-        String quoteString = configuration.get(ExternalDataConstants.KEY_QUOTE);
-        if (quoteString == null || quoteString.length() != 1) {
-            throw new AsterixException(ExternalDataExceptionUtils.incorrectParameterMessage(
-                    ExternalDataConstants.KEY_QUOTE, ExternalDataConstants.PARAMETER_OF_SIZE_ONE, quoteString));
-        }
-        this.quote = quoteString.charAt(0);
-    }
-
-    @Override
-    public boolean hasNext() throws IOException {
-        if (done) {
-            return false;
-        }
-        newlineLength = 0;
-        prevCharCR = false;
-        prevCharEscape = false;
-        record.reset();
-        int readLength = 0;
-        inQuote = false;
-        do {
-            int startPosn = bufferPosn;
-            if (bufferPosn >= bufferLength) {
-                startPosn = bufferPosn = 0;
-                bufferLength = reader.read(inputBuffer);
-                if (bufferLength <= 0) {
-                    {
-                        if (readLength > 0) {
-                            if (inQuote) {
-                                throw new IOException("malformed input record ended inside quote");
-                            }
-                            record.endRecord();
-                            recordNumber++;
-                            return true;
-                        }
-                        close();
-                        return false;
-                    }
-                }
-            }
-            for (; bufferPosn < bufferLength; ++bufferPosn) {
-                if (!inQuote) {
-                    if (inputBuffer[bufferPosn] == ExternalDataConstants.LF) {
-                        newlineLength = (prevCharCR) ? 2 : 1;
-                        ++bufferPosn;
-                        break;
-                    }
-                    if (prevCharCR) {
-                        newlineLength = 1;
-                        break;
-                    }
-                    prevCharCR = (inputBuffer[bufferPosn] == ExternalDataConstants.CR);
-                    if (inputBuffer[bufferPosn] == quote) {
-                        if (!prevCharEscape) {
-                            inQuote = true;
-                        }
-                    }
-                    if (prevCharEscape) {
-                        prevCharEscape = false;
-                    } else {
-                        prevCharEscape = inputBuffer[bufferPosn] == ExternalDataConstants.ESCAPE;
-                    }
-                } else {
-                    // only look for next quote
-                    if (inputBuffer[bufferPosn] == quote) {
-                        if (!prevCharEscape) {
-                            inQuote = false;
-                        }
-                    }
-                    prevCharEscape = inputBuffer[bufferPosn] == ExternalDataConstants.ESCAPE;
-                }
-            }
-            readLength = bufferPosn - startPosn;
-            if (prevCharCR && newlineLength == 0) {
-                --readLength;
-            }
-            if (readLength > 0) {
-                record.append(inputBuffer, startPosn, readLength);
-            }
-        } while (newlineLength == 0);
-        recordNumber++;
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/RCLookupReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/RCLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/RCLookupReader.java
deleted file mode 100644
index 5c33502..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/RCLookupReader.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.io.IOException;
-
-import org.apache.asterix.external.api.IRawRecord;
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.asterix.external.indexing.RecordId;
-import org.apache.asterix.external.input.record.GenericRecord;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.ql.io.RCFile.Reader;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Writable;
-import org.apache.log4j.Logger;
-
-public class RCLookupReader extends AbstractHDFSLookupRecordReader<BytesRefArrayWritable> {
-    public RCLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs, Configuration conf) {
-        super(snapshotAccessor, fs, conf);
-    }
-
-    private static final Logger LOGGER = Logger.getLogger(RCLookupReader.class.getName());
-    private Reader reader;
-    private LongWritable key = new LongWritable();
-    private BytesRefArrayWritable value = new BytesRefArrayWritable();
-    private GenericRecord<BytesRefArrayWritable> record = new GenericRecord<BytesRefArrayWritable>();
-    private long offset;
-    private int row;
-
-    @Override
-    public Class<?> getRecordClass() throws IOException {
-        return Writable.class;
-    }
-
-    @Override
-    protected IRawRecord<BytesRefArrayWritable> lookup(RecordId rid) throws IOException {
-        if (rid.getOffset() != offset) {
-            offset = rid.getOffset();
-            if (reader.getPosition() != offset)
-                reader.seek(offset);
-            reader.resetBuffer();
-            row = -1;
-        }
-
-        // skip rows to the record row
-        while (row < rid.getRow()) {
-            reader.next(key);
-            reader.getCurrentRow(value);
-            row++;
-        }
-        record.set(value);
-        return record;
-    }
-
-    @Override
-    protected void closeFile() {
-        if (reader == null) {
-            return;
-        }
-        try {
-            reader.close();
-        } catch (Exception e) {
-            LOGGER.warn("Error closing HDFS file", e);
-        }
-    }
-
-    @Override
-    protected void openFile() throws IllegalArgumentException, IOException {
-        reader = new Reader(fs, new Path(file.getFileName()), conf);
-        offset = -1;
-        row = -1;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/RSSRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/RSSRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/RSSRecordReader.java
deleted file mode 100644
index 1c2dc30..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/RSSRecordReader.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.io.IOException;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-
-import org.apache.asterix.external.api.IRawRecord;
-import org.apache.asterix.external.api.IRecordReader;
-import org.apache.asterix.external.input.record.GenericRecord;
-import org.apache.log4j.Logger;
-
-import com.sun.syndication.feed.synd.SyndEntryImpl;
-import com.sun.syndication.feed.synd.SyndFeed;
-import com.sun.syndication.fetcher.FeedFetcher;
-import com.sun.syndication.fetcher.FetcherEvent;
-import com.sun.syndication.fetcher.FetcherException;
-import com.sun.syndication.fetcher.FetcherListener;
-import com.sun.syndication.fetcher.impl.FeedFetcherCache;
-import com.sun.syndication.fetcher.impl.HashMapFeedInfoCache;
-import com.sun.syndication.fetcher.impl.HttpURLFeedFetcher;
-import com.sun.syndication.io.FeedException;
-
-public class RSSRecordReader implements IRecordReader<SyndEntryImpl> {
-
-    private static final Logger LOGGER = Logger.getLogger(RSSRecordReader.class.getName());
-    private boolean modified = false;
-    private Queue<SyndEntryImpl> rssFeedBuffer = new LinkedList<SyndEntryImpl>();
-    private FeedFetcherCache feedInfoCache;
-    private FeedFetcher fetcher;
-    private FetcherEventListenerImpl listener;
-    private URL feedUrl;
-    private GenericRecord<SyndEntryImpl> record = new GenericRecord<SyndEntryImpl>();
-    private boolean done = false;
-
-    public RSSRecordReader(String url) throws MalformedURLException {
-        feedUrl = new URL(url);
-        feedInfoCache = HashMapFeedInfoCache.getInstance();
-        fetcher = new HttpURLFeedFetcher(feedInfoCache);
-        listener = new FetcherEventListenerImpl(this, LOGGER);
-        fetcher.addFetcherEventListener(listener);
-    }
-
-    public boolean isModified() {
-        return modified;
-    }
-
-    @Override
-    public void close() throws IOException {
-        fetcher.removeFetcherEventListener(listener);
-    }
-
-    @Override
-    public void configure(Map<String, String> configurations) throws Exception {
-    }
-
-    @Override
-    public boolean hasNext() throws Exception {
-        return !done;
-    }
-
-    @Override
-    public IRawRecord<SyndEntryImpl> next() throws IOException {
-        if (done) {
-            return null;
-        }
-        try {
-            SyndEntryImpl feedEntry;
-            feedEntry = getNextRSSFeed();
-            if (feedEntry == null) {
-                return null;
-            }
-            record.set(feedEntry);
-            return record;
-        } catch (Exception e) {
-            throw new IOException(e);
-        }
-    }
-
-    @Override
-    public Class<SyndEntryImpl> getRecordClass() throws IOException {
-        return SyndEntryImpl.class;
-    }
-
-    @Override
-    public boolean stop() {
-        done = true;
-        return true;
-    }
-
-    public void setModified(boolean modified) {
-        this.modified = modified;
-    }
-
-    private SyndEntryImpl getNextRSSFeed() throws Exception {
-        if (rssFeedBuffer.isEmpty()) {
-            fetchFeed();
-        }
-        if (rssFeedBuffer.isEmpty()) {
-            return null;
-        } else {
-            return rssFeedBuffer.remove();
-        }
-    }
-
-    @SuppressWarnings("unchecked")
-    private void fetchFeed() throws IllegalArgumentException, IOException, FeedException, FetcherException {
-        // Retrieve the feed.
-        // We will get a Feed Polled Event and then a
-        // Feed Retrieved event (assuming the feed is valid)
-        SyndFeed feed = fetcher.retrieveFeed(feedUrl);
-        if (modified) {
-            if (LOGGER.isInfoEnabled()) {
-                LOGGER.info(feedUrl + " retrieved");
-                LOGGER.info(feedUrl + " has a title: " + feed.getTitle() + " and contains " + feed.getEntries().size()
-                        + " entries.");
-            }
-            List<? extends SyndEntryImpl> fetchedFeeds = feed.getEntries();
-            rssFeedBuffer.addAll(fetchedFeeds);
-        }
-    }
-}
-
-class FetcherEventListenerImpl implements FetcherListener {
-
-    private RSSRecordReader reader;
-    private Logger LOGGER;
-
-    public FetcherEventListenerImpl(RSSRecordReader reader, Logger LOGGER) {
-        this.reader = reader;
-        this.LOGGER = LOGGER;
-    }
-
-    /**
-     * @see com.sun.syndication.fetcher.FetcherListener#fetcherEvent(com.sun.syndication.fetcher.FetcherEvent)
-     */
-    @Override
-    public void fetcherEvent(FetcherEvent event) {
-        String eventType = event.getEventType();
-        if (FetcherEvent.EVENT_TYPE_FEED_POLLED.equals(eventType)) {
-            if (LOGGER.isInfoEnabled()) {
-                LOGGER.info("\tEVENT: Feed Polled. URL = " + event.getUrlString());
-            }
-        } else if (FetcherEvent.EVENT_TYPE_FEED_RETRIEVED.equals(eventType)) {
-            if (LOGGER.isInfoEnabled()) {
-                LOGGER.info("\tEVENT: Feed Retrieved. URL = " + event.getUrlString());
-            }
-            (reader).setModified(true);
-        } else if (FetcherEvent.EVENT_TYPE_FEED_UNCHANGED.equals(eventType)) {
-            if (LOGGER.isInfoEnabled()) {
-                LOGGER.info("\tEVENT: Feed Unchanged. URL = " + event.getUrlString());
-            }
-            (reader).setModified(true);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/SemiStructuredRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/SemiStructuredRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/SemiStructuredRecordReader.java
deleted file mode 100644
index 84c96d0..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/SemiStructuredRecordReader.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataExceptionUtils;
-
-public class SemiStructuredRecordReader extends AbstractStreamRecordReader {
-
-    private int depth;
-    private boolean prevCharEscape;
-    private boolean inString;
-    private char recordStart;
-    private char recordEnd;
-    private int recordNumber = 0;
-
-    public int getRecordNumber() {
-        return recordNumber;
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration) throws Exception {
-        super.configure(configuration);
-        String recStartString = configuration.get(ExternalDataConstants.KEY_RECORD_START);
-        String recEndString = configuration.get(ExternalDataConstants.KEY_RECORD_END);
-        if (recStartString != null) {
-            if (recStartString.length() != 1) {
-                throw new AsterixException(
-                        ExternalDataExceptionUtils.incorrectParameterMessage(ExternalDataConstants.KEY_RECORD_START,
-                                ExternalDataConstants.PARAMETER_OF_SIZE_ONE, recStartString));
-            }
-            recordStart = recStartString.charAt(0);
-        } else {
-            recordStart = ExternalDataConstants.DEFAULT_RECORD_START;
-        }
-        if (recEndString != null) {
-            if (recEndString.length() != 1) {
-                throw new AsterixException(
-                        ExternalDataExceptionUtils.incorrectParameterMessage(ExternalDataConstants.KEY_RECORD_END,
-                                ExternalDataConstants.PARAMETER_OF_SIZE_ONE, recEndString));
-            }
-            recordEnd = recEndString.charAt(0);
-        } else {
-            recordEnd = ExternalDataConstants.DEFAULT_RECORD_END;
-        }
-    }
-
-    @Override
-    public boolean hasNext() throws Exception {
-        if (done) {
-            return false;
-        }
-        record.reset();
-        boolean hasStarted = false;
-        boolean hasFinished = false;
-        prevCharEscape = false;
-        inString = false;
-        depth = 0;
-        do {
-            int startPosn = bufferPosn; //starting from where we left off the last time
-            if (bufferPosn >= bufferLength) {
-                startPosn = bufferPosn = 0;
-                bufferLength = reader.read(inputBuffer);
-                if (bufferLength <= 0) {
-                    close();
-                    return false; // EOF
-                }
-            }
-            if (!hasStarted) {
-                for (; bufferPosn < bufferLength; ++bufferPosn) { //search for record begin
-                    if (inputBuffer[bufferPosn] == recordStart) {
-                        startPosn = bufferPosn;
-                        hasStarted = true;
-                        depth = 1;
-                        ++bufferPosn; // at next invocation proceed from following byte
-                        break;
-                    } else if (inputBuffer[bufferPosn] != ExternalDataConstants.SPACE
-                            && inputBuffer[bufferPosn] != ExternalDataConstants.TAB
-                            && inputBuffer[bufferPosn] != ExternalDataConstants.LF
-                            && inputBuffer[bufferPosn] != ExternalDataConstants.CR) {
-                        // corrupted file. clear the buffer and stop reading
-                        reader.skipError();
-                        bufferPosn = bufferLength = 0;
-                        throw new IOException("Malformed input stream");
-                    }
-                }
-            }
-            if (hasStarted) {
-                for (; bufferPosn < bufferLength; ++bufferPosn) { //search for record begin
-                    if (inString) {
-                        // we are in a string, we only care about the string end
-                        if (inputBuffer[bufferPosn] == ExternalDataConstants.QUOTE && !prevCharEscape) {
-                            inString = false;
-                        }
-                        if (prevCharEscape) {
-                            prevCharEscape = false;
-                        } else {
-                            prevCharEscape = inputBuffer[bufferPosn] == ExternalDataConstants.ESCAPE;
-                        }
-                    } else {
-                        if (inputBuffer[bufferPosn] == ExternalDataConstants.QUOTE) {
-                            inString = true;
-                        } else if (inputBuffer[bufferPosn] == recordStart) {
-                            depth += 1;
-                        } else if (inputBuffer[bufferPosn] == recordEnd) {
-                            depth -= 1;
-                            if (depth == 0) {
-                                hasFinished = true;
-                                ++bufferPosn; // at next invocation proceed from following byte
-                                break;
-                            }
-                        }
-                    }
-                }
-            }
-
-            int appendLength = bufferPosn - startPosn;
-            if (appendLength > 0) {
-                record.append(inputBuffer, startPosn, appendLength);
-            }
-        } while (!hasFinished);
-        record.endRecord();
-        recordNumber++;
-        return true;
-    }
-
-    @Override
-    public boolean stop() {
-        try {
-            reader.stop();
-        } catch (Exception e) {
-            e.printStackTrace();
-            return false;
-        }
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/SequenceLookupReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/SequenceLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/SequenceLookupReader.java
deleted file mode 100644
index c294ccb..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/SequenceLookupReader.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.io.IOException;
-
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.asterix.external.indexing.RecordId;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.SequenceFile.Reader;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.log4j.Logger;
-
-public class SequenceLookupReader extends AbstractCharRecordLookupReader {
-
-    public SequenceLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs, Configuration conf) {
-        super(snapshotAccessor, fs, conf);
-    }
-
-    private static final Logger LOGGER = Logger.getLogger(SequenceLookupReader.class.getName());
-    private Reader reader;
-    private Writable key;
-
-    @Override
-    protected void readRecord(RecordId rid) throws IOException {
-        reader.seek(rid.getOffset());
-        reader.next(key, value);
-    }
-
-    @Override
-    protected void closeFile() {
-        if (reader == null) {
-            return;
-        }
-        try {
-            reader.close();
-        } catch (Exception e) {
-            LOGGER.warn("Error closing HDFS file ", e);
-        }
-    }
-
-    @SuppressWarnings("deprecation")
-    @Override
-    protected void openFile() throws IllegalArgumentException, IOException {
-        reader = new SequenceFile.Reader(fs, new Path(file.getFileName()), conf);
-        key = (Writable) ReflectionUtils.newInstance(reader.getKeyClass(), conf);
-        value = (Text) ReflectionUtils.newInstance(reader.getValueClass(), conf);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TextLookupReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TextLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TextLookupReader.java
deleted file mode 100644
index b276bfa..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TextLookupReader.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.io.IOException;
-
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.asterix.external.indexing.RecordId;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.log4j.Logger;
-
-public class TextLookupReader extends AbstractCharRecordLookupReader {
-
-    public TextLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs, Configuration conf) {
-        super(snapshotAccessor, fs, conf);
-    }
-
-    private static final Logger LOGGER = Logger.getLogger(TextLookupReader.class.getName());
-    private HDFSTextLineReader reader;
-
-    @Override
-    protected void readRecord(RecordId rid) throws IOException {
-        reader.seek(rid.getOffset());
-        reader.readLine(value);
-    }
-
-    @Override
-    protected void closeFile() {
-        if (reader == null) {
-            return;
-        }
-        try {
-            reader.close();
-        } catch (Exception e) {
-            LOGGER.warn("Error closing HDFS file ", e);
-        }
-    }
-
-    @Override
-    protected void openFile() throws IllegalArgumentException, IOException {
-        if (reader == null) {
-            reader = new HDFSTextLineReader();
-        }
-        reader.resetReader(fs.open(new Path(file.getFileName())));;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TwitterPullRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TwitterPullRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TwitterPullRecordReader.java
deleted file mode 100644
index 34d8122..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TwitterPullRecordReader.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.external.api.IRawRecord;
-import org.apache.asterix.external.api.IRecordReader;
-import org.apache.asterix.external.input.record.GenericRecord;
-import org.apache.asterix.external.util.TwitterUtil;
-import org.apache.asterix.external.util.TwitterUtil.SearchAPIConstants;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import twitter4j.Query;
-import twitter4j.QueryResult;
-import twitter4j.Status;
-import twitter4j.Twitter;
-import twitter4j.TwitterException;
-
-public class TwitterPullRecordReader implements IRecordReader<Status> {
-
-    private String keywords;
-    private Query query;
-    private Twitter twitter;
-    private int requestInterval = 5; // seconds
-    private QueryResult result;
-    private int nextTweetIndex = 0;
-    private long lastTweetIdReceived = 0;
-    private GenericRecord<Status> record;
-
-    public TwitterPullRecordReader() {
-    }
-
-    @Override
-    public void close() throws IOException {
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration) throws Exception {
-        twitter = TwitterUtil.getTwitterService(configuration);
-        keywords = configuration.get(SearchAPIConstants.QUERY);
-        requestInterval = Integer.parseInt(configuration.get(SearchAPIConstants.INTERVAL));
-        query = new Query(keywords);
-        query.setCount(100);
-        record = new GenericRecord<Status>();
-    }
-
-    @Override
-    public boolean hasNext() throws Exception {
-        return true;
-    }
-
-    @Override
-    public IRawRecord<Status> next() throws IOException, InterruptedException {
-        if (result == null || nextTweetIndex >= result.getTweets().size()) {
-            Thread.sleep(1000 * requestInterval);
-            query.setSinceId(lastTweetIdReceived);
-            try {
-                result = twitter.search(query);
-            } catch (TwitterException e) {
-                throw new HyracksDataException(e);
-            }
-            nextTweetIndex = 0;
-        }
-        if (result != null && !result.getTweets().isEmpty()) {
-            List<Status> tw = result.getTweets();
-            Status tweet = tw.get(nextTweetIndex++);
-            if (lastTweetIdReceived < tweet.getId()) {
-                lastTweetIdReceived = tweet.getId();
-            }
-            record.set(tweet);
-            return record;
-        } else {
-            return null;
-        }
-    }
-
-    @Override
-    public Class<Status> getRecordClass() throws IOException {
-        return Status.class;
-    }
-
-    @Override
-    public boolean stop() {
-        return false;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TwitterPushRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TwitterPushRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TwitterPushRecordReader.java
deleted file mode 100644
index 3ce6a81..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/TwitterPushRecordReader.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.asterix.external.api.IRawRecord;
-import org.apache.asterix.external.api.IRecordReader;
-import org.apache.asterix.external.input.record.GenericRecord;
-import org.apache.asterix.external.util.TwitterUtil;
-
-import twitter4j.FilterQuery;
-import twitter4j.StallWarning;
-import twitter4j.Status;
-import twitter4j.StatusDeletionNotice;
-import twitter4j.StatusListener;
-import twitter4j.TwitterStream;
-
-public class TwitterPushRecordReader implements IRecordReader<Status> {
-    private LinkedBlockingQueue<Status> inputQ;
-    private TwitterStream twitterStream;
-    private GenericRecord<Status> record;
-    private boolean closed = false;
-
-    @Override
-    public void close() throws IOException {
-        if (!closed) {
-            twitterStream.clearListeners();
-            twitterStream.cleanUp();
-            twitterStream = null;
-            closed = true;
-        }
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration) throws Exception {
-        record = new GenericRecord<Status>();
-        inputQ = new LinkedBlockingQueue<Status>();
-        twitterStream = TwitterUtil.getTwitterStream(configuration);
-        twitterStream.addListener(new TweetListener(inputQ));
-        FilterQuery query = TwitterUtil.getFilterQuery(configuration);
-        if (query != null) {
-            twitterStream.filter(query);
-        } else {
-            twitterStream.sample();
-        }
-    }
-
-    @Override
-    public boolean hasNext() throws Exception {
-        return !closed;
-    }
-
-    @Override
-    public IRawRecord<Status> next() throws IOException, InterruptedException {
-        Status tweet = inputQ.poll();
-        if (tweet == null) {
-            return null;
-        }
-        record.set(tweet);
-        return record;
-    }
-
-    @Override
-    public Class<? extends Status> getRecordClass() throws IOException {
-        return Status.class;
-    }
-
-    @Override
-    public boolean stop() {
-        try {
-            close();
-        } catch (Exception e) {
-            return false;
-        }
-        return true;
-    }
-
-    private class TweetListener implements StatusListener {
-
-        private LinkedBlockingQueue<Status> inputQ;
-
-        public TweetListener(LinkedBlockingQueue<Status> inputQ) {
-            this.inputQ = inputQ;
-        }
-
-        @Override
-        public void onStatus(Status tweet) {
-            inputQ.add(tweet);
-        }
-
-        @Override
-        public void onException(Exception arg0) {
-
-        }
-
-        @Override
-        public void onDeletionNotice(StatusDeletionNotice arg0) {
-        }
-
-        @Override
-        public void onScrubGeo(long arg0, long arg1) {
-        }
-
-        @Override
-        public void onStallWarning(StallWarning arg0) {
-        }
-
-        @Override
-        public void onTrackLimitationNotice(int arg0) {
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/couchbase/CouchbaseReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/couchbase/CouchbaseReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/couchbase/CouchbaseReader.java
new file mode 100644
index 0000000..895af1b
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/couchbase/CouchbaseReader.java
@@ -0,0 +1,259 @@
+/*
+ * 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.asterix.external.input.record.reader.couchbase;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+
+import org.apache.asterix.external.api.IDataFlowController;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
+import org.apache.asterix.external.input.record.CharArrayRecord;
+import org.apache.asterix.external.input.record.GenericRecord;
+import org.apache.asterix.external.input.record.RecordWithMetadata;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.log4j.Logger;
+
+import com.couchbase.client.core.CouchbaseCore;
+import com.couchbase.client.core.dcp.BucketStreamAggregator;
+import com.couchbase.client.core.dcp.BucketStreamAggregatorState;
+import com.couchbase.client.core.dcp.BucketStreamState;
+import com.couchbase.client.core.dcp.BucketStreamStateUpdatedEvent;
+import com.couchbase.client.core.env.DefaultCoreEnvironment;
+import com.couchbase.client.core.env.DefaultCoreEnvironment.Builder;
+import com.couchbase.client.core.message.cluster.CloseBucketRequest;
+import com.couchbase.client.core.message.cluster.OpenBucketRequest;
+import com.couchbase.client.core.message.cluster.SeedNodesRequest;
+import com.couchbase.client.core.message.dcp.DCPRequest;
+import com.couchbase.client.core.message.dcp.MutationMessage;
+import com.couchbase.client.core.message.dcp.RemoveMessage;
+import com.couchbase.client.core.message.dcp.SnapshotMarkerMessage;
+import com.couchbase.client.deps.io.netty.buffer.ByteBuf;
+
+import rx.functions.Action1;
+
+public class CouchbaseReader implements IRecordReader<RecordWithMetadata<char[]>> {
+
+    private static final MutationMessage POISON_PILL = new MutationMessage((short) 0, null, null, 0, 0L, 0L, 0, 0, 0L,
+            null);
+    private final String feedName;
+    private final short[] vbuckets;
+    private final String bucket;
+    private final String password;
+    private final String[] couchbaseNodes;
+    private AbstractFeedDataFlowController controller;
+    private Builder builder;
+    private BucketStreamAggregator bucketStreamAggregator;
+    private CouchbaseCore core;
+    private DefaultCoreEnvironment env;
+    private Thread pushThread;
+    private ArrayBlockingQueue<MutationMessage> messages;
+    private GenericRecord<RecordWithMetadata<char[]>> record;
+    private RecordWithMetadata<char[]> recordWithMetadata;
+    private boolean done = false;
+    private CharArrayRecord value;
+    private CharsetDecoder decoder = StandardCharsets.UTF_8.newDecoder();
+    private ByteBuffer bytes = ByteBuffer.allocateDirect(ExternalDataConstants.DEFAULT_BUFFER_SIZE);
+    private CharBuffer chars = CharBuffer.allocate(ExternalDataConstants.DEFAULT_BUFFER_SIZE);
+    // metaTypes = {key(string), bucket(string), vbucket(int32), seq(long), cas(long), creationTime(long),expiration(int32),flags(int32),revSeqNumber(long),lockTime(int32)}
+    private static final IAType[] metaTypes = new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING,
+            BuiltinType.AINT32, BuiltinType.AINT64, BuiltinType.AINT64, BuiltinType.AINT64, BuiltinType.AINT32,
+            BuiltinType.AINT32, BuiltinType.AINT64, BuiltinType.AINT32 };
+    private static final Logger LOGGER = Logger.getLogger(CouchbaseReader.class);
+
+    public CouchbaseReader(String feedName, String bucket, String password, String[] couchbaseNodes, short[] vbuckets,
+            int queueSize) throws HyracksDataException {
+        this.feedName = feedName;
+        this.bucket = bucket;
+        this.password = password;
+        this.couchbaseNodes = couchbaseNodes;
+        this.vbuckets = vbuckets;
+        this.recordWithMetadata = new RecordWithMetadata<char[]>(metaTypes, char[].class);
+        this.messages = new ArrayBlockingQueue<MutationMessage>(queueSize);
+        this.value = new CharArrayRecord();
+        recordWithMetadata.setRecord(value);
+        this.record = new GenericRecord<RecordWithMetadata<char[]>>(recordWithMetadata);
+    }
+
+    @Override
+    public void close() {
+        if (!done) {
+            done = true;
+        }
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        this.builder = DefaultCoreEnvironment.builder().dcpEnabled(CouchbaseReaderFactory.DCP_ENABLED)
+                .autoreleaseAfter(CouchbaseReaderFactory.AUTO_RELEASE_AFTER_MILLISECONDS);
+        this.env = builder.build();
+        this.core = new CouchbaseCore(env);
+        this.bucketStreamAggregator = new BucketStreamAggregator(feedName, core, bucket);
+        connect();
+    }
+
+    private void connect() {
+        core.send(new SeedNodesRequest(couchbaseNodes))
+                .timeout(CouchbaseReaderFactory.TIMEOUT, CouchbaseReaderFactory.TIME_UNIT).toBlocking().single();
+        core.send(new OpenBucketRequest(bucket, password))
+                .timeout(CouchbaseReaderFactory.TIMEOUT, CouchbaseReaderFactory.TIME_UNIT).toBlocking().single();
+        this.pushThread = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                CouchbaseReader.this.run(bucketStreamAggregator);
+            }
+        }, feedName);
+        pushThread.start();
+    }
+
+    private void run(BucketStreamAggregator bucketStreamAggregator) {
+        BucketStreamAggregatorState state = new BucketStreamAggregatorState();
+        for (int i = 0; i < vbuckets.length; i++) {
+            state.put(new BucketStreamState(vbuckets[i], 0, 0, 0xffffffff, 0, 0xffffffff));
+        }
+        state.updates().subscribe(new Action1<BucketStreamStateUpdatedEvent>() {
+            @Override
+            public void call(BucketStreamStateUpdatedEvent event) {
+                if (event.partialUpdate()) {
+                } else {
+                }
+            }
+        });
+        try {
+            bucketStreamAggregator.feed(state).toBlocking().forEach(new Action1<DCPRequest>() {
+                @Override
+                public void call(final DCPRequest dcpRequest) {
+                    try {
+                        if (dcpRequest instanceof SnapshotMarkerMessage) {
+                            SnapshotMarkerMessage message = (SnapshotMarkerMessage) dcpRequest;
+                            final BucketStreamState oldState = state.get(message.partition());
+                            state.put(new BucketStreamState(message.partition(), oldState.vbucketUUID(),
+                                    message.endSequenceNumber(), oldState.endSequenceNumber(),
+                                    message.endSequenceNumber(), oldState.snapshotEndSequenceNumber()));
+                        } else if (dcpRequest instanceof MutationMessage) {
+
+                            messages.put((MutationMessage) dcpRequest);
+                        } else if (dcpRequest instanceof RemoveMessage) {
+                            RemoveMessage message = (RemoveMessage) dcpRequest;
+                            LOGGER.info(message.key() + " was deleted.");
+                        }
+                    } catch (Throwable th) {
+                        LOGGER.error(th);
+                    }
+                }
+            });
+        } catch (Throwable th) {
+            if (th.getCause() instanceof InterruptedException) {
+                LOGGER.warn("dcp thread was interrupted", th);
+                synchronized (this) {
+                    CouchbaseReader.this.close();
+                    notifyAll();
+                }
+            }
+            throw th;
+        }
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        return !done;
+    }
+
+    @Override
+    public IRawRecord<RecordWithMetadata<char[]>> next() throws IOException, InterruptedException {
+        if (messages.isEmpty()) {
+            controller.flush();
+        }
+        MutationMessage message = messages.take();
+        if (message == POISON_PILL) {
+            return null;
+        }
+        String key = message.key();
+        int vbucket = message.partition();
+        long seq = message.bySequenceNumber();
+        String bucket = message.bucket();
+        long cas = message.cas();
+        long creationTime = message.creationTime();
+        int expiration = message.expiration();
+        int flags = message.flags();
+        long revSeqNumber = message.revisionSequenceNumber();
+        int lockTime = message.lockTime();
+        recordWithMetadata.reset();
+        recordWithMetadata.setMetadata(0, key);
+        recordWithMetadata.setMetadata(1, bucket);
+        recordWithMetadata.setMetadata(2, vbucket);
+        recordWithMetadata.setMetadata(3, seq);
+        recordWithMetadata.setMetadata(4, cas);
+        recordWithMetadata.setMetadata(5, creationTime);
+        recordWithMetadata.setMetadata(6, expiration);
+        recordWithMetadata.setMetadata(7, flags);
+        recordWithMetadata.setMetadata(8, revSeqNumber);
+        recordWithMetadata.setMetadata(9, lockTime);
+        CouchbaseReader.set(message.content(), decoder, bytes, chars, value);
+        return record;
+    }
+
+    @Override
+    public boolean stop() {
+        done = true;
+        core.send(new CloseBucketRequest(bucket)).toBlocking();
+        try {
+            messages.put(CouchbaseReader.POISON_PILL);
+        } catch (InterruptedException e) {
+            LOGGER.warn(e);
+            return false;
+        }
+        return true;
+    }
+
+    @Override
+    public void setController(IDataFlowController controller) {
+        this.controller = (AbstractFeedDataFlowController) controller;
+    }
+
+    public static void set(ByteBuf content, CharsetDecoder decoder, ByteBuffer bytes, CharBuffer chars,
+            CharArrayRecord record) {
+        int position = content.readerIndex();
+        int limit = content.writerIndex();
+        int contentSize = content.capacity();
+        while (position < limit) {
+            bytes.clear();
+            chars.clear();
+            if (contentSize - position < bytes.capacity()) {
+                bytes.limit(contentSize - position);
+            }
+            content.getBytes(position, bytes);
+            position += bytes.position();
+            bytes.flip();
+            decoder.decode(bytes, chars, false);
+            chars.flip();
+            record.append(chars);
+        }
+        record.endRecord();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/couchbase/CouchbaseReaderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/couchbase/CouchbaseReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/couchbase/CouchbaseReaderFactory.java
new file mode 100644
index 0000000..b9b6f65
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/couchbase/CouchbaseReaderFactory.java
@@ -0,0 +1,153 @@
+/*
+ * 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.asterix.external.input.record.reader.couchbase;
+
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.api.IRecordReaderFactory;
+import org.apache.asterix.external.input.record.RecordWithMetadata;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.om.util.AsterixClusterProperties;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+import com.couchbase.client.core.CouchbaseCore;
+import com.couchbase.client.core.config.CouchbaseBucketConfig;
+import com.couchbase.client.core.env.DefaultCoreEnvironment;
+import com.couchbase.client.core.env.DefaultCoreEnvironment.Builder;
+import com.couchbase.client.core.message.cluster.CloseBucketRequest;
+import com.couchbase.client.core.message.cluster.GetClusterConfigRequest;
+import com.couchbase.client.core.message.cluster.GetClusterConfigResponse;
+import com.couchbase.client.core.message.cluster.OpenBucketRequest;
+import com.couchbase.client.core.message.cluster.SeedNodesRequest;
+
+import rx.functions.Func1;
+
+public class CouchbaseReaderFactory implements IRecordReaderFactory<RecordWithMetadata<char[]>> {
+
+    private static final long serialVersionUID = 1L;
+    // Constant fields
+    public static final boolean DCP_ENABLED = true;
+    public static final long AUTO_RELEASE_AFTER_MILLISECONDS = 5000L;
+    public static final int TIMEOUT = 5;
+    public static final TimeUnit TIME_UNIT = TimeUnit.SECONDS;
+    // Dynamic fields
+    private Map<String, String> configuration;
+    private String bucket;
+    private String password = "";
+    private String[] couchbaseNodes;
+    private int numOfVBuckets;
+    private int[] schedule;
+    private String feedName;
+    // Transient fields
+    private transient CouchbaseCore core;
+    private transient Builder builder;
+    private transient DefaultCoreEnvironment env;
+
+    @Override
+    public DataSourceType getDataSourceType() {
+        return DataSourceType.RECORDS;
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return AsterixClusterProperties.INSTANCE.getClusterLocations();
+    }
+
+    @Override
+    public void configure(Map<String, String> configuration) throws Exception {
+        // validate first
+        if (!configuration.containsKey(ExternalDataConstants.KEY_BUCKET)) {
+            throw new AsterixException("Unspecified bucket");
+        }
+        if (!configuration.containsKey(ExternalDataConstants.KEY_NODES)) {
+            throw new AsterixException("Unspecified Couchbase nodes");
+        }
+        if (configuration.containsKey(ExternalDataConstants.KEY_PASSWORD)) {
+            password = configuration.get(ExternalDataConstants.KEY_PASSWORD);
+        }
+        this.configuration = configuration;
+        bucket = configuration.get(ExternalDataConstants.KEY_BUCKET);
+        couchbaseNodes = configuration.get(ExternalDataConstants.KEY_NODES).split(",");
+        feedName = configuration.get(ExternalDataConstants.KEY_FEED_NAME);
+        builder = DefaultCoreEnvironment.builder().dcpEnabled(DCP_ENABLED)
+                .autoreleaseAfter(AUTO_RELEASE_AFTER_MILLISECONDS);
+        env = builder.build();
+        core = new CouchbaseCore(env);
+        getNumberOfVbuckets();
+        schedule();
+    }
+
+    /*
+     * We distribute the work of streaming vbuckets between all the partitions in a round robin fashion.
+     */
+    private void schedule() {
+        schedule = new int[numOfVBuckets];
+        String[] locations = AsterixClusterProperties.INSTANCE.getClusterLocations().getLocations();
+        for (int i = 0; i < numOfVBuckets; i++) {
+            schedule[i] = i % locations.length;
+        }
+    }
+
+    private void getNumberOfVbuckets() {
+        core.send(new SeedNodesRequest(couchbaseNodes)).timeout(TIMEOUT, TIME_UNIT).toBlocking().single();
+        core.send(new OpenBucketRequest(bucket, password)).timeout(TIMEOUT, TIME_UNIT).toBlocking().single();
+        numOfVBuckets = core.<GetClusterConfigResponse> send(new GetClusterConfigRequest())
+                .map(new Func1<GetClusterConfigResponse, Integer>() {
+                    @Override
+                    public Integer call(GetClusterConfigResponse response) {
+                        CouchbaseBucketConfig config = (CouchbaseBucketConfig) response.config().bucketConfig(bucket);
+                        return config.numberOfPartitions();
+                    }
+                }).timeout(TIMEOUT, TIME_UNIT).toBlocking().single();
+        core.send(new CloseBucketRequest(bucket)).toBlocking();
+    }
+
+    @Override
+    public IRecordReader<? extends RecordWithMetadata<char[]>> createRecordReader(IHyracksTaskContext ctx,
+            int partition) throws Exception {
+        String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+        ArrayList<Short> listOfAssignedVBuckets = new ArrayList<Short>();
+        for (int i = 0; i < schedule.length; i++) {
+            if (schedule[i] == partition) {
+                listOfAssignedVBuckets.add((short) i);
+            }
+        }
+        short[] vbuckets = new short[listOfAssignedVBuckets.size()];
+        for (int i = 0; i < vbuckets.length; i++) {
+            vbuckets[i] = listOfAssignedVBuckets.get(i);
+        }
+        CouchbaseReader reader = new CouchbaseReader(feedName + ":" + nodeName + ":" + partition, bucket, password,
+                couchbaseNodes, vbuckets, ExternalDataUtils.getQueueSize(configuration));
+        reader.configure(configuration);
+        return reader;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public Class<? extends RecordWithMetadata<char[]>> getRecordClass() {
+        RecordWithMetadata<char[]> record = new RecordWithMetadata<char[]>(char[].class);
+        return (Class<? extends RecordWithMetadata<char[]>>) record.getClass();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/HDFSLookupReaderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/HDFSLookupReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/HDFSLookupReaderFactory.java
deleted file mode 100644
index e9fad25..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/HDFSLookupReaderFactory.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader.factory;
-
-import java.util.Map;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.api.ILookupReaderFactory;
-import org.apache.asterix.external.api.ILookupRecordReader;
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.asterix.external.input.record.reader.RCLookupReader;
-import org.apache.asterix.external.input.record.reader.SequenceLookupReader;
-import org.apache.asterix.external.input.record.reader.TextLookupReader;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.HDFSUtils;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.hdfs.dataflow.ConfFactory;
-
-public class HDFSLookupReaderFactory<T> implements ILookupReaderFactory<T> {
-
-    protected static final long serialVersionUID = 1L;
-    protected transient AlgebricksPartitionConstraint clusterLocations;
-    protected ConfFactory confFactory;
-    protected Map<String, String> configuration;
-
-    public HDFSLookupReaderFactory() {
-    }
-
-    @Override
-    public DataSourceType getDataSourceType() {
-        return DataSourceType.RECORDS;
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        clusterLocations = HDFSUtils.getPartitionConstraints(clusterLocations);
-        return clusterLocations;
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration) throws Exception {
-        this.configuration = configuration;
-        JobConf conf = HDFSUtils.configureHDFSJobConf(configuration);
-        confFactory = new ConfFactory(conf);
-
-    }
-
-    @Override
-    public boolean isIndexible() {
-        return false;
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public ILookupRecordReader<? extends T> createRecordReader(IHyracksTaskContext ctx, int partition,
-            ExternalFileIndexAccessor snapshotAccessor) throws Exception {
-        String inputFormatParameter = configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT).trim();
-        JobConf conf = confFactory.getConf();
-        FileSystem fs = FileSystem.get(conf);
-        switch (inputFormatParameter) {
-            case ExternalDataConstants.INPUT_FORMAT_TEXT:
-                return (ILookupRecordReader<? extends T>) new TextLookupReader(snapshotAccessor, fs, conf);
-            case ExternalDataConstants.INPUT_FORMAT_SEQUENCE:
-                return (ILookupRecordReader<? extends T>) new SequenceLookupReader(snapshotAccessor, fs, conf);
-            case ExternalDataConstants.INPUT_FORMAT_RC:
-                return (ILookupRecordReader<? extends T>) new RCLookupReader(snapshotAccessor, fs, conf);
-            default:
-                throw new AsterixException("Unrecognised input format: " + inputFormatParameter);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/LineRecordReaderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/LineRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/LineRecordReaderFactory.java
deleted file mode 100644
index 05d419d..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/LineRecordReaderFactory.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader.factory;
-
-import java.util.Map;
-
-import org.apache.asterix.external.api.IRecordReader;
-import org.apache.asterix.external.input.record.reader.AbstractStreamRecordReaderFactory;
-import org.apache.asterix.external.input.record.reader.LineRecordReader;
-import org.apache.asterix.external.input.record.reader.QuotedLineRecordReader;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-public class LineRecordReaderFactory extends AbstractStreamRecordReaderFactory<char[]> {
-
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public IRecordReader<? extends char[]> createRecordReader(IHyracksTaskContext ctx, int partition) throws Exception {
-        String quoteString = configuration.get(ExternalDataConstants.KEY_QUOTE);
-        LineRecordReader recordReader;
-        if (quoteString != null) {
-            recordReader = new QuotedLineRecordReader();
-        } else {
-            recordReader = new LineRecordReader();
-        }
-        return configureReader(recordReader, ctx, partition);
-    }
-
-    @Override
-    public Class<? extends char[]> getRecordClass() {
-        return char[].class;
-    }
-
-    @Override
-    protected void configureStreamReaderFactory(Map<String, String> configuration) throws Exception {
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/RSSRecordReaderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/RSSRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/RSSRecordReaderFactory.java
deleted file mode 100644
index a672f2f..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/RSSRecordReaderFactory.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader.factory;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.external.api.IRecordReader;
-import org.apache.asterix.external.api.IRecordReaderFactory;
-import org.apache.asterix.external.input.record.reader.RSSRecordReader;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-import com.sun.syndication.feed.synd.SyndEntryImpl;
-
-public class RSSRecordReaderFactory implements IRecordReaderFactory<SyndEntryImpl> {
-
-    private static final long serialVersionUID = 1L;
-    private Map<String, String> configuration;
-    private List<String> urls = new ArrayList<String>();
-
-    @Override
-    public DataSourceType getDataSourceType() {
-        return DataSourceType.RECORDS;
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return new AlgebricksCountPartitionConstraint(urls.size());
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration) throws Exception {
-        this.configuration = configuration;
-        String url = configuration.get(ExternalDataConstants.KEY_RSS_URL);
-        if (url == null) {
-            throw new IllegalArgumentException("no RSS URL provided");
-        }
-        initializeURLs(url);
-    }
-
-    private void initializeURLs(String url) {
-        urls.clear();
-        String[] rssURLs = url.split(",");
-        for (String rssURL : rssURLs) {
-            urls.add(rssURL);
-        }
-    }
-
-    @Override
-    public boolean isIndexible() {
-        return false;
-    }
-
-    @Override
-    public IRecordReader<? extends SyndEntryImpl> createRecordReader(IHyracksTaskContext ctx, int partition)
-            throws Exception {
-        RSSRecordReader reader = new RSSRecordReader(urls.get(partition));
-        reader.configure(configuration);
-        return reader;
-    }
-
-    @Override
-    public Class<? extends SyndEntryImpl> getRecordClass() {
-        return SyndEntryImpl.class;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/SemiStructuredRecordReaderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/SemiStructuredRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/SemiStructuredRecordReaderFactory.java
deleted file mode 100644
index 91b439c..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/SemiStructuredRecordReaderFactory.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader.factory;
-
-import java.util.Map;
-
-import org.apache.asterix.external.api.IRecordReader;
-import org.apache.asterix.external.input.record.reader.AbstractStreamRecordReaderFactory;
-import org.apache.asterix.external.input.record.reader.SemiStructuredRecordReader;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-public class SemiStructuredRecordReaderFactory extends AbstractStreamRecordReaderFactory<char[]> {
-
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public IRecordReader<? extends char[]> createRecordReader(IHyracksTaskContext ctx, int partition) throws Exception {
-        SemiStructuredRecordReader recordReader = new SemiStructuredRecordReader();
-        return configureReader(recordReader, ctx, partition);
-    }
-
-    @Override
-    public Class<? extends char[]> getRecordClass() {
-        return char[].class;
-    }
-
-    @Override
-    protected void configureStreamReaderFactory(Map<String, String> configuration) throws Exception {
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/TwitterRecordReaderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/TwitterRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/TwitterRecordReaderFactory.java
deleted file mode 100644
index 6840c11..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/factory/TwitterRecordReaderFactory.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader.factory;
-
-import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.api.IRecordReader;
-import org.apache.asterix.external.api.IRecordReaderFactory;
-import org.apache.asterix.external.input.record.reader.TwitterPullRecordReader;
-import org.apache.asterix.external.input.record.reader.TwitterPushRecordReader;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
-import org.apache.asterix.external.util.TwitterUtil;
-import org.apache.asterix.external.util.TwitterUtil.AuthenticationConstants;
-import org.apache.asterix.external.util.TwitterUtil.SearchAPIConstants;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-import twitter4j.Status;
-
-public class TwitterRecordReaderFactory implements IRecordReaderFactory<Status> {
-
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = Logger.getLogger(TwitterRecordReaderFactory.class.getName());
-
-    private static final String DEFAULT_INTERVAL = "10"; // 10 seconds
-    private static final int INTAKE_CARDINALITY = 1; // degree of parallelism at intake stage
-
-    private Map<String, String> configuration;
-    private boolean pull;
-
-    @Override
-    public DataSourceType getDataSourceType() {
-        return DataSourceType.RECORDS;
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return new AlgebricksCountPartitionConstraint(INTAKE_CARDINALITY);
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration) throws Exception {
-        this.configuration = configuration;
-        TwitterUtil.initializeConfigurationWithAuthInfo(configuration);
-        if (!validateConfiguration(configuration)) {
-            StringBuilder builder = new StringBuilder();
-            builder.append("One or more parameters are missing from adapter configuration\n");
-            builder.append(AuthenticationConstants.OAUTH_CONSUMER_KEY + "\n");
-            builder.append(AuthenticationConstants.OAUTH_CONSUMER_SECRET + "\n");
-            builder.append(AuthenticationConstants.OAUTH_ACCESS_TOKEN + "\n");
-            builder.append(AuthenticationConstants.OAUTH_ACCESS_TOKEN_SECRET + "\n");
-            throw new Exception(builder.toString());
-        }
-        if (ExternalDataUtils.isPull(configuration)) {
-            pull = true;
-            if (configuration.get(SearchAPIConstants.QUERY) == null) {
-                throw new AsterixException(
-                        "parameter " + SearchAPIConstants.QUERY + " not specified as part of adaptor configuration");
-            }
-            String interval = configuration.get(SearchAPIConstants.INTERVAL);
-            if (interval != null) {
-                try {
-                    Integer.parseInt(interval);
-                } catch (NumberFormatException nfe) {
-                    throw new IllegalArgumentException(
-                            "parameter " + SearchAPIConstants.INTERVAL + " is defined incorrectly, expecting a number");
-                }
-            } else {
-                configuration.put(SearchAPIConstants.INTERVAL, DEFAULT_INTERVAL);
-                if (LOGGER.isLoggable(Level.WARNING)) {
-                    LOGGER.warning(" Parameter " + SearchAPIConstants.INTERVAL + " not defined, using default ("
-                            + DEFAULT_INTERVAL + ")");
-                }
-            }
-        } else if (ExternalDataUtils.isPush(configuration)) {
-            pull = false;
-        } else {
-            throw new AsterixException("One of boolean parameters " + ExternalDataConstants.KEY_PULL + " and "
-                    + ExternalDataConstants.KEY_PUSH + " must be specified as part of adaptor configuration");
-        }
-    }
-
-    @Override
-    public boolean isIndexible() {
-        return false;
-    }
-
-    @Override
-    public IRecordReader<? extends Status> createRecordReader(IHyracksTaskContext ctx, int partition) throws Exception {
-        IRecordReader<Status> reader;
-        if (pull) {
-            reader = new TwitterPullRecordReader();
-        } else {
-            reader = new TwitterPushRecordReader();
-        }
-        reader.configure(configuration);
-        return reader;
-    }
-
-    @Override
-    public Class<? extends Status> getRecordClass() {
-        return Status.class;
-    }
-
-    private boolean validateConfiguration(Map<String, String> configuration) {
-        String consumerKey = configuration.get(AuthenticationConstants.OAUTH_CONSUMER_KEY);
-        String consumerSecret = configuration.get(AuthenticationConstants.OAUTH_CONSUMER_SECRET);
-        String accessToken = configuration.get(AuthenticationConstants.OAUTH_ACCESS_TOKEN);
-        String tokenSecret = configuration.get(AuthenticationConstants.OAUTH_ACCESS_TOKEN_SECRET);
-        if (consumerKey == null || consumerSecret == null || accessToken == null || tokenSecret == null) {
-            return false;
-        }
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractCharRecordLookupReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractCharRecordLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractCharRecordLookupReader.java
new file mode 100644
index 0000000..0627660
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractCharRecordLookupReader.java
@@ -0,0 +1,79 @@
+/*
+ * 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.asterix.external.input.record.reader.hdfs;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
+import org.apache.asterix.external.indexing.RecordId;
+import org.apache.asterix.external.input.record.CharArrayRecord;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.Text;
+
+public abstract class AbstractCharRecordLookupReader extends AbstractHDFSLookupRecordReader<char[]> {
+    public AbstractCharRecordLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs,
+            Configuration conf) {
+        super(snapshotAccessor, fs, conf);
+    }
+
+    protected CharArrayRecord record = new CharArrayRecord();
+    protected Text value = new Text();
+    protected CharsetDecoder decoder = StandardCharsets.UTF_8.newDecoder();
+    protected ByteBuffer reusableByteBuffer = ByteBuffer.allocateDirect(ExternalDataConstants.DEFAULT_BUFFER_SIZE);
+    protected CharBuffer reusableCharBuffer = CharBuffer.allocate(ExternalDataConstants.DEFAULT_BUFFER_SIZE);
+
+    @Override
+    public Class<?> getRecordClass() throws IOException {
+        return char[].class;
+    }
+
+    @Override
+    protected IRawRecord<char[]> lookup(RecordId rid) throws IOException {
+        record.reset();
+        readRecord(rid);
+        writeRecord();
+        return record;
+    }
+
+    protected abstract void readRecord(RecordId rid) throws IOException;
+
+    private void writeRecord() {
+        reusableByteBuffer.clear();
+        if (reusableByteBuffer.remaining() < value.getLength()) {
+            reusableByteBuffer = ByteBuffer
+                    .allocateDirect(value.getLength() + ExternalDataConstants.DEFAULT_BUFFER_INCREMENT);
+        }
+        reusableByteBuffer.put(value.getBytes(), 0, value.getLength());
+        reusableByteBuffer.flip();
+        while (reusableByteBuffer.hasRemaining()) {
+            reusableCharBuffer.clear();
+            decoder.decode(reusableByteBuffer, reusableCharBuffer, false);
+            reusableCharBuffer.flip();
+            record.append(reusableCharBuffer);
+        }
+        record.endRecord();
+    }
+}


[5/5] incubator-asterixdb git commit: Add flush() to IFrameWriter

Posted by am...@apache.org.
Add flush() to IFrameWriter

This method is expected to be used with feeds to push
frames all the way to storage when needed. As of now, it is
needed in two cases:
1. No activities in ingestion node and need to push content
   so it can be stored.
2. When the ingestion node needs to move the checkpoint ahead
   if the at least once semantics are used.

Two feeds make use of this function. The filesystem feed and
couchbase feed which was introduced as well in this change.

Change-Id: Id862ce9e9b1360864c6976f2aea2137092f51203
Reviewed-on: https://asterix-gerrit.ics.uci.edu/585
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Murtadha Hubail <hu...@gmail.com>


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

Branch: refs/heads/master
Commit: ee387c12bc25b2b5812e639694965105eb47dfa2
Parents: e2439b4
Author: Abdullah Alamoudi <ba...@gmail.com>
Authored: Sat Jan 30 02:33:44 2016 +0300
Committer: abdullah alamoudi <ba...@gmail.com>
Committed: Fri Jan 29 15:47:27 2016 -0800

----------------------------------------------------------------------
 .../operators/physical/CommitRuntime.java       |   4 +
 .../asterix/feed/FeedMessageReceiver.java       |   7 +-
 ...ConstantTupleSourceOperatorNodePushable.java |   2 +-
 asterix-external-data/pom.xml                   |  10 +
 .../adapter/factory/LookupAdapterFactory.java   |   2 +-
 .../asterix/external/api/IDataParser.java       |   8 +
 .../api/IExternalDataSourceFactory.java         |   4 +-
 .../apache/asterix/external/api/IRawRecord.java |  11 +-
 .../asterix/external/api/IRecordReader.java     |  15 +-
 .../AbstractFeedDataFlowController.java         |   4 +
 .../dataflow/FeedRecordDataFlowController.java  |   2 +
 .../external/dataflow/FeedTupleForwarder.java   |   4 +
 .../external/dataset/adapter/LookupAdapter.java |   7 +-
 .../CollectTransformFeedFrameWriter.java        |   6 +
 .../dataflow/DistributeFeedFrameWriter.java     |   5 +
 .../external/feed/dataflow/FeedFrameCache.java  |   5 +
 .../feed/dataflow/FeedFrameCollector.java       |   9 +
 .../feed/dataflow/FeedFrameHandlers.java        |   4 +
 .../feed/dataflow/FeedRuntimeInputHandler.java  |  10 +-
 .../feed/dataflow/FrameDistributor.java         |  12 +-
 .../external/feed/message/MessageReceiver.java  |  10 +-
 .../external/feed/watch/MonitoredBuffer.java    |   5 +
 .../feed/watch/StorageSideMonitoredBuffer.java  |   4 +-
 .../external/indexing/FileOffsetIndexer.java    |   2 +-
 .../indexing/RecordColumnarIndexer.java         |   2 +-
 .../external/input/HDFSDataSourceFactory.java   |   4 +-
 .../external/input/record/CharArrayRecord.java  |  18 +-
 .../external/input/record/GenericRecord.java    |   5 -
 .../input/record/RecordWithMetadata.java        | 138 ++++++++++
 .../reader/AbstractCharRecordLookupReader.java  |  78 ------
 .../reader/AbstractHDFSLookupRecordReader.java  | 113 --------
 .../reader/AbstractStreamRecordReader.java      |  90 -------
 .../AbstractStreamRecordReaderFactory.java      |  99 -------
 .../input/record/reader/HDFSRecordReader.java   | 194 --------------
 .../input/record/reader/HDFSTextLineReader.java | 234 -----------------
 .../input/record/reader/LineRecordReader.java   | 106 --------
 .../reader/LookupReaderFactoryProvider.java     |  44 ----
 .../record/reader/QuotedLineRecordReader.java   | 119 ---------
 .../input/record/reader/RCLookupReader.java     |  92 -------
 .../input/record/reader/RSSRecordReader.java    | 177 -------------
 .../reader/SemiStructuredRecordReader.java      | 157 -----------
 .../record/reader/SequenceLookupReader.java     |  71 -----
 .../input/record/reader/TextLookupReader.java   |  64 -----
 .../record/reader/TwitterPullRecordReader.java  | 106 --------
 .../record/reader/TwitterPushRecordReader.java  | 132 ----------
 .../reader/couchbase/CouchbaseReader.java       | 259 +++++++++++++++++++
 .../couchbase/CouchbaseReaderFactory.java       | 153 +++++++++++
 .../reader/factory/HDFSLookupReaderFactory.java |  90 -------
 .../reader/factory/LineRecordReaderFactory.java |  55 ----
 .../reader/factory/RSSRecordReaderFactory.java  |  87 -------
 .../SemiStructuredRecordReaderFactory.java      |  46 ----
 .../factory/TwitterRecordReaderFactory.java     | 136 ----------
 .../hdfs/AbstractCharRecordLookupReader.java    |  79 ++++++
 .../hdfs/AbstractHDFSLookupRecordReader.java    | 113 ++++++++
 .../reader/hdfs/HDFSLookupReaderFactory.java    |  87 +++++++
 .../record/reader/hdfs/HDFSRecordReader.java    | 185 +++++++++++++
 .../record/reader/hdfs/HDFSTextLineReader.java  | 234 +++++++++++++++++
 .../record/reader/hdfs/RCLookupReader.java      |  92 +++++++
 .../reader/hdfs/SequenceLookupReader.java       |  71 +++++
 .../record/reader/hdfs/TextLookupReader.java    |  64 +++++
 .../record/reader/rss/RSSRecordReader.java      | 172 ++++++++++++
 .../reader/rss/RSSRecordReaderFactory.java      |  86 ++++++
 .../stream/AbstractStreamRecordReader.java      |  92 +++++++
 .../AbstractStreamRecordReaderFactory.java      |  99 +++++++
 .../record/reader/stream/LineRecordReader.java  | 106 ++++++++
 .../reader/stream/LineRecordReaderFactory.java  |  52 ++++
 .../reader/stream/QuotedLineRecordReader.java   | 119 +++++++++
 .../stream/SemiStructuredRecordReader.java      | 157 +++++++++++
 .../SemiStructuredRecordReaderFactory.java      |  44 ++++
 .../reader/twitter/TwitterPullRecordReader.java | 100 +++++++
 .../reader/twitter/TwitterPushRecordReader.java | 126 +++++++++
 .../twitter/TwitterRecordReaderFactory.java     | 134 ++++++++++
 .../external/input/stream/AInputStream.java     |   5 +
 .../input/stream/AInputStreamReader.java        |   6 +
 .../input/stream/HDFSInputStreamProvider.java   | 118 ---------
 .../stream/LocalFSInputStreamProvider.java      |  62 -----
 .../stream/LocalFileSystemInputStream.java      |   6 +
 .../input/stream/SocketInputStreamProvider.java |  36 ---
 .../TwitterFirehoseInputStreamProvider.java     | 183 -------------
 .../LocalFSInputStreamProviderFactory.java      |   2 +-
 .../SocketInputStreamProviderFactory.java       |   2 +-
 .../provider/HDFSInputStreamProvider.java       | 119 +++++++++
 .../provider/LocalFSInputStreamProvider.java    |  64 +++++
 .../provider/SocketInputStreamProvider.java     |  38 +++
 .../TwitterFirehoseInputStreamProvider.java     | 184 +++++++++++++
 .../ExternalLookupOperatorDescriptor.java       |   7 +-
 .../operators/FeedMetaComputeNodePushable.java  |  14 +-
 .../operators/FeedMetaNodePushable.java         |   5 +
 .../operators/FeedMetaStoreNodePushable.java    |  11 +-
 .../asterix/external/parser/ADMDataParser.java  |  40 +--
 .../external/parser/DelimitedDataParser.java    |  16 +-
 .../parser/RecordWithMetadataParser.java        | 107 ++++++++
 .../RecordWithMetadataParserFactory.java        | 100 +++++++
 .../provider/DatasourceFactoryProvider.java     |  12 +-
 .../provider/LookupReaderFactoryProvider.java   |  44 ++++
 .../provider/ParserFactoryProvider.java         |   9 +-
 .../asterix/external/util/DataflowUtils.java    |   2 +-
 .../external/util/ExternalDataConstants.java    |  21 ++
 .../external/util/ExternalDataUtils.java        |  20 +-
 .../external/util/FileSystemWatcher.java        |  12 +-
 .../apache/asterix/external/util/HDFSUtils.java |   2 +-
 .../metadata/declared/AqlMetadataProvider.java  |   2 +-
 .../metadata/feeds/FeedMetadataUtil.java        |   3 +-
 .../om/util/AsterixClusterProperties.java       |   3 +-
 104 files changed, 3697 insertions(+), 2791 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java
index 92cd61a..ce80291 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java
@@ -139,4 +139,8 @@ public class CommitRuntime implements IPushRuntime {
     public void setInputRecordDescriptor(int index, RecordDescriptor recordDescriptor) {
         this.frameTupleAccessor = new FrameTupleAccessor(recordDescriptor);
     }
+
+    @Override
+    public void flush() throws HyracksDataException {
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-app/src/main/java/org/apache/asterix/feed/FeedMessageReceiver.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/org/apache/asterix/feed/FeedMessageReceiver.java b/asterix-app/src/main/java/org/apache/asterix/feed/FeedMessageReceiver.java
index 66eca0c..4ae2e59 100644
--- a/asterix-app/src/main/java/org/apache/asterix/feed/FeedMessageReceiver.java
+++ b/asterix-app/src/main/java/org/apache/asterix/feed/FeedMessageReceiver.java
@@ -20,7 +20,6 @@ package org.apache.asterix.feed;
 
 import java.util.logging.Level;
 
-import org.json.JSONObject;
 import org.apache.asterix.external.feed.api.IFeedLoadManager;
 import org.apache.asterix.external.feed.api.IFeedTrackingManager;
 import org.apache.asterix.external.feed.api.IFeedMessage.MessageType;
@@ -35,6 +34,8 @@ import org.apache.asterix.external.feed.watch.NodeLoadReport;
 import org.apache.asterix.external.util.FeedConstants;
 import org.apache.asterix.feed.CentralFeedManager.AQLExecutor;
 import org.apache.asterix.hyracks.bootstrap.FeedBootstrap;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.json.JSONObject;
 
 public class FeedMessageReceiver extends MessageReceiver<String> {
 
@@ -88,4 +89,8 @@ public class FeedMessageReceiver extends MessageReceiver<String> {
         }
 
     }
+
+    @Override
+    public void emptyInbox() throws HyracksDataException {
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-app/src/main/java/org/apache/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/org/apache/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java b/asterix-app/src/main/java/org/apache/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
index 48bb6a9..0f705a9 100644
--- a/asterix-app/src/main/java/org/apache/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
+++ b/asterix-app/src/main/java/org/apache/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
@@ -49,7 +49,7 @@ public class ConstantTupleSourceOperatorNodePushable extends AbstractUnaryOutput
         }
         try {
             writer.open();
-            appender.flush(writer, true);
+            appender.write(writer, true);
         } catch (Throwable th) {
             writer.fail();
             throw new HyracksDataException(th);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/pom.xml
----------------------------------------------------------------------
diff --git a/asterix-external-data/pom.xml b/asterix-external-data/pom.xml
index 867c96b..7801fd7 100644
--- a/asterix-external-data/pom.xml
+++ b/asterix-external-data/pom.xml
@@ -277,5 +277,15 @@
             <version>1.2.2</version>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>com.couchbase.client</groupId>
+            <artifactId>core-io</artifactId>
+            <version>1.2.3</version>
+        </dependency>
+        <dependency>
+            <groupId>io.reactivex</groupId>
+            <artifactId>rxjava</artifactId>
+            <version>1.0.15</version>
+        </dependency>
     </dependencies>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java
index 866910b..76c8b85 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java
@@ -29,7 +29,7 @@ import org.apache.asterix.external.dataset.adapter.LookupAdapter;
 import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
 import org.apache.asterix.external.indexing.RecordIdReader;
 import org.apache.asterix.external.indexing.RecordIdReaderFactory;
-import org.apache.asterix.external.input.record.reader.LookupReaderFactoryProvider;
+import org.apache.asterix.external.provider.LookupReaderFactoryProvider;
 import org.apache.asterix.external.provider.ParserFactoryProvider;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.api.comm.IFrameWriter;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParser.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParser.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParser.java
index a4a5a43..4ad4c4f 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParser.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParser.java
@@ -37,6 +37,7 @@ import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 
@@ -124,4 +125,11 @@ public interface IDataParser {
                 break;
         }
     }
+
+    public static <T> void toBytes(T serializable, ArrayBackedValueStorage buffer, ISerializerDeserializer<T> serde)
+            throws HyracksDataException {
+        buffer.reset();
+        DataOutput out = buffer.getDataOutput();
+        serde.serialize(serializable, out);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java
index 580ac99..370ea93 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java
@@ -59,6 +59,8 @@ public interface IExternalDataSourceFactory extends Serializable {
      * Specify whether the external data source can be indexed
      * @return
      */
-    public boolean isIndexible();
+    public default boolean isIndexible() {
+        return false;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRawRecord.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRawRecord.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRawRecord.java
index 92b500d..fe15244 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRawRecord.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRawRecord.java
@@ -35,11 +35,6 @@ public interface IRawRecord<T> {
     public T get();
 
     /**
-     * @return The class of the record objects.
-     */
-    public Class<?> getRecordClass();
-
-    /**
      * Resets the object to prepare it for another write operation.
      */
     public void reset();
@@ -48,4 +43,10 @@ public interface IRawRecord<T> {
      * @return The size of the valid bytes of the object. If the object can't be serialized, this method returns -1
      */
     int size();
+
+    /**
+     * Sets the new value of the record
+     * @param t
+     */
+    public void set(T t);
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
index 019fe8f..3cf467e 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
@@ -52,15 +52,16 @@ public interface IRecordReader<T> extends Closeable {
     public IRawRecord<T> next() throws IOException, InterruptedException;
 
     /**
-     * @return the class of the java objects representing the records. used to check compatibility between readers and
-     *         parsers.
-     * @throws IOException
-     */
-    public Class<? extends T> getRecordClass() throws IOException;
-
-    /**
      * used to stop reader from producing more records.
      * @return true if the connection to the external source has been suspended, false otherwise.
      */
     public boolean stop();
+
+    /**
+     * set a pointer to the controller of the feed. the controller can be used to flush()
+     * parsed records when waiting for more records to be pushed
+     */
+    public default void setController(IDataFlowController controller) throws UnsupportedOperationException {
+        throw new UnsupportedOperationException();
+    };
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractFeedDataFlowController.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractFeedDataFlowController.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractFeedDataFlowController.java
index aab4bf6..461eaf9 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractFeedDataFlowController.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractFeedDataFlowController.java
@@ -65,4 +65,8 @@ public abstract class AbstractFeedDataFlowController implements IDataFlowControl
         tupleForwarder.resume();
         return true;
     }
+
+    public void flush() throws HyracksDataException {
+        tupleForwarder.flush();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
index fe4557d..2a4eaf9 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
@@ -43,6 +43,7 @@ public class FeedRecordDataFlowController<T> extends AbstractFeedDataFlowControl
             while (recordReader.hasNext()) {
                 IRawRecord<? extends T> record = recordReader.next();
                 if (record == null) {
+                    flush();
                     Thread.sleep(interval);
                     continue;
                 }
@@ -110,5 +111,6 @@ public class FeedRecordDataFlowController<T> extends AbstractFeedDataFlowControl
     @Override
     public void setRecordReader(IRecordReader<T> recordReader) {
         this.recordReader = recordReader;
+        recordReader.setController(this);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java
index d170766..b46a338 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java
@@ -80,4 +80,8 @@ public class FeedTupleForwarder implements ITupleForwarder {
             FrameUtils.flushFrame(frame.getBuffer(), writer);
         }
     }
+
+    public void flush() throws HyracksDataException {
+        appender.flush(writer);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java
index ba6f83c..a97182c 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java
@@ -154,9 +154,14 @@ public final class LookupAdapter<T> implements IFrameWriter {
     @Override
     public void close() throws HyracksDataException {
         try {
-            appender.flush(writer, true);
+            appender.write(writer, true);
         } finally {
             writer.close();
         }
     }
+
+    @Override
+    public void flush() throws HyracksDataException {
+        appender.flush(writer);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/CollectTransformFeedFrameWriter.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/CollectTransformFeedFrameWriter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/CollectTransformFeedFrameWriter.java
index 18b6ec0..3fc7ac8 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/CollectTransformFeedFrameWriter.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/CollectTransformFeedFrameWriter.java
@@ -63,6 +63,7 @@ public class CollectTransformFeedFrameWriter implements IFeedOperatorOutputSideH
 
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        // always project the first field only. why?
         inputFrameTupleAccessor.reset(buffer);
         int nTuple = inputFrameTupleAccessor.getTupleCount();
         for (int t = 0; t < nTuple; t++) {
@@ -116,4 +117,9 @@ public class CollectTransformFeedFrameWriter implements IFeedOperatorOutputSideH
         this.downstreamWriter = writer;
     }
 
+    @Override
+    public void flush() throws HyracksDataException {
+        tupleAppender.flush(downstreamWriter);
+    }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DistributeFeedFrameWriter.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DistributeFeedFrameWriter.java b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DistributeFeedFrameWriter.java
index 7367d5a..d314f74 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DistributeFeedFrameWriter.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DistributeFeedFrameWriter.java
@@ -156,4 +156,9 @@ public class DistributeFeedFrameWriter implements IFrameWriter {
     public FrameDistributor.DistributionMode getDistributionMode() {
         return frameDistributor.getDistributionMode();
     }
+
+    @Override
+    public void flush() throws HyracksDataException {
+        frameDistributor.flush();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCache.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCache.java b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCache.java
index 0a595b7..159bc43 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCache.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCache.java
@@ -169,4 +169,9 @@ public class FeedFrameCache extends MessageReceiver<ByteBuffer> {
             frameWriter.nextFrame(frame);
         }
     }
+
+    @Override
+    public void emptyInbox() throws HyracksDataException {
+        frameWriter.flush();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCollector.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCollector.java b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCollector.java
index 0d53524..ef4b87d 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCollector.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCollector.java
@@ -157,4 +157,13 @@ public class FeedFrameCollector extends MessageReceiver<DataBucket> {
         return connectionId.toString().hashCode();
     }
 
+    @Override
+    public void emptyInbox() throws HyracksDataException {
+        flush();
+    }
+
+    public synchronized void flush() throws HyracksDataException {
+        frameWriter.flush();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameHandlers.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameHandlers.java b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameHandlers.java
index 6ad00f1..3c45a20 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameHandlers.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameHandlers.java
@@ -272,6 +272,10 @@ public class FeedFrameHandlers {
                 };
             }
 
+            @Override
+            public void emptyInbox() throws HyracksDataException {
+            }
+
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedRuntimeInputHandler.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedRuntimeInputHandler.java b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedRuntimeInputHandler.java
index 3a46b1a..a00e732 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedRuntimeInputHandler.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedRuntimeInputHandler.java
@@ -112,7 +112,6 @@ public class FeedRuntimeInputHandler implements IFrameWriter {
         this.mBuffer = MonitoredBuffer.getMonitoredBuffer(ctx, this, coreOperator, fta, recordDesc,
                 feedManager.getFeedMetricCollector(), connectionId, runtimeId, exceptionHandler, frameEventCallback,
                 nPartitions, fpa);
-        this.mBuffer.start();
         this.throttlingEnabled = false;
     }
 
@@ -414,6 +413,7 @@ public class FeedRuntimeInputHandler implements IFrameWriter {
     @Override
     public void open() throws HyracksDataException {
         coreOperator.open();
+        mBuffer.start();
     }
 
     @Override
@@ -465,4 +465,12 @@ public class FeedRuntimeInputHandler implements IFrameWriter {
     public void setBufferingEnabled(boolean bufferingEnabled) {
         this.bufferingEnabled = bufferingEnabled;
     }
+
+    @Override
+    public void flush() throws HyracksDataException {
+        // Only flush when in process mode.
+        if (mode == Mode.PROCESS) {
+            coreOperator.flush();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameDistributor.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameDistributor.java b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameDistributor.java
index 543efb2..85308df 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameDistributor.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameDistributor.java
@@ -25,8 +25,8 @@ import java.util.Map;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.asterix.external.feed.api.IFeedMemoryManager;
 import org.apache.asterix.external.feed.api.IFeedMemoryComponent.Type;
+import org.apache.asterix.external.feed.api.IFeedMemoryManager;
 import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
 import org.apache.asterix.external.feed.management.FeedId;
 import org.apache.hyracks.api.comm.IFrameWriter;
@@ -358,4 +358,14 @@ public class FrameDistributor {
         return fta;
     }
 
+    public void flush() throws HyracksDataException {
+        switch (distributionMode) {
+            case SINGLE:
+                FeedFrameCollector collector = registeredCollectors.values().iterator().next();
+                collector.flush();
+            default:
+                break;
+        }
+    }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/MessageReceiver.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/MessageReceiver.java b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/MessageReceiver.java
index abeb994..f69c552 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/MessageReceiver.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/MessageReceiver.java
@@ -25,6 +25,7 @@ import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import org.apache.asterix.external.feed.api.IMessageReceiver;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public abstract class MessageReceiver<T> implements IMessageReceiver<T> {
 
@@ -74,12 +75,17 @@ public abstract class MessageReceiver<T> implements IMessageReceiver<T> {
             this.inbox = messageReceiver.inbox;
             this.messageReceiver = messageReceiver;
         }
+        // TODO: this should handle exceptions better
 
         @Override
         public void run() {
             while (true) {
                 try {
-                    T message = inbox.take();
+                    T message = inbox.poll();
+                    if (message == null) {
+                        messageReceiver.emptyInbox();
+                        message = inbox.take();
+                    }
                     messageReceiver.processMessage(message);
                 } catch (InterruptedException e) {
                     e.printStackTrace();
@@ -108,4 +114,6 @@ public abstract class MessageReceiver<T> implements IMessageReceiver<T> {
         }
     }
 
+    public abstract void emptyInbox() throws HyracksDataException;
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/MonitoredBuffer.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/MonitoredBuffer.java b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/MonitoredBuffer.java
index db38edf..b93410a 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/MonitoredBuffer.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/MonitoredBuffer.java
@@ -393,4 +393,9 @@ public abstract class MonitoredBuffer extends MessageReceiver<DataBucket> {
         return storageTimeTrackingRateTask;
     }
 
+    @Override
+    public void emptyInbox() throws HyracksDataException {
+        inputHandler.flush();
+    }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StorageSideMonitoredBuffer.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StorageSideMonitoredBuffer.java b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StorageSideMonitoredBuffer.java
index 1f9551d..9db930e 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StorageSideMonitoredBuffer.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StorageSideMonitoredBuffer.java
@@ -38,8 +38,7 @@ import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
 public class StorageSideMonitoredBuffer extends MonitoredBuffer {
 
-    private static final long STORAGE_TIME_TRACKING_FREQUENCY = 5000; // 10
-                                                                      // seconds
+    private static final long STORAGE_TIME_TRACKING_FREQUENCY = 5000;
 
     private boolean ackingEnabled;
     private final boolean timeTrackingEnabled;
@@ -207,5 +206,4 @@ public class StorageSideMonitoredBuffer extends MonitoredBuffer {
     protected boolean reportInflowRate() {
         return false;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileOffsetIndexer.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileOffsetIndexer.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileOffsetIndexer.java
index 932aece..0fbbd2e 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileOffsetIndexer.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/FileOffsetIndexer.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 
 import org.apache.asterix.external.api.IExternalIndexer;
 import org.apache.asterix.external.api.IRecordReader;
-import org.apache.asterix.external.input.record.reader.HDFSRecordReader;
+import org.apache.asterix.external.input.record.reader.hdfs.HDFSRecordReader;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.AMutableInt32;
 import org.apache.asterix.om.base.AMutableInt64;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordColumnarIndexer.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordColumnarIndexer.java b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordColumnarIndexer.java
index 14235c0..9fa26f0 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordColumnarIndexer.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/RecordColumnarIndexer.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 
 import org.apache.asterix.external.api.IExternalIndexer;
 import org.apache.asterix.external.api.IRecordReader;
-import org.apache.asterix.external.input.record.reader.HDFSRecordReader;
+import org.apache.asterix.external.input.record.reader.hdfs.HDFSRecordReader;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.AMutableInt32;
 import org.apache.asterix.om.base.AMutableInt64;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
index 7e9fdcb..aa35383 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
@@ -29,8 +29,8 @@ import org.apache.asterix.external.api.IRecordReader;
 import org.apache.asterix.external.api.IRecordReaderFactory;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.IndexingScheduler;
-import org.apache.asterix.external.input.record.reader.HDFSRecordReader;
-import org.apache.asterix.external.input.stream.HDFSInputStreamProvider;
+import org.apache.asterix.external.input.record.reader.hdfs.HDFSRecordReader;
+import org.apache.asterix.external.input.stream.provider.HDFSInputStreamProvider;
 import org.apache.asterix.external.provider.ExternalIndexerProvider;
 import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.external.util.HDFSUtils;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/CharArrayRecord.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/CharArrayRecord.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/CharArrayRecord.java
index fd5c397..365aeb0 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/CharArrayRecord.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/CharArrayRecord.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.external.input.record;
 
+import java.nio.CharBuffer;
 import java.util.Arrays;
 
 import org.apache.asterix.external.api.IRawRecord;
@@ -99,8 +100,21 @@ public class CharArrayRecord implements IRawRecord<char[]> {
         size++;
     }
 
+    public void append(char[] recordBuffer) {
+        ensureCapacity(size + recordBuffer.length);
+        System.arraycopy(recordBuffer, 0, value, size, recordBuffer.length);
+        size += recordBuffer.length;
+    }
+
+    public void append(CharBuffer chars) {
+        ensureCapacity(size + chars.limit());
+        chars.get(value, size, chars.limit());
+        size += chars.limit();
+    }
+
     @Override
-    public Class<char[]> getRecordClass() {
-        return char[].class;
+    public void set(char[] value) {
+        this.value = value;
+        this.size = value.length;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/GenericRecord.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/GenericRecord.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/GenericRecord.java
index 365bc22..f405b82 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/GenericRecord.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/GenericRecord.java
@@ -47,10 +47,6 @@ public class GenericRecord<T> implements IRawRecord<T> {
     }
 
     @Override
-    public Class<?> getRecordClass() {
-        return record.getClass();
-    }
-
     public void set(T record) {
         this.record = record;
     }
@@ -58,5 +54,4 @@ public class GenericRecord<T> implements IRawRecord<T> {
     @Override
     public void reset() {
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithMetadata.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithMetadata.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithMetadata.java
new file mode 100644
index 0000000..d5640a6
--- /dev/null
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithMetadata.java
@@ -0,0 +1,138 @@
+/*
+ * 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.asterix.external.input.record;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.external.api.IDataParser;
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.AMutableString;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+
+public class RecordWithMetadata<T> {
+
+    private ArrayBackedValueStorage[] fieldValueBuffers;
+    private DataOutput[] fieldValueBufferOutputs;
+    private IValueParserFactory[] valueParserFactories;
+    private byte[] fieldTypeTags;
+    private IRawRecord<T> record;
+
+    // Serializers
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<ADouble> doubleSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ADOUBLE);
+    private AMutableDouble mutableDouble = new AMutableDouble(0);
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ASTRING);
+    private AMutableString mutableString = new AMutableString(null);
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<AInt32> int32Serde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AINT32);
+    private AMutableInt32 mutableInt = new AMutableInt32(0);
+    @SuppressWarnings("unchecked")
+    protected ISerializerDeserializer<AInt64> int64Serde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AINT64);
+    private AMutableInt64 mutableLong = new AMutableInt64(0);
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ABOOLEAN);
+
+    public RecordWithMetadata(Class<? extends T> recordClass) {
+    }
+
+    public RecordWithMetadata(IAType[] metaTypes, Class<? extends T> recordClass) {
+        int n = metaTypes.length;
+        this.fieldValueBuffers = new ArrayBackedValueStorage[n];
+        this.fieldValueBufferOutputs = new DataOutput[n];
+        this.valueParserFactories = new IValueParserFactory[n];
+        this.fieldTypeTags = new byte[n];
+        for (int i = 0; i < n; i++) {
+            ATypeTag tag = metaTypes[i].getTypeTag();
+            fieldTypeTags[i] = tag.serialize();
+            fieldValueBuffers[i] = new ArrayBackedValueStorage();
+            fieldValueBufferOutputs[i] = fieldValueBuffers[i].getDataOutput();
+            valueParserFactories[i] = ExternalDataUtils.getParserFactory(tag);
+        }
+    }
+
+    public IRawRecord<T> getRecord() {
+        return record;
+    }
+
+    public ArrayBackedValueStorage getMetadata(int index) {
+        return fieldValueBuffers[index];
+    }
+
+    public void setRecord(IRawRecord<T> record) {
+        this.record = record;
+    }
+
+    public void reset() {
+        record.reset();
+        for (ArrayBackedValueStorage fieldBuffer : fieldValueBuffers) {
+            fieldBuffer.reset();
+        }
+    }
+
+    public void setMetadata(int index, int value) throws IOException {
+        fieldValueBufferOutputs[index].writeByte(fieldTypeTags[index]);
+        mutableInt.setValue(value);
+        IDataParser.toBytes(mutableInt, fieldValueBuffers[index], int32Serde);
+    }
+
+    public void setMetadata(int index, long value) throws IOException {
+        fieldValueBufferOutputs[index].writeByte(fieldTypeTags[index]);
+        mutableLong.setValue(value);
+        IDataParser.toBytes(mutableLong, fieldValueBuffers[index], int64Serde);
+    }
+
+    public void setMetadata(int index, String value) throws IOException {
+        fieldValueBufferOutputs[index].writeByte(fieldTypeTags[index]);
+        mutableString.setValue(value);
+        IDataParser.toBytes(mutableString, fieldValueBuffers[index], stringSerde);
+    }
+
+    public void setMeta(int index, boolean value) throws IOException {
+        fieldValueBufferOutputs[index].writeByte(fieldTypeTags[index]);
+        IDataParser.toBytes(value ? ABoolean.TRUE : ABoolean.FALSE, fieldValueBuffers[index], booleanSerde);
+    }
+
+    public void setMeta(int index, double value) throws IOException {
+        fieldValueBufferOutputs[index].writeByte(fieldTypeTags[index]);
+        mutableDouble.setValue(value);
+        IDataParser.toBytes(mutableDouble, fieldValueBuffers[index], doubleSerde);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractCharRecordLookupReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractCharRecordLookupReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractCharRecordLookupReader.java
deleted file mode 100644
index 1b84e7a..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractCharRecordLookupReader.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.CharBuffer;
-import java.nio.charset.CharsetDecoder;
-import java.nio.charset.StandardCharsets;
-
-import org.apache.asterix.external.api.IRawRecord;
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.asterix.external.indexing.RecordId;
-import org.apache.asterix.external.input.record.CharArrayRecord;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.io.Text;
-
-public abstract class AbstractCharRecordLookupReader extends AbstractHDFSLookupRecordReader<char[]> {
-    public AbstractCharRecordLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs,
-            Configuration conf) {
-        super(snapshotAccessor, fs, conf);
-    }
-
-    protected CharArrayRecord record = new CharArrayRecord();
-    protected Text value = new Text();
-    protected CharsetDecoder decoder = StandardCharsets.UTF_8.newDecoder();
-    protected ByteBuffer reusableByteBuffer = ByteBuffer.allocateDirect(ExternalDataConstants.DEFAULT_BUFFER_SIZE);
-    protected CharBuffer reusableCharBuffer = CharBuffer.allocate(ExternalDataConstants.DEFAULT_BUFFER_SIZE);
-
-    @Override
-    public Class<?> getRecordClass() throws IOException {
-        return char[].class;
-    }
-
-    @Override
-    protected IRawRecord<char[]> lookup(RecordId rid) throws IOException {
-        record.reset();
-        readRecord(rid);
-        writeRecord();
-        return record;
-    }
-
-    protected abstract void readRecord(RecordId rid) throws IOException;
-
-    private void writeRecord() {
-        reusableByteBuffer.clear();
-        if (reusableByteBuffer.remaining() < value.getLength()) {
-            reusableByteBuffer = ByteBuffer
-                    .allocateDirect(value.getLength() + ExternalDataConstants.DEFAULT_BUFFER_INCREMENT);
-        }
-        reusableByteBuffer.put(value.getBytes(), 0, value.getLength());
-        reusableByteBuffer.flip();
-        while (reusableByteBuffer.hasRemaining()) {
-            decoder.decode(reusableByteBuffer, reusableCharBuffer, false);
-            record.append(reusableCharBuffer.array(), 0, reusableCharBuffer.position());
-            reusableCharBuffer.clear();
-        }
-        record.endRecord();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractHDFSLookupRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractHDFSLookupRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractHDFSLookupRecordReader.java
deleted file mode 100644
index 5a20962..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractHDFSLookupRecordReader.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.asterix.external.api.ILookupRecordReader;
-import org.apache.asterix.external.api.IRawRecord;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.asterix.external.indexing.RecordId;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public abstract class AbstractHDFSLookupRecordReader<T> implements ILookupRecordReader<T> {
-
-    protected int fileId;
-    private ExternalFileIndexAccessor snapshotAccessor;
-    protected ExternalFile file;
-    protected FileSystem fs;
-    protected Configuration conf;
-    protected boolean replaced;
-
-    public AbstractHDFSLookupRecordReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs,
-            Configuration conf) {
-        this.snapshotAccessor = snapshotAccessor;
-        this.fs = fs;
-        this.conf = conf;
-        this.fileId = -1;
-        this.file = new ExternalFile();
-    }
-
-    @Override
-    public void configure(Map<String, String> configurations) throws Exception {
-    }
-
-    @Override
-    public IRawRecord<T> read(RecordId rid) throws Exception {
-        if (rid.getFileId() != fileId) {
-            // close current file
-            closeFile();
-            // lookup new file
-            snapshotAccessor.lookup(rid.getFileId(), file);
-            fileId = rid.getFileId();
-            try {
-                validate();
-                if (!replaced) {
-                    openFile();
-                    validate();
-                    if (replaced) {
-                        closeFile();
-                    }
-                }
-            } catch (FileNotFoundException e) {
-                replaced = true;
-            }
-        }
-        if (replaced) {
-            return null;
-        }
-        return lookup(rid);
-    }
-
-    protected abstract IRawRecord<T> lookup(RecordId rid) throws IOException;
-
-    private void validate() throws IllegalArgumentException, IOException {
-        FileStatus fileStatus = fs.getFileStatus(new Path(file.getFileName()));
-        replaced = fileStatus.getModificationTime() != file.getLastModefiedTime().getTime();
-    }
-
-    protected abstract void closeFile();
-
-    protected abstract void openFile() throws IllegalArgumentException, IOException;
-
-    @Override
-    public final void open() throws HyracksDataException {
-        snapshotAccessor.open();
-    }
-
-    @Override
-    public void close() throws IOException {
-        try {
-            closeFile();
-        } finally {
-            snapshotAccessor.close();
-        }
-    }
-
-    @Override
-    public void fail() {
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractStreamRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractStreamRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractStreamRecordReader.java
deleted file mode 100644
index 93ba0a0..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractStreamRecordReader.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.asterix.external.api.IExternalIndexer;
-import org.apache.asterix.external.api.IIndexingDatasource;
-import org.apache.asterix.external.api.IRawRecord;
-import org.apache.asterix.external.api.IRecordReader;
-import org.apache.asterix.external.input.record.CharArrayRecord;
-import org.apache.asterix.external.input.stream.AInputStream;
-import org.apache.asterix.external.input.stream.AInputStreamReader;
-import org.apache.asterix.external.util.ExternalDataConstants;
-
-public abstract class AbstractStreamRecordReader implements IRecordReader<char[]>, IIndexingDatasource {
-    protected AInputStreamReader reader;
-    protected CharArrayRecord record;
-    protected char[] inputBuffer;
-    protected int bufferLength = 0;
-    protected int bufferPosn = 0;
-    protected IExternalIndexer indexer;
-    protected boolean done = false;
-
-    @Override
-    public IRawRecord<char[]> next() throws IOException {
-        return record;
-    }
-
-    @Override
-    public void close() throws IOException {
-        if (!done) {
-            reader.close();
-        }
-        done = true;
-    }
-
-    public void setInputStream(AInputStream inputStream) throws IOException {
-        this.reader = new AInputStreamReader(inputStream);
-    }
-
-    @Override
-    public Class<char[]> getRecordClass() {
-        return char[].class;
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration) throws Exception {
-        record = new CharArrayRecord();
-        inputBuffer = new char[ExternalDataConstants.DEFAULT_BUFFER_SIZE];
-    }
-
-    @Override
-    public IExternalIndexer getIndexer() {
-        return indexer;
-    }
-
-    @Override
-    public void setIndexer(IExternalIndexer indexer) {
-        this.indexer = indexer;
-    }
-
-    @Override
-    public boolean stop() {
-        try {
-            reader.stop();
-            return true;
-        } catch (Exception e) {
-            e.printStackTrace();
-            return false;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractStreamRecordReaderFactory.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractStreamRecordReaderFactory.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractStreamRecordReaderFactory.java
deleted file mode 100644
index c7acb1a..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/AbstractStreamRecordReaderFactory.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.external.api.IExternalIndexer;
-import org.apache.asterix.external.api.IIndexibleExternalDataSource;
-import org.apache.asterix.external.api.IIndexingDatasource;
-import org.apache.asterix.external.api.IInputStreamProvider;
-import org.apache.asterix.external.api.IInputStreamProviderFactory;
-import org.apache.asterix.external.api.IRecordReader;
-import org.apache.asterix.external.api.IRecordReaderFactory;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-
-public abstract class AbstractStreamRecordReaderFactory<T>
-        implements IRecordReaderFactory<T>, IIndexibleExternalDataSource {
-
-    private static final long serialVersionUID = 1L;
-    protected IInputStreamProviderFactory inputStreamFactory;
-    protected Map<String, String> configuration;
-
-    public AbstractStreamRecordReaderFactory<T> setInputStreamFactoryProvider(
-            IInputStreamProviderFactory inputStreamFactory) {
-        this.inputStreamFactory = inputStreamFactory;
-        return this;
-    }
-
-    @Override
-    public DataSourceType getDataSourceType() {
-        return DataSourceType.RECORDS;
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return inputStreamFactory.getPartitionConstraint();
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration) throws Exception {
-        this.configuration = configuration;
-        inputStreamFactory.configure(configuration);
-        configureStreamReaderFactory(configuration);
-    }
-
-    protected abstract void configureStreamReaderFactory(Map<String, String> configuration) throws Exception;
-
-    @Override
-    public boolean isIndexible() {
-        return inputStreamFactory.isIndexible();
-    }
-
-    @Override
-    public void setSnapshot(List<ExternalFile> files, boolean indexingOp) throws Exception {
-        ((IIndexibleExternalDataSource) inputStreamFactory).setSnapshot(files, indexingOp);
-    }
-
-    @Override
-    public boolean isIndexingOp() {
-        if (inputStreamFactory.isIndexible()) {
-            return ((IIndexibleExternalDataSource) inputStreamFactory).isIndexingOp();
-        }
-        return false;
-    }
-
-    protected IRecordReader<char[]> configureReader(AbstractStreamRecordReader recordReader, IHyracksTaskContext ctx,
-            int partition) throws Exception {
-        IInputStreamProvider inputStreamProvider = inputStreamFactory.createInputStreamProvider(ctx, partition);
-        IExternalIndexer indexer = null;
-        if (inputStreamFactory.isIndexible()) {
-            if (((IIndexibleExternalDataSource) inputStreamFactory).isIndexingOp()) {
-                indexer = ((IIndexingDatasource) inputStreamProvider).getIndexer();
-            }
-        }
-        recordReader.setInputStream(inputStreamProvider.getInputStream());
-        recordReader.setIndexer(indexer);
-        recordReader.configure(configuration);
-        return recordReader;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/HDFSRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/HDFSRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/HDFSRecordReader.java
deleted file mode 100644
index d88f967..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/HDFSRecordReader.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.external.api.IExternalIndexer;
-import org.apache.asterix.external.api.IIndexingDatasource;
-import org.apache.asterix.external.api.IRawRecord;
-import org.apache.asterix.external.api.IRecordReader;
-import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.input.record.GenericRecord;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class HDFSRecordReader<K, V extends Writable> implements IRecordReader<Writable>, IIndexingDatasource {
-
-    protected RecordReader<K, Writable> reader;
-    protected V value = null;
-    protected K key = null;
-    protected int currentSplitIndex = 0;
-    protected boolean read[];
-    protected InputFormat<?, ?> inputFormat;
-    protected InputSplit[] inputSplits;
-    protected String[] readSchedule;
-    protected String nodeName;
-    protected JobConf conf;
-    protected GenericRecord<Writable> record;
-    // Indexing variables
-    protected IExternalIndexer indexer;
-    protected List<ExternalFile> snapshot;
-    protected FileSystem hdfs;
-
-    public HDFSRecordReader(boolean read[], InputSplit[] inputSplits, String[] readSchedule, String nodeName,
-            JobConf conf) {
-        this.read = read;
-        this.inputSplits = inputSplits;
-        this.readSchedule = readSchedule;
-        this.nodeName = nodeName;
-        this.conf = conf;
-        this.inputFormat = conf.getInputFormat();
-        this.reader = new EmptyRecordReader<K, Writable>();
-    }
-
-    @Override
-    public void close() throws IOException {
-        reader.close();
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration) throws Exception {
-        record = new GenericRecord<Writable>();
-        nextInputSplit();
-    }
-
-    @Override
-    public boolean hasNext() throws Exception {
-        if (reader.next(key, value)) {
-            return true;
-        }
-        while (nextInputSplit()) {
-            if (reader.next(key, value)) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    @Override
-    public IRawRecord<Writable> next() throws IOException {
-        record.set(value);
-        return record;
-    }
-
-    @Override
-    public Class<? extends Writable> getRecordClass() throws IOException {
-        if (value == null) {
-            if (!nextInputSplit()) {
-                return null;
-            }
-        }
-        return value.getClass();
-    }
-
-    private boolean nextInputSplit() throws IOException {
-        for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-            /**
-             * read all the partitions scheduled to the current node
-             */
-            if (readSchedule[currentSplitIndex].equals(nodeName)) {
-                /**
-                 * pick an unread split to read synchronize among
-                 * simultaneous partitions in the same machine
-                 */
-                synchronized (read) {
-                    if (read[currentSplitIndex] == false) {
-                        read[currentSplitIndex] = true;
-                    } else {
-                        continue;
-                    }
-                }
-                if (snapshot != null) {
-                    String fileName = ((FileSplit) (inputSplits[currentSplitIndex])).getPath().toUri().getPath();
-                    FileStatus fileStatus = hdfs.getFileStatus(new Path(fileName));
-                    // Skip if not the same file stored in the files snapshot
-                    if (fileStatus.getModificationTime() != snapshot.get(currentSplitIndex).getLastModefiedTime()
-                            .getTime())
-                        continue;
-                }
-
-                reader.close();
-                reader = getRecordReader(currentSplitIndex);
-                return true;
-            }
-        }
-        return false;
-    }
-
-    @SuppressWarnings("unchecked")
-    private RecordReader<K, Writable> getRecordReader(int splitIndex) throws IOException {
-        reader = (RecordReader<K, Writable>) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
-        if (key == null) {
-            key = reader.createKey();
-            value = (V) reader.createValue();
-        }
-        if (indexer != null) {
-            try {
-                indexer.reset(this);
-            } catch (Exception e) {
-                throw new HyracksDataException(e);
-            }
-        }
-        return reader;
-    }
-
-    @Override
-    public boolean stop() {
-        return false;
-    }
-
-    @Override
-    public IExternalIndexer getIndexer() {
-        return indexer;
-    }
-
-    @Override
-    public void setIndexer(IExternalIndexer indexer) {
-        this.indexer = indexer;
-    }
-
-    public List<ExternalFile> getSnapshot() {
-        return snapshot;
-    }
-
-    public void setSnapshot(List<ExternalFile> snapshot) throws IOException {
-        this.snapshot = snapshot;
-        hdfs = FileSystem.get(conf);
-    }
-
-    public int getCurrentSplitIndex() {
-        return currentSplitIndex;
-    }
-
-    public RecordReader<K, Writable> getReader() {
-        return reader;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/HDFSTextLineReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/HDFSTextLineReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/HDFSTextLineReader.java
deleted file mode 100644
index ea851a5..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/HDFSTextLineReader.java
+++ /dev/null
@@ -1,234 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.io.Text;
-
-public class HDFSTextLineReader {
-    private static final int DEFAULT_BUFFER_SIZE = 32 * 1024;
-    private int bufferSize = DEFAULT_BUFFER_SIZE;
-    private FSDataInputStream reader;
-
-    private byte[] buffer;
-    // the number of bytes of real data in the buffer
-    private int bufferLength = 0;
-    // the current position in the buffer
-    private int bufferPosn = 0;
-
-    private long currentFilePos = 0L;
-
-    private static final byte CR = '\r';
-    private static final byte LF = '\n';
-
-    public static final String KEY_BUFFER_SIZE = "io.file.buffer.size";
-
-    /**
-     * Create a line reader that reads from the given stream using the
-     * default buffer-size (32k).
-     *
-     * @param in
-     *            The input stream
-     * @throws IOException
-     */
-    public HDFSTextLineReader(FSDataInputStream in) throws IOException {
-        this(in, DEFAULT_BUFFER_SIZE);
-    }
-
-    /**
-     * Create a line reader that reads from the given stream using the
-     * given buffer-size.
-     *
-     * @param in
-     *            The input stream
-     * @param bufferSize
-     *            Size of the read buffer
-     * @throws IOException
-     */
-    public HDFSTextLineReader(FSDataInputStream in, int bufferSize) throws IOException {
-        this.reader = in;
-        this.bufferSize = bufferSize;
-        this.buffer = new byte[this.bufferSize];
-        currentFilePos = in.getPos();
-    }
-
-    public HDFSTextLineReader() throws IOException {
-        this.bufferSize = DEFAULT_BUFFER_SIZE;
-        this.buffer = new byte[this.bufferSize];
-    }
-
-    /**
-     * Create a line reader that reads from the given stream using the <code>io.file.buffer.size</code> specified in the given <code>Configuration</code>.
-     *
-     * @param in
-     *            input stream
-     * @param conf
-     *            configuration
-     * @throws IOException
-     */
-    public HDFSTextLineReader(FSDataInputStream in, Configuration conf) throws IOException {
-        this(in, conf.getInt(KEY_BUFFER_SIZE, DEFAULT_BUFFER_SIZE));
-    }
-
-    /**
-     * Read one line from the InputStream into the given Text. A line
-     * can be terminated by one of the following: '\n' (LF) , '\r' (CR),
-     * or '\r\n' (CR+LF). EOF also terminates an otherwise unterminated
-     * line.
-     *
-     * @param str
-     *            the object to store the given line (without newline)
-     * @param maxLineLength
-     *            the maximum number of bytes to store into str;
-     *            the rest of the line is silently discarded.
-     * @param maxBytesToConsume
-     *            the maximum number of bytes to consume
-     *            in this call. This is only a hint, because if the line cross
-     *            this threshold, we allow it to happen. It can overshoot
-     *            potentially by as much as one buffer length.
-     * @return the number of bytes read including the (longest) newline
-     *         found.
-     * @throws IOException
-     *             if the underlying stream throws
-     */
-    public int readLine(Text str, int maxLineLength, int maxBytesToConsume) throws IOException {
-        /* We're reading data from in, but the head of the stream may be
-         * already buffered in buffer, so we have several cases:
-         * 1. No newline characters are in the buffer, so we need to copy
-         *    everything and read another buffer from the stream.
-         * 2. An unambiguously terminated line is in buffer, so we just
-         *    copy to str.
-         * 3. Ambiguously terminated line is in buffer, i.e. buffer ends
-         *    in CR.  In this case we copy everything up to CR to str, but
-         *    we also need to see what follows CR: if it's LF, then we
-         *    need consume LF as well, so next call to readLine will read
-         *    from after that.
-         * We use a flag prevCharCR to signal if previous character was CR
-         * and, if it happens to be at the end of the buffer, delay
-         * consuming it until we have a chance to look at the char that
-         * follows.
-         */
-        str.clear();
-        int txtLength = 0; //tracks str.getLength(), as an optimization
-        int newlineLength = 0; //length of terminating newline
-        boolean prevCharCR = false; //true of prev char was CR
-        long bytesConsumed = 0;
-        do {
-            int startPosn = bufferPosn; //starting from where we left off the last time
-            if (bufferPosn >= bufferLength) {
-                startPosn = bufferPosn = 0;
-                if (prevCharCR)
-                    ++bytesConsumed; //account for CR from previous read
-                bufferLength = reader.read(buffer);
-                if (bufferLength <= 0)
-                    break; // EOF
-            }
-            for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline
-                if (buffer[bufferPosn] == LF) {
-                    newlineLength = (prevCharCR) ? 2 : 1;
-                    ++bufferPosn; // at next invocation proceed from following byte
-                    break;
-                }
-                if (prevCharCR) { //CR + notLF, we are at notLF
-                    newlineLength = 1;
-                    break;
-                }
-                prevCharCR = (buffer[bufferPosn] == CR);
-            }
-            int readLength = bufferPosn - startPosn;
-            if (prevCharCR && newlineLength == 0)
-                --readLength; //CR at the end of the buffer
-            bytesConsumed += readLength;
-            int appendLength = readLength - newlineLength;
-            if (appendLength > maxLineLength - txtLength) {
-                appendLength = maxLineLength - txtLength;
-            }
-            if (appendLength > 0) {
-                str.append(buffer, startPosn, appendLength);
-                txtLength += appendLength;
-            }
-        } while (newlineLength == 0 && bytesConsumed < maxBytesToConsume);
-
-        if (bytesConsumed > Integer.MAX_VALUE)
-            throw new IOException("Too many bytes before newline: " + bytesConsumed);
-        currentFilePos = reader.getPos() - bufferLength + bufferPosn;
-        return (int) bytesConsumed;
-    }
-
-    /**
-     * Read from the InputStream into the given Text.
-     *
-     * @param str
-     *            the object to store the given line
-     * @param maxLineLength
-     *            the maximum number of bytes to store into str.
-     * @return the number of bytes read including the newline
-     * @throws IOException
-     *             if the underlying stream throws
-     */
-    public int readLine(Text str, int maxLineLength) throws IOException {
-        return readLine(str, maxLineLength, Integer.MAX_VALUE);
-    }
-
-    /**
-     * Read from the InputStream into the given Text.
-     *
-     * @param str
-     *            the object to store the given line
-     * @return the number of bytes read including the newline
-     * @throws IOException
-     *             if the underlying stream throws
-     */
-    public int readLine(Text str) throws IOException {
-        return readLine(str, Integer.MAX_VALUE, Integer.MAX_VALUE);
-    }
-
-    public void seek(long desired) throws IOException {
-        if (reader.getPos() <= desired || currentFilePos > desired) {
-            // desired position is ahead of stream or before the current position, seek to position
-            reader.seek(desired);
-            bufferLength = 0;
-            bufferPosn = 0;
-            currentFilePos = desired;
-        } else if (currentFilePos < desired) {
-            // desired position is in the buffer
-            int difference = (int) (desired - currentFilePos);
-            bufferPosn += difference;
-            currentFilePos = desired;
-        }
-    }
-
-    public FSDataInputStream getReader() {
-        return reader;
-    }
-
-    public void resetReader(FSDataInputStream reader) throws IOException {
-        this.reader = reader;
-        bufferLength = 0;
-        bufferPosn = 0;
-        currentFilePos = reader.getPos();
-    }
-
-    public void close() throws IOException {
-        reader.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/ee387c12/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/LineRecordReader.java
----------------------------------------------------------------------
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/LineRecordReader.java b/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/LineRecordReader.java
deleted file mode 100644
index 2b33d7a..0000000
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/LineRecordReader.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * 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.asterix.external.input.record.reader;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
-
-public class LineRecordReader extends AbstractStreamRecordReader {
-
-    protected boolean prevCharCR;
-    protected int newlineLength;
-    protected int recordNumber = 0;
-
-    @Override
-    public boolean hasNext() throws IOException {
-        if (done) {
-            return false;
-        }
-        /* We're reading data from in, but the head of the stream may be
-         * already buffered in buffer, so we have several cases:
-         * 1. No newline characters are in the buffer, so we need to copy
-         *    everything and read another buffer from the stream.
-         * 2. An unambiguously terminated line is in buffer, so we just
-         *    copy to record.
-         * 3. Ambiguously terminated line is in buffer, i.e. buffer ends
-         *    in CR.  In this case we copy everything up to CR to record, but
-         *    we also need to see what follows CR: if it's LF, then we
-         *    need consume LF as well, so next call to readLine will read
-         *    from after that.
-         * We use a flag prevCharCR to signal if previous character was CR
-         * and, if it happens to be at the end of the buffer, delay
-         * consuming it until we have a chance to look at the char that
-         * follows.
-         */
-        newlineLength = 0; //length of terminating newline
-        prevCharCR = false; //true of prev char was CR
-        record.reset();
-        int readLength = 0;
-        do {
-            int startPosn = bufferPosn; //starting from where we left off the last time
-            if (bufferPosn >= bufferLength) {
-                startPosn = bufferPosn = 0;
-                bufferLength = reader.read(inputBuffer);
-                if (bufferLength <= 0) {
-                    if (readLength > 0) {
-                        record.endRecord();
-                        recordNumber++;
-                        return true;
-                    }
-                    close();
-                    return false; //EOF
-                }
-            }
-            for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline
-                if (inputBuffer[bufferPosn] == ExternalDataConstants.LF) {
-                    newlineLength = (prevCharCR) ? 2 : 1;
-                    ++bufferPosn; // at next invocation proceed from following byte
-                    break;
-                }
-                if (prevCharCR) { //CR + notLF, we are at notLF
-                    newlineLength = 1;
-                    break;
-                }
-                prevCharCR = (inputBuffer[bufferPosn] == ExternalDataConstants.CR);
-            }
-            readLength = bufferPosn - startPosn;
-            if (prevCharCR && newlineLength == 0) {
-                --readLength; //CR at the end of the buffer
-            }
-            if (readLength > 0) {
-                record.append(inputBuffer, startPosn, readLength);
-            }
-        } while (newlineLength == 0);
-        recordNumber++;
-        return true;
-    }
-
-    @Override
-    public void configure(Map<String, String> configuration) throws Exception {
-        super.configure(configuration);
-        if (ExternalDataUtils.hasHeader(configuration)) {
-            if (hasNext()) {
-                next();
-            }
-        }
-    }
-}
\ No newline at end of file