You are viewing a plain text version of this content. The canonical link for it is here.
Posted to blur-commits@incubator.apache.org by am...@apache.org on 2016/08/30 01:57:46 UTC

[01/13] git commit: First patch of updates.

Repository: incubator-blur
Updated Branches:
  refs/heads/master 65640200a -> 991fb0435


First patch of updates.


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

Branch: refs/heads/master
Commit: 0e8d0e82700e2a056ab4bab72ce80bab29946d6b
Parents: 6564020
Author: Aaron McCurry <am...@gmail.com>
Authored: Sat May 7 13:09:44 2016 -0400
Committer: Aaron McCurry <am...@gmail.com>
Committed: Sat May 7 13:09:44 2016 -0400

----------------------------------------------------------------------
 .../apache/blur/command/CommandStatusUtil.java  |  83 +++++++++++++
 .../manager/writer/MergeSortRowIdLookup.java    | 104 ++++++++++++++++
 ...cumentVisibilityFilterCacheStrategyTest.java | 122 +++++++++++++++++++
 3 files changed, 309 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/0e8d0e82/blur-core/src/main/java/org/apache/blur/command/CommandStatusUtil.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/command/CommandStatusUtil.java b/blur-core/src/main/java/org/apache/blur/command/CommandStatusUtil.java
new file mode 100644
index 0000000..eeb9d08
--- /dev/null
+++ b/blur-core/src/main/java/org/apache/blur/command/CommandStatusUtil.java
@@ -0,0 +1,83 @@
+/**
+ * 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.blur.command;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.Map.Entry;
+
+import org.apache.blur.thrift.generated.CommandStatus;
+import org.apache.blur.thrift.generated.CommandStatusState;
+
+public class CommandStatusUtil {
+
+  public static CommandStatus mergeCommandStatus(CommandStatus cs1, CommandStatus cs2) {
+    if (cs1 == null && cs2 == null) {
+      return null;
+    } else if (cs1 == null) {
+      return cs2;
+    } else if (cs2 == null) {
+      return cs1;
+    } else {
+      Map<String, Map<CommandStatusState, Long>> serverStateMap1 = cs1.getServerStateMap();
+      Map<String, Map<CommandStatusState, Long>> serverStateMap2 = cs2.getServerStateMap();
+      Map<String, Map<CommandStatusState, Long>> merge = mergeServerStateMap(serverStateMap1, serverStateMap2);
+      return new CommandStatus(cs1.getExecutionId(), cs1.getCommandName(), cs1.getArguments(), merge, cs1.getUser());
+    }
+  }
+
+  private static Map<String, Map<CommandStatusState, Long>> mergeServerStateMap(
+      Map<String, Map<CommandStatusState, Long>> serverStateMap1,
+      Map<String, Map<CommandStatusState, Long>> serverStateMap2) {
+    Map<String, Map<CommandStatusState, Long>> result = new HashMap<String, Map<CommandStatusState, Long>>();
+    Set<String> keys = new HashSet<String>();
+    keys.addAll(serverStateMap1.keySet());
+    keys.addAll(serverStateMap2.keySet());
+    for (String key : keys) {
+      Map<CommandStatusState, Long> css1 = serverStateMap1.get(key);
+      Map<CommandStatusState, Long> css2 = serverStateMap2.get(key);
+      result.put(key, mergeCommandStatusState(css1, css2));
+    }
+    return result;
+  }
+
+  private static Map<CommandStatusState, Long> mergeCommandStatusState(Map<CommandStatusState, Long> css1,
+      Map<CommandStatusState, Long> css2) {
+    if (css1 == null && css2 == null) {
+      return new HashMap<CommandStatusState, Long>();
+    } else if (css1 == null) {
+      return css2;
+    } else if (css2 == null) {
+      return css1;
+    } else {
+      Map<CommandStatusState, Long> result = new HashMap<CommandStatusState, Long>(css1);
+      for (Entry<CommandStatusState, Long> e : css2.entrySet()) {
+        CommandStatusState key = e.getKey();
+        Long l = result.get(key);
+        Long value = e.getValue();
+        if (l == null) {
+          result.put(key, value);
+        } else {
+          result.put(key, l + value);
+        }
+      }
+      return result;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/0e8d0e82/blur-core/src/main/java/org/apache/blur/manager/writer/MergeSortRowIdLookup.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/manager/writer/MergeSortRowIdLookup.java b/blur-core/src/main/java/org/apache/blur/manager/writer/MergeSortRowIdLookup.java
new file mode 100644
index 0000000..b500a4f
--- /dev/null
+++ b/blur-core/src/main/java/org/apache/blur/manager/writer/MergeSortRowIdLookup.java
@@ -0,0 +1,104 @@
+/**
+ * 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.blur.manager.writer;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.blur.utils.BlurConstants;
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+
+public class MergeSortRowIdLookup {
+
+  public interface Action {
+    void found(AtomicReader reader, Bits liveDocs, TermsEnum termsEnum) throws IOException;
+  }
+
+  private final List<TermsEnumReader> _termsEnumList = new ArrayList<TermsEnumReader>();
+
+  public MergeSortRowIdLookup(IndexReader indexReader) throws IOException {
+    if (indexReader instanceof AtomicReader) {
+      addAtomicReader((AtomicReader) indexReader);
+    } else {
+      for (AtomicReaderContext context : indexReader.leaves()) {
+        addAtomicReader(context.reader());
+      }
+    }
+  }
+
+  private void addAtomicReader(AtomicReader atomicReader) throws IOException {
+    Terms terms = atomicReader.fields().terms(BlurConstants.ROW_ID);
+    TermsEnum termsEnum = terms.iterator(null);
+    _termsEnumList.add(new TermsEnumReader(termsEnum, atomicReader));
+  }
+
+  public void lookup(BytesRef rowId, Action action) throws IOException {
+    advance(_termsEnumList, rowId);
+    sort(_termsEnumList);
+    for (TermsEnumReader reader : _termsEnumList) {
+      if (reader._termsEnum.term().equals(rowId)) {
+        action.found(reader._reader, reader._liveDocs, reader._termsEnum);
+      }
+    }
+  }
+
+  private static void advance(List<TermsEnumReader> termsEnumList, BytesRef rowId) throws IOException {
+    for (TermsEnumReader reader : termsEnumList) {
+      BytesRef term = reader._termsEnum.term();
+      if (term.compareTo(rowId) < 0) {
+        reader._termsEnum.seekCeil(rowId);
+      }
+    }
+  }
+
+  private static void sort(List<TermsEnumReader> termsEnumList) {
+    Collections.sort(termsEnumList);
+  }
+
+  private static class TermsEnumReader implements Comparable<TermsEnumReader> {
+
+    final Bits _liveDocs;
+    final TermsEnum _termsEnum;
+    final AtomicReader _reader;
+
+    TermsEnumReader(TermsEnum termsEnum, AtomicReader reader) {
+      _termsEnum = termsEnum;
+      _reader = reader;
+      _liveDocs = reader.getLiveDocs();
+    }
+
+    @Override
+    public int compareTo(TermsEnumReader o) {
+      try {
+        BytesRef t1 = _termsEnum.term();
+        BytesRef t2 = o._termsEnum.term();
+        return t1.compareTo(t2);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/0e8d0e82/blur-document-security/src/test/java/org/apache/blur/lucene/security/search/BitSetDocumentVisibilityFilterCacheStrategyTest.java
----------------------------------------------------------------------
diff --git a/blur-document-security/src/test/java/org/apache/blur/lucene/security/search/BitSetDocumentVisibilityFilterCacheStrategyTest.java b/blur-document-security/src/test/java/org/apache/blur/lucene/security/search/BitSetDocumentVisibilityFilterCacheStrategyTest.java
new file mode 100644
index 0000000..be289b0
--- /dev/null
+++ b/blur-document-security/src/test/java/org/apache/blur/lucene/security/search/BitSetDocumentVisibilityFilterCacheStrategyTest.java
@@ -0,0 +1,122 @@
+/**
+ * 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.blur.lucene.security.search;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.OpenBitSet;
+import org.junit.Test;
+
+public class BitSetDocumentVisibilityFilterCacheStrategyTest {
+
+  @Test
+  public void testIdFullySet1() {
+    int maxDoc = 10;
+    OpenBitSet bitSet = new OpenBitSet(maxDoc);
+    assertFalse(BitSetDocumentVisibilityFilterCacheStrategy.isFullySet(maxDoc, bitSet, bitSet.cardinality()));
+  }
+
+  @Test
+  public void testIdFullySet2() {
+    int maxDoc = 10;
+    OpenBitSet bitSet = new OpenBitSet(maxDoc);
+    for (int d = 0; d < maxDoc; d++) {
+      bitSet.set(d);
+    }
+    assertTrue(BitSetDocumentVisibilityFilterCacheStrategy.isFullySet(maxDoc, bitSet, bitSet.cardinality()));
+  }
+
+  @Test
+  public void testIdFullyEmpty1() {
+    int maxDoc = 10;
+    OpenBitSet bitSet = new OpenBitSet(maxDoc);
+    assertTrue(BitSetDocumentVisibilityFilterCacheStrategy.isFullyEmpty(bitSet, bitSet.cardinality()));
+  }
+
+  @Test
+  public void testIdFullyEmpty2() {
+    int maxDoc = 10;
+    OpenBitSet bitSet = new OpenBitSet(maxDoc);
+    bitSet.set(3);
+    assertFalse(BitSetDocumentVisibilityFilterCacheStrategy.isFullyEmpty(bitSet, bitSet.cardinality()));
+  }
+
+  @Test
+  public void testFullySetDocIdSet() throws IOException {
+    int len = 10;
+    DocIdSet docIdSet = BitSetDocumentVisibilityFilterCacheStrategy.getFullySetDocIdSet(len);
+    Bits bits = docIdSet.bits();
+    assertEquals(len, bits.length());
+    for (int i = 0; i < len; i++) {
+      assertTrue(bits.get(i));
+    }
+    assertTrue(docIdSet.isCacheable());
+    {
+      DocIdSetIterator iterator = docIdSet.iterator();
+      int adoc;
+      int edoc = 0;
+      assertEquals(-1, iterator.docID());
+      while ((adoc = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+        assertEquals(edoc, adoc);
+        assertEquals(edoc, iterator.docID());
+        edoc++;
+      }
+      assertEquals(len, edoc);
+    }
+    {
+      DocIdSetIterator iterator = docIdSet.iterator();
+      int adoc;
+      int edoc = 0;
+      assertEquals(-1, iterator.docID());
+      while ((adoc = iterator.advance(edoc)) != DocIdSetIterator.NO_MORE_DOCS) {
+        assertEquals(edoc, adoc);
+        assertEquals(edoc, iterator.docID());
+        edoc++;
+      }
+      assertEquals(len, edoc);
+    }
+  }
+
+  @Test
+  public void testFullyEmptyDocIdSet() throws IOException {
+    int len = 10;
+    DocIdSet docIdSet = BitSetDocumentVisibilityFilterCacheStrategy.getFullyEmptyDocIdSet(len);
+    Bits bits = docIdSet.bits();
+    assertEquals(len, bits.length());
+    for (int i = 0; i < len; i++) {
+      assertFalse(bits.get(i));
+    }
+    assertTrue(docIdSet.isCacheable());
+    {
+      DocIdSetIterator iterator = docIdSet.iterator();
+      assertEquals(-1, iterator.docID());
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, iterator.nextDoc());
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, iterator.docID());
+    }
+    {
+      DocIdSetIterator iterator = docIdSet.iterator();
+      assertEquals(-1, iterator.docID());
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, iterator.advance(0));
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, iterator.docID());
+    }
+  }
+}


[02/13] git commit: Second patch of updates.

Posted by am...@apache.org.
Second patch of updates.


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

Branch: refs/heads/master
Commit: 014165621b11c6c1814d5233baae9f85f99c6ae6
Parents: 0e8d0e8
Author: Aaron McCurry <am...@gmail.com>
Authored: Sat May 7 13:10:19 2016 -0400
Committer: Aaron McCurry <am...@gmail.com>
Committed: Sat May 7 13:10:19 2016 -0400

----------------------------------------------------------------------
 .../apache/blur/command/BaseCommandManager.java |  23 ++-
 .../java/org/apache/blur/command/Command.java   |   2 +-
 .../blur/command/ControllerClusterContext.java  |   6 +-
 .../org/apache/blur/command/ResponseFuture.java |   4 +
 .../blur/manager/stats/MergerTableStats.java    |   2 +-
 .../manager/writer/BlurIndexSimpleWriter.java   |  72 ++++----
 .../blur/manager/writer/IndexImporter.java      | 130 +++++++++++---
 .../writer/SnapshotIndexDeletionPolicy.java     |  81 +++++----
 .../blur/thrift/BlurControllerServer.java       |  80 +++------
 .../org/apache/blur/thrift/BlurShardServer.java |   6 +-
 .../blur/thrift/ThriftBlurShardServer.java      |   2 +-
 .../java/org/apache/blur/utils/GCWatcher.java   |  28 ++-
 .../org/apache/blur/utils/GCWatcherJdk6.java    |   2 +-
 .../blur/manager/writer/IndexImporterTest.java  |  17 ++
 .../index/FilterAccessControlFactory.java       |   1 +
 ...etDocumentVisibilityFilterCacheStrategy.java | 172 ++++++++++++++++++-
 16 files changed, 448 insertions(+), 180 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-core/src/main/java/org/apache/blur/command/BaseCommandManager.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/command/BaseCommandManager.java b/blur-core/src/main/java/org/apache/blur/command/BaseCommandManager.java
index ad542ef..be92e34 100644
--- a/blur-core/src/main/java/org/apache/blur/command/BaseCommandManager.java
+++ b/blur-core/src/main/java/org/apache/blur/command/BaseCommandManager.java
@@ -134,19 +134,20 @@ public abstract class BaseCommandManager implements Closeable {
   }
 
   public CommandStatus getCommandStatus(String commandExecutionId) {
-    CommandStatus cso = findCommandStatusObject(commandExecutionId, _workerRunningMap.values());
-    if (cso != null) {
-      return cso;
-    }
-    return findCommandStatusObject(commandExecutionId, _driverRunningMap.values());
+    CommandStatus cso1 = findCommandStatusObject(commandExecutionId, _workerRunningMap.values());
+    CommandStatus cso2 = findCommandStatusObject(commandExecutionId, _driverRunningMap.values());
+    return CommandStatusUtil.mergeCommandStatus(cso1, cso2);
   }
 
   private CommandStatus findCommandStatusObject(String commandExecutionId, Collection<ResponseFuture<?>> values) {
     Map<String, Map<CommandStatusState, Long>> serverStateMap = new HashMap<String, Map<CommandStatusState, Long>>();
     CommandStatus commandStatus = null;
     for (ResponseFuture<?> responseFuture : values) {
+      if (responseFuture == null) {
+        continue;
+      }
       Command<?> commandExecuting = responseFuture.getCommandExecuting();
-      if (commandExecuting.getCommandExecutionId().equals(commandExecutionId)) {
+      if (commandExecutionId.equals(commandExecuting.getCommandExecutionId())) {
         if (commandStatus == null) {
           CommandStatus originalCommandStatusObject = responseFuture.getOriginalCommandStatusObject();
           String commandName = responseFuture.getCommandExecuting().getName();
@@ -182,7 +183,10 @@ public abstract class BaseCommandManager implements Closeable {
     List<String> result = new ArrayList<String>();
     for (ResponseFuture<?> responseFuture : values) {
       Command<?> commandExecuting = responseFuture.getCommandExecuting();
-      result.add(commandExecuting.getCommandExecutionId());
+      String commandExecutionId = commandExecuting.getCommandExecutionId();
+      if (commandExecutionId != null) {
+        result.add(commandExecutionId);
+      }
     }
     return result;
   }
@@ -400,11 +404,12 @@ public abstract class BaseCommandManager implements Closeable {
   }
 
   protected Response submitDriverCallable(Callable<Response> callable, Command<?> commandExecuting,
-      CommandStatus originalCommandStatusObject, AtomicBoolean running) throws IOException, TimeoutException, ExceptionCollector {
+      CommandStatus originalCommandStatusObject, AtomicBoolean running) throws IOException, TimeoutException,
+      ExceptionCollector {
     Future<Response> future = _executorServiceDriver.submit(callable);
     Long instanceExecutionId = getInstanceExecutionId();
     _driverRunningMap.put(instanceExecutionId, new ResponseFuture<Response>(_runningCacheTombstoneTime, future,
-        commandExecuting, originalCommandStatusObject,running));
+        commandExecuting, originalCommandStatusObject, running));
     try {
       return future.get(_connectionTimeout, TimeUnit.MILLISECONDS);
     } catch (CancellationException e) {

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-core/src/main/java/org/apache/blur/command/Command.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/command/Command.java b/blur-core/src/main/java/org/apache/blur/command/Command.java
index 9cf2719..ff6f559 100644
--- a/blur-core/src/main/java/org/apache/blur/command/Command.java
+++ b/blur-core/src/main/java/org/apache/blur/command/Command.java
@@ -30,7 +30,7 @@ import org.apache.blur.thrift.generated.Blur.Iface;
 
 public abstract class Command<R> implements Cloneable {
 
-  @OptionalArgument("The ")
+  @OptionalArgument
   private String commandExecutionId;
 
   public abstract String getName();

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-core/src/main/java/org/apache/blur/command/ControllerClusterContext.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/command/ControllerClusterContext.java b/blur-core/src/main/java/org/apache/blur/command/ControllerClusterContext.java
index b7a1a63..59f5b7c 100644
--- a/blur-core/src/main/java/org/apache/blur/command/ControllerClusterContext.java
+++ b/blur-core/src/main/java/org/apache/blur/command/ControllerClusterContext.java
@@ -23,6 +23,7 @@ import org.apache.blur.thirdparty.thrift_0_9_0.TException;
 import org.apache.blur.thrift.BlurClientManager;
 import org.apache.blur.thrift.ClientPool;
 import org.apache.blur.thrift.Connection;
+import org.apache.blur.thrift.UserConverter;
 import org.apache.blur.thrift.generated.Arguments;
 import org.apache.blur.thrift.generated.Blur.Client;
 import org.apache.blur.thrift.generated.BlurException;
@@ -31,6 +32,7 @@ import org.apache.blur.thrift.generated.Response;
 import org.apache.blur.thrift.generated.TimeoutException;
 import org.apache.blur.thrift.generated.ValueObject;
 import org.apache.blur.trace.Tracer;
+import org.apache.blur.user.UserContext;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -134,7 +136,7 @@ public class ControllerClusterContext extends ClusterContext implements Closeabl
 
     final Arguments arguments = _manager.toArguments(command);
 
-    CommandStatus originalCommandStatusObject = new CommandStatus(null, command.getName(), arguments, null, null);
+    CommandStatus originalCommandStatusObject = new CommandStatus(null, command.getName(), arguments, null, UserConverter.toThriftUser(UserContext.getUser()));
     for (Entry<Server, Client> e : clientMap.entrySet()) {
       Server server = e.getKey();
       final Client client = e.getValue();
@@ -226,7 +228,7 @@ public class ControllerClusterContext extends ClusterContext implements Closeabl
     Set<Shard> shards = command.routeShards(this, tables);
     Map<Server, Client> clientMap = getClientMap(command, tables, shards);
     final Arguments arguments = _manager.toArguments(command);
-    CommandStatus originalCommandStatusObject = new CommandStatus(null, command.getName(), arguments, null, null);
+    CommandStatus originalCommandStatusObject = new CommandStatus(null, command.getName(), arguments, null, UserConverter.toThriftUser(UserContext.getUser()));
     for (Entry<Server, Client> e : clientMap.entrySet()) {
       Server server = e.getKey();
       final Client client = e.getValue();

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-core/src/main/java/org/apache/blur/command/ResponseFuture.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/command/ResponseFuture.java b/blur-core/src/main/java/org/apache/blur/command/ResponseFuture.java
index a5a629e..ef4a046 100644
--- a/blur-core/src/main/java/org/apache/blur/command/ResponseFuture.java
+++ b/blur-core/src/main/java/org/apache/blur/command/ResponseFuture.java
@@ -16,6 +16,7 @@
  */
 package org.apache.blur.command;
 
+import java.util.UUID;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
@@ -39,6 +40,9 @@ public class ResponseFuture<T> implements Future<T> {
     _tombstone = tombstone;
     _future = future;
     _commandExecuting = commandExecuting;
+    if (_commandExecuting.getCommandExecutionId() == null) {
+      _commandExecuting.setCommandExecutionId(UUID.randomUUID().toString());
+    }
     _originalCommandStatusObject = originalCommandStatusObject;
     _running = running;
   }

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-core/src/main/java/org/apache/blur/manager/stats/MergerTableStats.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/manager/stats/MergerTableStats.java b/blur-core/src/main/java/org/apache/blur/manager/stats/MergerTableStats.java
index 2f89c9e..4ac5631 100644
--- a/blur-core/src/main/java/org/apache/blur/manager/stats/MergerTableStats.java
+++ b/blur-core/src/main/java/org/apache/blur/manager/stats/MergerTableStats.java
@@ -46,7 +46,7 @@ public class MergerTableStats implements Merger<TableStats> {
 
   private TableStats merge(TableStats s1, TableStats s2) {
     s1.tableName = s2.tableName;
-    s1.bytes = Math.max(s1.bytes, s2.bytes);
+    s1.bytes = s1.bytes + s2.bytes;
     s1.recordCount = s1.recordCount + s2.recordCount;
     s1.rowCount = s1.rowCount + s2.rowCount;
     s1.segmentImportInProgressCount = s1.segmentImportInProgressCount + s2.segmentImportInProgressCount;

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-core/src/main/java/org/apache/blur/manager/writer/BlurIndexSimpleWriter.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/manager/writer/BlurIndexSimpleWriter.java b/blur-core/src/main/java/org/apache/blur/manager/writer/BlurIndexSimpleWriter.java
index e21a952..ff17e27 100644
--- a/blur-core/src/main/java/org/apache/blur/manager/writer/BlurIndexSimpleWriter.java
+++ b/blur-core/src/main/java/org/apache/blur/manager/writer/BlurIndexSimpleWriter.java
@@ -25,7 +25,6 @@ import static org.apache.blur.utils.BlurConstants.BLUR_SHARD_INDEX_WRITER_SORT_M
 import static org.apache.blur.utils.BlurConstants.BLUR_SHARD_QUEUE_MAX_INMEMORY_LENGTH;
 
 import java.io.Closeable;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
@@ -77,6 +76,7 @@ import org.apache.blur.user.User;
 import org.apache.blur.user.UserContext;
 import org.apache.blur.utils.BlurConstants;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -428,31 +428,31 @@ public class BlurIndexSimpleWriter extends BlurIndex {
   }
 
   private void closeWriter() {
-    if (_lastWrite.get() + _maxWriterIdle < System.currentTimeMillis()) {
-      synchronized (_writer) {
-        _writeLock.lock();
-        try {
-          BlurIndexWriter writer = _writer.getAndSet(null);
-          if (writer != null) {
-            LOG.info("Closing idle writer for table [{0}] shard [{1}]", _tableContext.getTable(),
-                _shardContext.getShard());
-            IOUtils.cleanup(LOG, writer);
-          }
-        } finally {
-          _writeLock.unlock();
+    _writeLock.lock();
+    try {
+      if (_lastWrite.get() + _maxWriterIdle < System.currentTimeMillis()) {
+        BlurIndexWriter writer = _writer.getAndSet(null);
+        if (writer != null) {
+          LOG.info("Closing idle writer for table [{0}] shard [{1}]", _tableContext.getTable(),
+              _shardContext.getShard());
+          IOUtils.cleanup(LOG, writer);
         }
       }
+    } finally {
+      _writeLock.unlock();
     }
   }
 
+  /**
+   * Testing only.
+   */
   protected boolean isWriterClosed() {
-    synchronized (_writer) {
-      return _writer.get() == null;
-    }
+    return _writer.get() == null;
   }
 
   private BlurIndexWriter getBlurIndexWriter() throws IOException {
-    synchronized (_writer) {
+    _writeLock.lock();
+    try {
       BlurIndexWriter blurIndexWriter = _writer.get();
       if (blurIndexWriter == null) {
         blurIndexWriter = new BlurIndexWriter(_directory, _conf.clone());
@@ -460,12 +460,17 @@ public class BlurIndexSimpleWriter extends BlurIndex {
         _lastWrite.set(System.currentTimeMillis());
       }
       return blurIndexWriter;
+    } finally {
+      _writeLock.unlock();
     }
   }
 
   private void resetBlurIndexWriter() {
-    synchronized (_writer) {
+    _writeLock.lock();
+    try {
       _writer.set(null);
+    } finally {
+      _writeLock.unlock();
     }
   }
 
@@ -501,22 +506,12 @@ public class BlurIndexSimpleWriter extends BlurIndex {
 
   @Override
   public void createSnapshot(String name) throws IOException {
-    _writeLock.lock();
-    try {
-      _snapshotIndexDeletionPolicy.createSnapshot(name, _indexReader.get(), _context);
-    } finally {
-      _writeLock.unlock();
-    }
+    _snapshotIndexDeletionPolicy.createSnapshot(name, _indexReader.get(), _context);
   }
 
   @Override
   public void removeSnapshot(String name) throws IOException {
-    _writeLock.lock();
-    try {
-      _snapshotIndexDeletionPolicy.removeSnapshot(name, _context);
-    } finally {
-      _writeLock.unlock();
-    }
+    _snapshotIndexDeletionPolicy.removeSnapshot(name, _context);
   }
 
   @Override
@@ -1024,17 +1019,10 @@ public class BlurIndexSimpleWriter extends BlurIndex {
 
   @Override
   public long getOnDiskSize() throws IOException {
-    long total = 0;
-    String[] listAll = _directory.listAll();
-    for (String name : listAll) {
-      try {
-        total += _directory.fileLength(name);
-      } catch (FileNotFoundException e) {
-        // If file is not found that means that is was removed between the time
-        // we started iterating over the file names and when we asked for it's
-        // size.
-      }
-    }
-    return total;
+    Path hdfsDirPath = _shardContext.getHdfsDirPath();
+    Configuration configuration = _tableContext.getConfiguration();
+    FileSystem fileSystem = hdfsDirPath.getFileSystem(configuration);
+    ContentSummary contentSummary = fileSystem.getContentSummary(hdfsDirPath);
+    return contentSummary.getLength();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-core/src/main/java/org/apache/blur/manager/writer/IndexImporter.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/manager/writer/IndexImporter.java b/blur-core/src/main/java/org/apache/blur/manager/writer/IndexImporter.java
index 42171e8..33db0ae 100644
--- a/blur-core/src/main/java/org/apache/blur/manager/writer/IndexImporter.java
+++ b/blur-core/src/main/java/org/apache/blur/manager/writer/IndexImporter.java
@@ -36,6 +36,7 @@ import org.apache.blur.log.Log;
 import org.apache.blur.log.LogFactory;
 import org.apache.blur.lucene.search.IndexSearcherCloseable;
 import org.apache.blur.manager.BlurPartitioner;
+import org.apache.blur.manager.writer.MergeSortRowIdLookup.Action;
 import org.apache.blur.server.ShardContext;
 import org.apache.blur.server.TableContext;
 import org.apache.blur.server.cache.ThriftCache;
@@ -54,12 +55,16 @@ import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.CompositeReaderContext;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 
 public class IndexImporter extends TimerTask implements Closeable {
@@ -292,7 +297,9 @@ public class IndexImporter extends TimerTask implements Closeable {
       public void performMutate(IndexSearcherCloseable searcher, IndexWriter writer) throws IOException {
         LOG.info("About to import [{0}] into [{1}/{2}]", directory, _shard, _table);
         boolean emitDeletes = searcher.getIndexReader().numDocs() != 0;
-        applyDeletes(directory, writer, _shard, emitDeletes);
+        Configuration configuration = _shardContext.getTableContext().getConfiguration();
+
+        applyDeletes(directory, writer, searcher, _shard, emitDeletes, configuration);
         LOG.info("Add index [{0}] [{1}/{2}]", directory, _shard, _table);
         writer.addIndexes(directory);
         LOG.info("Removing delete markers [{0}] on [{1}/{2}]", directory, _shard, _table);
@@ -336,40 +343,113 @@ public class IndexImporter extends TimerTask implements Closeable {
     return result;
   }
 
-  private void applyDeletes(Directory directory, IndexWriter indexWriter, String shard, boolean emitDeletes)
-      throws IOException {
-    DirectoryReader reader = DirectoryReader.open(directory);
+  private void applyDeletes(Directory directory, IndexWriter indexWriter, IndexSearcherCloseable searcher,
+      String shard, boolean emitDeletes, Configuration configuration) throws IOException {
+    DirectoryReader newReader = DirectoryReader.open(directory);
     try {
-      LOG.info("Applying deletes in reader [{0}]", reader);
-      CompositeReaderContext compositeReaderContext = reader.getContext();
-      List<AtomicReaderContext> leaves = compositeReaderContext.leaves();
+      List<AtomicReaderContext> newLeaves = newReader.getContext().leaves();
       BlurPartitioner blurPartitioner = new BlurPartitioner();
       Text key = new Text();
       int numberOfShards = _shardContext.getTableContext().getDescriptor().getShardCount();
       int shardId = ShardUtil.getShardIndex(shard);
-      for (AtomicReaderContext context : leaves) {
-        AtomicReader atomicReader = context.reader();
-        Fields fields = atomicReader.fields();
-        Terms terms = fields.terms(BlurConstants.ROW_ID);
-        if (terms != null) {
-          TermsEnum termsEnum = terms.iterator(null);
-          BytesRef ref = null;
-          while ((ref = termsEnum.next()) != null) {
-            key.set(ref.bytes, ref.offset, ref.length);
-            int partition = blurPartitioner.getPartition(key, null, numberOfShards);
-            if (shardId != partition) {
-              throw new IOException("Index is corrupted, RowIds are found in wrong shard, partition [" + partition
-                  + "] does not shard [" + shardId + "], this can happen when rows are not hashed correctly.");
-            }
-            if (emitDeletes) {
-              indexWriter.deleteDocuments(new Term(BlurConstants.ROW_ID, BytesRef.deepCopyOf(ref)));
-            }
+
+      Action action = new Action() {
+        @Override
+        public void found(AtomicReader reader, Bits liveDocs, TermsEnum termsEnum) throws IOException {
+          DocsEnum docsEnum = termsEnum.docs(liveDocs, null);
+          if (docsEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
+            indexWriter.deleteDocuments(new Term(BlurConstants.ROW_ID, BytesRef.deepCopyOf(termsEnum.term())));
           }
         }
+      };
+
+      LOG.info("Applying deletes for table [{0}] shard [{1}] new reader [{2}]", _table, shard, newReader);
+      boolean skipCheckRowIds = isInternal(newReader);
+      LOG.info("Skip rowid check [{0}] for table [{1}] shard [{2}] new reader [{3}]", skipCheckRowIds, _table, shard,
+          newReader);
+      for (AtomicReaderContext context : newLeaves) {
+        AtomicReader newAtomicReader = context.reader();
+        if (isFastRowIdDeleteSupported(newAtomicReader)) {
+          runNewRowIdCheckAndDelete(indexWriter, emitDeletes, blurPartitioner, key, numberOfShards, shardId,
+              newAtomicReader, skipCheckRowIds);
+        } else {
+          runOldMergeSortRowIdCheckAndDelete(emitDeletes, searcher.getIndexReader(), blurPartitioner, key,
+              numberOfShards, shardId, action, newAtomicReader);
+        }
       }
     } finally {
-      reader.close();
+      newReader.close();
+    }
+  }
+
+  private boolean isInternal(DirectoryReader reader) throws IOException {
+    Map<String, String> map = reader.getIndexCommit().getUserData();
+    return BlurConstants.INTERNAL.equals(map.get(BlurConstants.INTERNAL));
+  }
+
+  private void runNewRowIdCheckAndDelete(IndexWriter indexWriter, boolean emitDeletes, BlurPartitioner blurPartitioner,
+      Text key, int numberOfShards, int shardId, AtomicReader atomicReader, boolean skipCheckRowIds) throws IOException {
+    Fields fields = atomicReader.fields();
+    if (skipCheckRowIds) {
+      Terms rowIdTerms = fields.terms(BlurConstants.ROW_ID);
+      if (rowIdTerms != null) {
+        LOG.info("Checking rowIds for import on table [{0}] shard [{1}]", _table, _shard);
+        TermsEnum rowIdTermsEnum = rowIdTerms.iterator(null);
+        BytesRef ref = null;
+        while ((ref = rowIdTermsEnum.next()) != null) {
+          key.set(ref.bytes, ref.offset, ref.length);
+          int partition = blurPartitioner.getPartition(key, null, numberOfShards);
+          if (shardId != partition) {
+            throw new IOException("Index is corrupted, RowIds are found in wrong shard, partition [" + partition
+                + "] does not shard [" + shardId + "], this can happen when rows are not hashed correctly.");
+          }
+        }
+      }
+    }
+    if (emitDeletes) {
+      Terms rowIdsToDeleteTerms = fields.terms(BlurConstants.UPDATE_ROW);
+      if (rowIdsToDeleteTerms != null) {
+        LOG.info("Performing deletes on rowIds for import on table [{0}] shard [{1}]", _table, _shard);
+        TermsEnum rowIdsToDeleteTermsEnum = rowIdsToDeleteTerms.iterator(null);
+        BytesRef ref = null;
+        while ((ref = rowIdsToDeleteTermsEnum.next()) != null) {
+          indexWriter.deleteDocuments(new Term(BlurConstants.ROW_ID, BytesRef.deepCopyOf(ref)));
+        }
+      }
+    }
+  }
+
+  private void runOldMergeSortRowIdCheckAndDelete(boolean emitDeletes, IndexReader currentIndexReader,
+      BlurPartitioner blurPartitioner, Text key, int numberOfShards, int shardId, Action action,
+      AtomicReader atomicReader) throws IOException {
+    MergeSortRowIdLookup lookup = new MergeSortRowIdLookup(currentIndexReader);
+    Fields fields = atomicReader.fields();
+    Terms terms = fields.terms(BlurConstants.ROW_ID);
+    if (terms != null) {
+      TermsEnum termsEnum = terms.iterator(null);
+      BytesRef ref = null;
+      while ((ref = termsEnum.next()) != null) {
+        key.set(ref.bytes, ref.offset, ref.length);
+        int partition = blurPartitioner.getPartition(key, null, numberOfShards);
+        if (shardId != partition) {
+          throw new IOException("Index is corrupted, RowIds are found in wrong shard, partition [" + partition
+              + "] does not shard [" + shardId + "], this can happen when rows are not hashed correctly.");
+        }
+        if (emitDeletes) {
+          lookup.lookup(ref, action);
+        }
+      }
+    }
+  }
+
+  private boolean isFastRowIdDeleteSupported(AtomicReader atomicReader) throws IOException {
+    if (atomicReader.fields().terms(BlurConstants.NEW_ROW) != null) {
+      return true;
+    }
+    if (atomicReader.fields().terms(BlurConstants.UPDATE_ROW) != null) {
+      return true;
     }
+    return false;
   }
 
   public void cleanupOldDirs() throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-core/src/main/java/org/apache/blur/manager/writer/SnapshotIndexDeletionPolicy.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/manager/writer/SnapshotIndexDeletionPolicy.java b/blur-core/src/main/java/org/apache/blur/manager/writer/SnapshotIndexDeletionPolicy.java
index 30690e5..15d9272 100644
--- a/blur-core/src/main/java/org/apache/blur/manager/writer/SnapshotIndexDeletionPolicy.java
+++ b/blur-core/src/main/java/org/apache/blur/manager/writer/SnapshotIndexDeletionPolicy.java
@@ -28,6 +28,8 @@ import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
 import org.apache.blur.log.Log;
 import org.apache.blur.log.LogFactory;
@@ -54,6 +56,7 @@ public class SnapshotIndexDeletionPolicy extends IndexDeletionPolicy {
   private final Path _path;
   private final Map<String, Long> _namesToGenerations = new ConcurrentHashMap<String, Long>();
   private final Map<Long, Set<String>> _generationsToNames = new ConcurrentHashMap<Long, Set<String>>();
+  private final WriteLock _writeLock = new ReentrantReadWriteLock().writeLock();
 
   public SnapshotIndexDeletionPolicy(Configuration configuration, Path path) throws IOException {
     _configuration = configuration;
@@ -70,13 +73,18 @@ public class SnapshotIndexDeletionPolicy extends IndexDeletionPolicy {
 
   @Override
   public void onCommit(List<? extends IndexCommit> commits) throws IOException {
-    int size = commits.size();
-    for (int i = 0; i < size - 1; i++) {
-      IndexCommit indexCommit = commits.get(i);
-      long generation = indexCommit.getGeneration();
-      if (!_generationsToNames.containsKey(generation)) {
-        indexCommit.delete();
+    _writeLock.lock();
+    try {
+      int size = commits.size();
+      for (int i = 0; i < size - 1; i++) {
+        IndexCommit indexCommit = commits.get(i);
+        long generation = indexCommit.getGeneration();
+        if (!_generationsToNames.containsKey(generation)) {
+          indexCommit.delete();
+        }
       }
+    } finally {
+      _writeLock.unlock();
     }
   }
 
@@ -147,36 +155,46 @@ public class SnapshotIndexDeletionPolicy extends IndexDeletionPolicy {
   }
 
   public void createSnapshot(String name, DirectoryReader reader, String context) throws IOException {
-    if (_namesToGenerations.containsKey(name)) {
-      throw new IOException("Snapshot [" + name + "] already exists.");
-    }
-    LOG.info("Creating snapshot [{0}] in [{1}].", name, context);
-    IndexCommit indexCommit = reader.getIndexCommit();
-    long generation = indexCommit.getGeneration();
-    _namesToGenerations.put(name, generation);
-    Set<String> names = _generationsToNames.get(generation);
-    if (names == null) {
-      names = Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
-      _generationsToNames.put(generation, names);
+    _writeLock.lock();
+    try {
+      if (_namesToGenerations.containsKey(name)) {
+        throw new IOException("Snapshot [" + name + "] already exists.");
+      }
+      LOG.info("Creating snapshot [{0}] in [{1}].", name, context);
+      IndexCommit indexCommit = reader.getIndexCommit();
+      long generation = indexCommit.getGeneration();
+      _namesToGenerations.put(name, generation);
+      Set<String> names = _generationsToNames.get(generation);
+      if (names == null) {
+        names = Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
+        _generationsToNames.put(generation, names);
+      }
+      names.add(name);
+      storeGenerations();
+    } finally {
+      _writeLock.unlock();
     }
-    names.add(name);
-    storeGenerations();
   }
 
   public void removeSnapshot(String name, String context) throws IOException {
-    Long gen = _namesToGenerations.get(name);
-    if (gen == null) {
-      LOG.info("Snapshot [{0}] does not exist in [{1}].", name, context);
-      return;
-    }
-    LOG.info("Removing snapshot [{0}] from [{1}].", name, context);
-    _namesToGenerations.remove(name);
-    Set<String> names = _generationsToNames.get(gen);
-    names.remove(name);
-    if (names.isEmpty()) {
-      _generationsToNames.remove(gen);
+    _writeLock.lock();
+    try {
+      Long gen = _namesToGenerations.get(name);
+      if (gen == null) {
+        LOG.info("Snapshot [{0}] does not exist in [{1}].", name, context);
+        return;
+      }
+      LOG.info("Removing snapshot [{0}] from [{1}].", name, context);
+      _namesToGenerations.remove(name);
+      Set<String> names = _generationsToNames.get(gen);
+      names.remove(name);
+      if (names.isEmpty()) {
+        _generationsToNames.remove(gen);
+      }
+      storeGenerations();
+    } finally {
+      _writeLock.unlock();
     }
-    storeGenerations();
   }
 
   public Collection<String> getSnapshots() {
@@ -194,5 +212,4 @@ public class SnapshotIndexDeletionPolicy extends IndexDeletionPolicy {
   public static Path getGenerationsPath(Path shardDir) {
     return new Path(shardDir, "generations");
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-core/src/main/java/org/apache/blur/thrift/BlurControllerServer.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/thrift/BlurControllerServer.java b/blur-core/src/main/java/org/apache/blur/thrift/BlurControllerServer.java
index 477c923..e4d29e0 100644
--- a/blur-core/src/main/java/org/apache/blur/thrift/BlurControllerServer.java
+++ b/blur-core/src/main/java/org/apache/blur/thrift/BlurControllerServer.java
@@ -51,6 +51,7 @@ import java.util.concurrent.atomic.AtomicReferenceArray;
 import org.apache.blur.command.ArgumentOverlay;
 import org.apache.blur.command.BlurObject;
 import org.apache.blur.command.BlurObjectSerDe;
+import org.apache.blur.command.CommandStatusUtil;
 import org.apache.blur.command.CommandUtil;
 import org.apache.blur.command.ControllerCommandManager;
 import org.apache.blur.command.Response;
@@ -92,7 +93,6 @@ import org.apache.blur.thrift.generated.BlurResults;
 import org.apache.blur.thrift.generated.ColumnDefinition;
 import org.apache.blur.thrift.generated.CommandDescriptor;
 import org.apache.blur.thrift.generated.CommandStatus;
-import org.apache.blur.thrift.generated.CommandStatusState;
 import org.apache.blur.thrift.generated.ErrorType;
 import org.apache.blur.thrift.generated.FetchResult;
 import org.apache.blur.thrift.generated.HighlightOptions;
@@ -108,6 +108,7 @@ import org.apache.blur.thrift.generated.User;
 import org.apache.blur.trace.Trace;
 import org.apache.blur.trace.Trace.TraceId;
 import org.apache.blur.trace.Tracer;
+import org.apache.blur.user.UserContext;
 import org.apache.blur.utils.BlurExecutorCompletionService;
 import org.apache.blur.utils.BlurIterator;
 import org.apache.blur.utils.BlurUtil;
@@ -1514,7 +1515,8 @@ public class BlurControllerServer extends TableAdmin implements Iface {
       throws BlurException, TException {
     try {
       BlurObject args = CommandUtil.toBlurObject(arguments);
-      CommandStatus originalCommandStatusObject = new CommandStatus(null, commandName, arguments, null, null);
+      CommandStatus originalCommandStatusObject = new CommandStatus(null, commandName, arguments, null,
+          UserConverter.toThriftUser(UserContext.getUser()));
       Response response = _commandManager.execute(getTableContextFactory(), getLayoutFactory(), commandName,
           new ArgumentOverlay(args, _serDe), originalCommandStatusObject);
       return CommandUtil.fromObjectToThrift(response, _serDe);
@@ -1861,7 +1863,7 @@ public class BlurControllerServer extends TableAdmin implements Iface {
           }
         }));
       }
-      return new ArrayList<String>(result).subList(startingAt, Math.min(fetch, result.size()));
+      return new ArrayList<String>(result).subList(startingAt, startingAt + Math.min(fetch, result.size()));
     } catch (Exception e) {
       throw new BException(e.getMessage(), e);
     }
@@ -1876,7 +1878,15 @@ public class BlurControllerServer extends TableAdmin implements Iface {
         CommandStatus cs = scatterGather(cluster, new BlurCommand<CommandStatus>() {
           @Override
           public CommandStatus call(Client client) throws BlurException, TException {
-            return client.commandStatus(commandExecutionId);
+            try {
+              return client.commandStatus(commandExecutionId);
+            } catch (BlurException e) {
+              String message = e.getMessage();
+              if (message.startsWith("NOT_FOUND")) {
+                return null;
+              }
+              throw e;
+            }
           }
         }, new Merger<CommandStatus>() {
           @Override
@@ -1884,12 +1894,16 @@ public class BlurControllerServer extends TableAdmin implements Iface {
             CommandStatus commandStatus = null;
             while (service.getRemainingCount() > 0) {
               Future<CommandStatus> future = service.poll(_defaultParallelCallTimeout, TimeUnit.MILLISECONDS, true);
-              commandStatus = mergeCommandStatus(commandStatus, service.getResultThrowException(future));
+              commandStatus = CommandStatusUtil.mergeCommandStatus(commandStatus,
+                  service.getResultThrowException(future));
             }
             return commandStatus;
           }
         });
-        commandStatus = mergeCommandStatus(commandStatus, cs);
+        commandStatus = CommandStatusUtil.mergeCommandStatus(commandStatus, cs);
+      }
+      if (commandStatus == null) {
+        throw new BException("NOT_FOUND {0}", commandExecutionId);
       }
       return commandStatus;
     } catch (Exception e) {
@@ -1897,60 +1911,6 @@ public class BlurControllerServer extends TableAdmin implements Iface {
     }
   }
 
-  private static CommandStatus mergeCommandStatus(CommandStatus cs1, CommandStatus cs2) {
-    if (cs1 == null && cs2 == null) {
-      return null;
-    } else if (cs1 == null) {
-      return cs2;
-    } else if (cs2 == null) {
-      return cs1;
-    } else {
-      Map<String, Map<CommandStatusState, Long>> serverStateMap1 = cs1.getServerStateMap();
-      Map<String, Map<CommandStatusState, Long>> serverStateMap2 = cs2.getServerStateMap();
-      Map<String, Map<CommandStatusState, Long>> merge = mergeServerStateMap(serverStateMap1, serverStateMap2);
-      return new CommandStatus(cs1.getExecutionId(), cs1.getCommandName(), cs1.getArguments(), merge, cs1.getUser());
-    }
-  }
-
-  private static Map<String, Map<CommandStatusState, Long>> mergeServerStateMap(
-      Map<String, Map<CommandStatusState, Long>> serverStateMap1,
-      Map<String, Map<CommandStatusState, Long>> serverStateMap2) {
-    Map<String, Map<CommandStatusState, Long>> result = new HashMap<String, Map<CommandStatusState, Long>>();
-    Set<String> keys = new HashSet<String>();
-    keys.addAll(serverStateMap1.keySet());
-    keys.addAll(serverStateMap2.keySet());
-    for (String key : keys) {
-      Map<CommandStatusState, Long> css1 = serverStateMap2.get(key);
-      Map<CommandStatusState, Long> css2 = serverStateMap2.get(key);
-      result.put(key, mergeCommandStatusState(css1, css2));
-    }
-    return result;
-  }
-
-  private static Map<CommandStatusState, Long> mergeCommandStatusState(Map<CommandStatusState, Long> css1,
-      Map<CommandStatusState, Long> css2) {
-    if (css1 == null && css2 == null) {
-      return new HashMap<CommandStatusState, Long>();
-    } else if (css1 == null) {
-      return css2;
-    } else if (css2 == null) {
-      return css1;
-    } else {
-      Map<CommandStatusState, Long> result = new HashMap<CommandStatusState, Long>(css1);
-      for (Entry<CommandStatusState, Long> e : css2.entrySet()) {
-        CommandStatusState key = e.getKey();
-        Long l = result.get(key);
-        Long value = e.getValue();
-        if (l == null) {
-          result.put(key, value);
-        } else {
-          result.put(key, l + value);
-        }
-      }
-      return result;
-    }
-  }
-
   @Override
   public void commandCancel(String commandExecutionId) throws BlurException, TException {
     try {

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-core/src/main/java/org/apache/blur/thrift/BlurShardServer.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/thrift/BlurShardServer.java b/blur-core/src/main/java/org/apache/blur/thrift/BlurShardServer.java
index 08b4400..ff03210 100644
--- a/blur-core/src/main/java/org/apache/blur/thrift/BlurShardServer.java
+++ b/blur-core/src/main/java/org/apache/blur/thrift/BlurShardServer.java
@@ -684,7 +684,11 @@ public class BlurShardServer extends TableAdmin implements Iface {
   @Override
   public CommandStatus commandStatus(String commandExecutionId) throws BlurException, TException {
     try {
-      return _commandManager.getCommandStatus(commandExecutionId);
+      CommandStatus commandStatus = _commandManager.getCommandStatus(commandExecutionId);
+      if (commandStatus == null) {
+        throw new BException("NOT_FOUND {0}", commandExecutionId);
+      }
+      return commandStatus;
     } catch (Exception e) {
       throw new BException(e.getMessage(), e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-core/src/main/java/org/apache/blur/thrift/ThriftBlurShardServer.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/thrift/ThriftBlurShardServer.java b/blur-core/src/main/java/org/apache/blur/thrift/ThriftBlurShardServer.java
index 0b4e290..46bd8b0 100644
--- a/blur-core/src/main/java/org/apache/blur/thrift/ThriftBlurShardServer.java
+++ b/blur-core/src/main/java/org/apache/blur/thrift/ThriftBlurShardServer.java
@@ -316,7 +316,7 @@ public class ThriftBlurShardServer extends ThriftServer {
     StreamServer streamServer;
     int streamThreadCount = configuration.getInt(BLUR_STREAM_SERVER_THREADS, 100);
     if (streamThreadCount > 0) {
-      StreamProcessor streamProcessor = new StreamProcessor(indexServer, tmpPath);
+      StreamProcessor streamProcessor = new StreamProcessor(indexServer, tmpPath, config);
       streamServer = new StreamServer(0, streamThreadCount, streamProcessor);
       streamServer.start();
       configuration.setInt(BLUR_STREAM_SERVER_RUNNING_PORT, streamServer.getPort());

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-core/src/main/java/org/apache/blur/utils/GCWatcher.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/utils/GCWatcher.java b/blur-core/src/main/java/org/apache/blur/utils/GCWatcher.java
index c9c3774..3242931 100644
--- a/blur-core/src/main/java/org/apache/blur/utils/GCWatcher.java
+++ b/blur-core/src/main/java/org/apache/blur/utils/GCWatcher.java
@@ -16,24 +16,50 @@
  */
 package org.apache.blur.utils;
 
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryUsage;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Properties;
 
 public class GCWatcher {
 
   private static final String JAVA_VERSION = "java.version";
   private static final String _1_7 = "1.7";
+  private static final String _1_8 = "1.8";
   private static final boolean JDK7;
 
   static {
     Properties properties = System.getProperties();
     String javaVersion = properties.getProperty(JAVA_VERSION);
-    if (javaVersion.startsWith(_1_7)) {
+    if (javaVersion.startsWith(_1_7) || javaVersion.startsWith(_1_8)) {
       JDK7 = true;
     } else {
       JDK7 = false;
     }
   }
 
+  public static void main(String[] args) {
+    GCWatcher.init(0.50);
+
+    GCWatcher.registerAction(new GCAction() {
+      @Override
+      public void takeAction() throws Exception {
+        System.out.println("OOM");
+        System.exit(0);
+      }
+    });
+
+    List<byte[]> lst = new ArrayList<byte[]>();
+
+    while (true) {
+      lst.add(new byte[1_000_000]);
+      MemoryUsage heapMemoryUsage = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
+      System.out.println(heapMemoryUsage.getMax() + " " + heapMemoryUsage.getUsed());
+    }
+
+  }
+
   /**
    * Initializes the GCWatcher to watch for any garbage collection that leaves
    * more then the given ratio free. If more remains then all the given

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-core/src/main/java/org/apache/blur/utils/GCWatcherJdk6.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/utils/GCWatcherJdk6.java b/blur-core/src/main/java/org/apache/blur/utils/GCWatcherJdk6.java
index 03bf6bb..2eb9f56 100644
--- a/blur-core/src/main/java/org/apache/blur/utils/GCWatcherJdk6.java
+++ b/blur-core/src/main/java/org/apache/blur/utils/GCWatcherJdk6.java
@@ -151,7 +151,7 @@ public class GCWatcherJdk6 extends TimerTask {
         }
         _lastIndex = _gcInfo.getIndex();
       }
-    } catch (Exception e) {
+    } catch (Throwable e) {
       e.printStackTrace();
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-core/src/test/java/org/apache/blur/manager/writer/IndexImporterTest.java
----------------------------------------------------------------------
diff --git a/blur-core/src/test/java/org/apache/blur/manager/writer/IndexImporterTest.java b/blur-core/src/test/java/org/apache/blur/manager/writer/IndexImporterTest.java
index ae635c3..f46b184 100644
--- a/blur-core/src/test/java/org/apache/blur/manager/writer/IndexImporterTest.java
+++ b/blur-core/src/test/java/org/apache/blur/manager/writer/IndexImporterTest.java
@@ -41,12 +41,15 @@ import org.apache.blur.thrift.generated.Column;
 import org.apache.blur.thrift.generated.Record;
 import org.apache.blur.thrift.generated.RowMutation;
 import org.apache.blur.thrift.generated.TableDescriptor;
+import org.apache.blur.utils.BlurConstants;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
@@ -292,6 +295,20 @@ public class IndexImporterTest {
   }
 
   @Test
+  public void testIndexImporterWithCorrectRowIdShardCombinationWithFastImport() throws IOException {
+    List<Field> document = _fieldManager.getFields("1", genRecord("1"));
+    document.add(new StringField(BlurConstants.NEW_ROW, BlurConstants.PRIME_DOC_VALUE, Store.NO));
+    _commitWriter.addDocument(document);
+    _commitWriter.commit();
+    _commitWriter.close();
+    _indexImporter.run();
+    assertFalse(_fileSystem.exists(_path));
+    assertFalse(_fileSystem.exists(_badRowIdsPath));
+    assertTrue(_fileSystem.exists(_inUsePath));
+    validateIndex();
+  }
+
+  @Test
   public void testIndexImporterWithWrongRowIdShardCombination() throws IOException {
     List<Field> document = _fieldManager.getFields("2", genRecord("1"));
     _commitWriter.addDocument(document);

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-document-security/src/main/java/org/apache/blur/lucene/security/index/FilterAccessControlFactory.java
----------------------------------------------------------------------
diff --git a/blur-document-security/src/main/java/org/apache/blur/lucene/security/index/FilterAccessControlFactory.java b/blur-document-security/src/main/java/org/apache/blur/lucene/security/index/FilterAccessControlFactory.java
index 059ad05..ea7e5ad 100644
--- a/blur-document-security/src/main/java/org/apache/blur/lucene/security/index/FilterAccessControlFactory.java
+++ b/blur-document-security/src/main/java/org/apache/blur/lucene/security/index/FilterAccessControlFactory.java
@@ -338,6 +338,7 @@ public class FilterAccessControlFactory extends AccessControlFactory {
       }
       List<IndexableField> result = new ArrayList<IndexableField>();
       for (IndexableField field : fields) {
+        // If field is to be indexed and is to be read masked.
         if (fieldsToMask.contains(field.name())) {
           // If field is a doc value, then don't bother indexing.
           if (!isDocValue(field)) {

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/01416562/blur-document-security/src/main/java/org/apache/blur/lucene/security/search/BitSetDocumentVisibilityFilterCacheStrategy.java
----------------------------------------------------------------------
diff --git a/blur-document-security/src/main/java/org/apache/blur/lucene/security/search/BitSetDocumentVisibilityFilterCacheStrategy.java b/blur-document-security/src/main/java/org/apache/blur/lucene/security/search/BitSetDocumentVisibilityFilterCacheStrategy.java
index 8a142cf..37e17a6 100644
--- a/blur-document-security/src/main/java/org/apache/blur/lucene/security/search/BitSetDocumentVisibilityFilterCacheStrategy.java
+++ b/blur-document-security/src/main/java/org/apache/blur/lucene/security/search/BitSetDocumentVisibilityFilterCacheStrategy.java
@@ -31,6 +31,7 @@ import org.apache.lucene.index.IndexReader.ReaderClosedListener;
 import org.apache.lucene.index.SegmentReader;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.OpenBitSet;
 
@@ -62,12 +63,16 @@ public class BitSetDocumentVisibilityFilterCacheStrategy extends DocumentVisibil
 
   @Override
   public Builder createBuilder(String fieldName, BytesRef term, final AtomicReader reader) {
-    final OpenBitSet bitSet = new OpenBitSet(reader.maxDoc());
+    int maxDoc = reader.maxDoc();
     final Key key = new Key(fieldName, term, reader.getCoreCacheKey());
     LOG.debug("Creating new bitset for key [" + key + "] on index [" + reader + "]");
     return new Builder() {
+
+      private OpenBitSet bitSet = new OpenBitSet(maxDoc);
+
       @Override
       public void or(DocIdSetIterator it) throws IOException {
+        LOG.debug("Building bitset for key [" + key + "]");
         int doc;
         while ((doc = it.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
           bitSet.set(doc);
@@ -76,7 +81,6 @@ public class BitSetDocumentVisibilityFilterCacheStrategy extends DocumentVisibil
 
       @Override
       public DocIdSet getDocIdSet() throws IOException {
-        LOG.debug("Building bitset for key [" + key + "]");
         SegmentReader segmentReader = getSegmentReader(reader);
         segmentReader.addReaderClosedListener(new ReaderClosedListener() {
           @Override
@@ -88,12 +92,172 @@ public class BitSetDocumentVisibilityFilterCacheStrategy extends DocumentVisibil
             }
           }
         });
-        _cache.put(key, bitSet);
-        return bitSet;
+        long cardinality = bitSet.cardinality();
+        DocIdSet cacheDocIdSet;
+        if (isFullySet(maxDoc, bitSet, cardinality)) {
+          cacheDocIdSet = getFullySetDocIdSet(maxDoc);
+        } else if (isFullyEmpty(bitSet, cardinality)) {
+          cacheDocIdSet = getFullyEmptyDocIdSet(maxDoc);
+        } else {
+          cacheDocIdSet = bitSet;
+        }
+        _cache.put(key, cacheDocIdSet);
+        return cacheDocIdSet;
+      }
+    };
+  }
+
+  public static DocIdSet getFullyEmptyDocIdSet(int maxDoc) {
+    Bits bits = getFullyEmptyBits(maxDoc);
+    return new DocIdSet() {
+      @Override
+      public DocIdSetIterator iterator() throws IOException {
+        return getFullyEmptyDocIdSetIterator(maxDoc);
+      }
+
+      @Override
+      public Bits bits() throws IOException {
+        return bits;
+      }
+
+      @Override
+      public boolean isCacheable() {
+        return true;
+      }
+    };
+  }
+
+  public static DocIdSetIterator getFullyEmptyDocIdSetIterator(int maxDoc) {
+    return new DocIdSetIterator() {
+
+      private int _docId = -1;
+
+      @Override
+      public int docID() {
+        return _docId;
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        return _docId = DocIdSetIterator.NO_MORE_DOCS;
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        return _docId = DocIdSetIterator.NO_MORE_DOCS;
+      }
+
+      @Override
+      public long cost() {
+        return 0;
+      }
+    };
+  }
+
+  public static Bits getFullyEmptyBits(int maxDoc) {
+    return new Bits() {
+      @Override
+      public boolean get(int index) {
+        return false;
+      }
+
+      @Override
+      public int length() {
+        return maxDoc;
+      }
+    };
+  }
+
+  public static DocIdSet getFullySetDocIdSet(int maxDoc) {
+    Bits bits = getFullySetBits(maxDoc);
+    return new DocIdSet() {
+      @Override
+      public DocIdSetIterator iterator() throws IOException {
+        return getFullySetDocIdSetIterator(maxDoc);
+      }
+
+      @Override
+      public Bits bits() throws IOException {
+        return bits;
+      }
+
+      @Override
+      public boolean isCacheable() {
+        return true;
+      }
+    };
+  }
+
+  public static DocIdSetIterator getFullySetDocIdSetIterator(int maxDoc) {
+    return new DocIdSetIterator() {
+
+      private int _docId = -1;
+
+      @Override
+      public int advance(int target) throws IOException {
+        if (_docId == DocIdSetIterator.NO_MORE_DOCS) {
+          return DocIdSetIterator.NO_MORE_DOCS;
+        }
+        _docId = target;
+        if (_docId >= maxDoc) {
+          return _docId = DocIdSetIterator.NO_MORE_DOCS;
+        }
+        return _docId;
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        if (_docId == DocIdSetIterator.NO_MORE_DOCS) {
+          return DocIdSetIterator.NO_MORE_DOCS;
+        }
+        _docId++;
+        if (_docId >= maxDoc) {
+          return _docId = DocIdSetIterator.NO_MORE_DOCS;
+        }
+        return _docId;
+      }
+
+      @Override
+      public int docID() {
+        return _docId;
+      }
+
+      @Override
+      public long cost() {
+        return 0l;
       }
+
     };
   }
 
+  public static Bits getFullySetBits(int maxDoc) {
+    return new Bits() {
+      @Override
+      public boolean get(int index) {
+        return true;
+      }
+
+      @Override
+      public int length() {
+        return maxDoc;
+      }
+    };
+  }
+
+  public static boolean isFullyEmpty(OpenBitSet bitSet, long cardinality) {
+    if (cardinality == 0) {
+      return true;
+    }
+    return false;
+  }
+
+  public static boolean isFullySet(int maxDoc, OpenBitSet bitSet, long cardinality) {
+    if (cardinality >= maxDoc) {
+      return true;
+    }
+    return false;
+  }
+
   public static SegmentReader getSegmentReader(IndexReader indexReader) throws IOException {
     if (indexReader instanceof SegmentReader) {
       return (SegmentReader) indexReader;


[09/13] git commit: Adding blur indexer project.

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/LookupBuilderReducer.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/LookupBuilderReducer.java b/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/LookupBuilderReducer.java
new file mode 100644
index 0000000..43763cf
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/LookupBuilderReducer.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.blur.indexer.mapreduce;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.blur.BlurConfiguration;
+import org.apache.blur.indexer.BlurIndexCounter;
+import org.apache.blur.indexer.IndexerJobDriver;
+import org.apache.blur.indexer.MergeSortRowIdMatcher;
+import org.apache.blur.indexer.MergeSortRowIdMatcher.Action;
+import org.apache.blur.manager.BlurPartitioner;
+import org.apache.blur.manager.writer.SnapshotIndexDeletionPolicy;
+import org.apache.blur.mapreduce.lib.BlurInputFormat;
+import org.apache.blur.mapreduce.lib.BlurOutputFormat;
+import org.apache.blur.store.BlockCacheDirectoryFactoryV2;
+import org.apache.blur.store.hdfs.HdfsDirectory;
+import org.apache.blur.thrift.generated.TableDescriptor;
+import org.apache.blur.utils.BlurConstants;
+import org.apache.blur.utils.ShardUtil;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexCommit;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.store.Directory;
+
+import com.google.common.io.Closer;
+
+public class LookupBuilderReducer extends Reducer<Text, NullWritable, Text, BooleanWritable> {
+
+  public static final String BLUR_CACHE_DIR_TOTAL_BYTES = "blur.cache.dir.total.bytes";
+  private Counter _rowIds;
+  private Counter _rowIdsToUpdate;
+
+  private MergeSortRowIdMatcher _matcher;
+  private int _numberOfShardsInTable;
+  private Configuration _configuration;
+  private String _snapshot;
+  private Path _tablePath;
+  private Counter _rowIdsFromIndex;
+  private long _totalNumberOfBytes;
+  private Action _action;
+  private Closer _closer;
+  private Path _cachePath;
+  private String _table;
+  private Writer _writer;
+
+  @Override
+  protected void setup(Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) throws IOException,
+      InterruptedException {
+    _configuration = context.getConfiguration();
+    _rowIds = context.getCounter(BlurIndexCounter.ROW_IDS_FROM_NEW_DATA);
+    _rowIdsToUpdate = context.getCounter(BlurIndexCounter.ROW_IDS_TO_UPDATE_FROM_NEW_DATA);
+    _rowIdsFromIndex = context.getCounter(BlurIndexCounter.ROW_IDS_FROM_INDEX);
+    TableDescriptor tableDescriptor = BlurOutputFormat.getTableDescriptor(_configuration);
+    _numberOfShardsInTable = tableDescriptor.getShardCount();
+    _tablePath = new Path(tableDescriptor.getTableUri());
+    _snapshot = ExistingDataIndexLookupMapper.getSnapshot(_configuration);
+    _totalNumberOfBytes = _configuration.getLong(BLUR_CACHE_DIR_TOTAL_BYTES, 128 * 1024 * 1024);
+    _cachePath = BlurInputFormat.getLocalCachePath(_configuration);
+    _table = tableDescriptor.getName();
+    _closer = Closer.create();
+  }
+
+  @Override
+  protected void reduce(Text rowId, Iterable<NullWritable> nothing,
+      Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) throws IOException, InterruptedException {
+    if (_matcher == null) {
+      _matcher = getMergeSortRowIdMatcher(rowId, context);
+    }
+    if (_writer == null) {
+      _writer = getRowIdWriter(rowId, context);
+    }
+    _writer.append(rowId, NullWritable.get());
+    _rowIds.increment(1);
+    if (_action == null) {
+      _action = new Action() {
+        @Override
+        public void found(Text rowId) throws IOException {
+          _rowIdsToUpdate.increment(1);
+          try {
+            context.write(rowId, new BooleanWritable(true));
+          } catch (InterruptedException e) {
+            throw new IOException(e);
+          }
+        }
+      };
+    }
+    _matcher.lookup(rowId, _action);
+  }
+
+  private Writer getRowIdWriter(Text rowId, Reducer<Text, NullWritable, Text, BooleanWritable>.Context context)
+      throws IOException {
+    BlurPartitioner blurPartitioner = new BlurPartitioner();
+    int shard = blurPartitioner.getShard(rowId, _numberOfShardsInTable);
+    String shardName = ShardUtil.getShardName(shard);
+    Path cachePath = MergeSortRowIdMatcher.getCachePath(_cachePath, _table, shardName);
+    Configuration configuration = context.getConfiguration();
+    String uuid = configuration.get(IndexerJobDriver.BLUR_UPDATE_ID);
+    Path tmpPath = new Path(cachePath, uuid + "_" + getAttemptString(context));
+    return _closer.register(MergeSortRowIdMatcher.createWriter(_configuration, tmpPath));
+  }
+
+  private String getAttemptString(Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) {
+    TaskAttemptID taskAttemptID = context.getTaskAttemptID();
+    return taskAttemptID.toString();
+  }
+
+  @Override
+  protected void cleanup(Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) throws IOException,
+      InterruptedException {
+    _closer.close();
+  }
+
+  private MergeSortRowIdMatcher getMergeSortRowIdMatcher(Text rowId,
+      Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) throws IOException {
+    BlurPartitioner blurPartitioner = new BlurPartitioner();
+    int shard = blurPartitioner.getShard(rowId, _numberOfShardsInTable);
+    String shardName = ShardUtil.getShardName(shard);
+
+    Path shardPath = new Path(_tablePath, shardName);
+    HdfsDirectory hdfsDirectory = new HdfsDirectory(_configuration, shardPath);
+    SnapshotIndexDeletionPolicy policy = new SnapshotIndexDeletionPolicy(_configuration,
+        SnapshotIndexDeletionPolicy.getGenerationsPath(shardPath));
+    Long generation = policy.getGeneration(_snapshot);
+    if (generation == null) {
+      hdfsDirectory.close();
+      throw new IOException("Snapshot [" + _snapshot + "] not found in shard [" + shardPath + "]");
+    }
+
+    BlurConfiguration bc = new BlurConfiguration();
+    BlockCacheDirectoryFactoryV2 blockCacheDirectoryFactoryV2 = new BlockCacheDirectoryFactoryV2(bc,
+        _totalNumberOfBytes);
+    _closer.register(blockCacheDirectoryFactoryV2);
+    Directory dir = blockCacheDirectoryFactoryV2.newDirectory("table", "shard", hdfsDirectory, null);
+    List<IndexCommit> listCommits = DirectoryReader.listCommits(dir);
+    IndexCommit indexCommit = ExistingDataIndexLookupMapper.findIndexCommit(listCommits, generation, shardPath);
+    DirectoryReader reader = DirectoryReader.open(indexCommit);
+    _rowIdsFromIndex.setValue(getTotalNumberOfRowIds(reader));
+
+    Path cachePath = MergeSortRowIdMatcher.getCachePath(_cachePath, _table, shardName);
+    return new MergeSortRowIdMatcher(dir, generation, _configuration, cachePath, context);
+  }
+
+  private long getTotalNumberOfRowIds(DirectoryReader reader) throws IOException {
+    long total = 0;
+    List<AtomicReaderContext> leaves = reader.leaves();
+    for (AtomicReaderContext context : leaves) {
+      AtomicReader atomicReader = context.reader();
+      Terms terms = atomicReader.terms(BlurConstants.ROW_ID);
+      long expectedInsertions = terms.size();
+      if (expectedInsertions < 0) {
+        return -1;
+      }
+      total += expectedInsertions;
+    }
+    return total;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/NewDataMapper.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/NewDataMapper.java b/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/NewDataMapper.java
new file mode 100644
index 0000000..c5ea87a
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/NewDataMapper.java
@@ -0,0 +1,85 @@
+/**
+ * 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.blur.indexer.mapreduce;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+
+import org.apache.blur.indexer.BlurIndexCounter;
+import org.apache.blur.mapreduce.lib.BlurRecord;
+import org.apache.blur.mapreduce.lib.update.IndexKey;
+import org.apache.blur.mapreduce.lib.update.IndexValue;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+public class NewDataMapper extends Mapper<Text, BlurRecord, IndexKey, IndexValue> {
+
+  private static final IndexValue EMPTY_RECORD = new IndexValue();
+  private long _timestamp;
+  private Counter _newRecords;
+
+  @Override
+  protected void setup(Context context) throws IOException, InterruptedException {
+    InputSplit inputSplit = context.getInputSplit();
+    FileSplit fileSplit = getFileSplit(inputSplit);
+    Path path = fileSplit.getPath();
+    FileSystem fileSystem = path.getFileSystem(context.getConfiguration());
+    FileStatus fileStatus = fileSystem.getFileStatus(path);
+    _timestamp = fileStatus.getModificationTime();
+    _newRecords = context.getCounter(BlurIndexCounter.NEW_RECORDS);
+  }
+
+  private FileSplit getFileSplit(InputSplit inputSplit) throws IOException {
+    if (inputSplit instanceof FileSplit) {
+      return (FileSplit) inputSplit;
+    }
+    if (inputSplit.getClass().getName().equals("org.apache.hadoop.mapreduce.lib.input.TaggedInputSplit")) {
+      try {
+        Field declaredField = inputSplit.getClass().getDeclaredField("inputSplit");
+        declaredField.setAccessible(true);
+        return getFileSplit((InputSplit) declaredField.get(inputSplit));
+      } catch (NoSuchFieldException e) {
+        throw new IOException(e);
+      } catch (SecurityException e) {
+        throw new IOException(e);
+      } catch (IllegalArgumentException e) {
+        throw new IOException(e);
+      } catch (IllegalAccessException e) {
+        throw new IOException(e);
+      }
+    } else {
+      throw new IOException("Unknown input split type [" + inputSplit + "] [" + inputSplit.getClass() + "]");
+    }
+  }
+
+  @Override
+  protected void map(Text key, BlurRecord blurRecord, Context context) throws IOException, InterruptedException {
+    IndexKey newDataKey = IndexKey.newData(blurRecord.getRowId(), blurRecord.getRecordId(), _timestamp);
+    context.write(newDataKey, new IndexValue(blurRecord));
+    _newRecords.increment(1L);
+
+    IndexKey newDataMarker = IndexKey.newDataMarker(blurRecord.getRowId());
+    context.write(newDataMarker, EMPTY_RECORD);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/PrunedBlurInputFormat.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/PrunedBlurInputFormat.java b/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/PrunedBlurInputFormat.java
new file mode 100644
index 0000000..bb957f5
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/PrunedBlurInputFormat.java
@@ -0,0 +1,74 @@
+/**
+ * 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.blur.indexer.mapreduce;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.blur.indexer.InputSplitPruneUtil;
+import org.apache.blur.mapreduce.lib.BlurInputFormat;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+
+public class PrunedBlurInputFormat extends BlurInputFormat {
+
+  private static final Log LOG = LogFactory.getLog(PrunedBlurInputFormat.class);
+
+  @Override
+  public List<InputSplit> getSplits(JobContext context) throws IOException {
+    Path[] dirs = getInputPaths(context);
+    Configuration configuration = context.getConfiguration();
+    List<BlurInputSplit> splits = getSplits(configuration, dirs);
+    Map<Path, List<BlurInputSplit>> splitMap = new TreeMap<Path, List<BlurInputSplit>>();
+    for (BlurInputSplit split : splits) {
+      Path path = split.getDir();
+      String table = split.getTable().toString();
+      int shard = InputSplitPruneUtil.getShardFromDirectoryPath(path);
+      long rowIdUpdateFromNewDataCount = InputSplitPruneUtil.getBlurLookupRowIdUpdateFromNewDataCount(configuration,
+          table, shard);
+      long indexCount = InputSplitPruneUtil.getBlurLookupRowIdFromIndexCount(configuration, table, shard);
+      if (rowIdUpdateFromNewDataCount == 0 || indexCount == 0) {
+        LOG.debug("Pruning id lookup input path [" + path + "] no overlapping ids.");
+      } else if (InputSplitPruneUtil.shouldLookupExecuteOnShard(configuration, table, shard)) {
+        LOG.debug("Pruning blur input path [" + split.getDir() + "]");
+      } else {
+        LOG.debug("Keeping blur input path [" + split.getDir() + "]");
+        List<BlurInputSplit> list = splitMap.get(path);
+        if (list == null) {
+          splitMap.put(path, list = new ArrayList<BlurInputSplit>());
+        }
+        list.add(split);
+      }
+    }
+    List<InputSplit> result = new ArrayList<InputSplit>();
+    for (List<BlurInputSplit> lst : splitMap.values()) {
+      BlurInputSplitColletion blurInputSplitColletion = new BlurInputSplitColletion();
+      for (BlurInputSplit blurInputSplit : lst) {
+        blurInputSplitColletion.add(blurInputSplit);
+      }
+      result.add(blurInputSplitColletion);
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/PrunedSequenceFileInputFormat.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/PrunedSequenceFileInputFormat.java b/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/PrunedSequenceFileInputFormat.java
new file mode 100644
index 0000000..49095d0
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/PrunedSequenceFileInputFormat.java
@@ -0,0 +1,76 @@
+/**
+ * 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.blur.indexer.mapreduce;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.blur.indexer.InputSplitPruneUtil;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+
+import com.google.common.base.Splitter;
+
+public class PrunedSequenceFileInputFormat<K, V> extends SequenceFileInputFormat<K, V> {
+
+  private static final Log LOG = LogFactory.getLog(PrunedSequenceFileInputFormat.class);
+
+  @Override
+  public List<InputSplit> getSplits(JobContext job) throws IOException {
+    List<InputSplit> splits = super.getSplits(job);
+    List<InputSplit> results = new ArrayList<InputSplit>();
+    Configuration configuration = job.getConfiguration();
+    String table = InputSplitPruneUtil.getTable(configuration);
+    for (InputSplit inputSplit : splits) {
+      FileSplit fileSplit = (FileSplit) inputSplit;
+      Path path = fileSplit.getPath();
+      LOG.debug("Getting shard index from path [" + path + "]");
+      String name = path.getName();
+      int shard = getShardIndex(name);
+      long rowIdUpdateFromNewDataCount = InputSplitPruneUtil.getBlurLookupRowIdUpdateFromNewDataCount(configuration,
+          table, shard);
+      long indexCount = InputSplitPruneUtil.getBlurLookupRowIdFromIndexCount(configuration, table, shard);
+      if (rowIdUpdateFromNewDataCount == 0 || indexCount == 0) {
+        LOG.debug("Pruning id lookup input path [" + path + "] no overlapping ids.");
+      } else if (InputSplitPruneUtil.shouldLookupExecuteOnShard(configuration, table, shard)) {
+        LOG.debug("Keeping id lookup input path [" + path + "]");
+        results.add(inputSplit);
+      } else {
+        LOG.debug("Pruning id lookup input path [" + path + "]");
+      }
+    }
+    return results;
+  }
+
+  private int getShardIndex(String name) {
+    // based on file format of "part-r-00000", etc
+    Iterable<String> split = Splitter.on('-').split(name);
+    List<String> parts = new ArrayList<String>();
+    for (String s : split) {
+      parts.add(s);
+    }
+    return Integer.parseInt(parts.get(2));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
deleted file mode 100644
index 590ba83..0000000
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
+++ /dev/null
@@ -1,27 +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.blur.mapreduce.lib.update;
-
-public enum BlurIndexCounter {
-
-  NEW_RECORDS, ROW_IDS_FROM_INDEX, ROW_IDS_TO_UPDATE_FROM_NEW_DATA, ROW_IDS_FROM_NEW_DATA,
-
-  INPUT_FORMAT_MAPPER, INPUT_FORMAT_EXISTING_RECORDS,
-
-  LOOKUP_MAPPER, LOOKUP_MAPPER_EXISTING_RECORDS, LOOKUP_MAPPER_ROW_LOOKUP_ATTEMPT
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
deleted file mode 100644
index f56b731..0000000
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
+++ /dev/null
@@ -1,378 +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.blur.mapreduce.lib.update;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URL;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.blur.log.Log;
-import org.apache.blur.log.LogFactory;
-import org.apache.blur.mapreduce.lib.BlurInputFormat;
-import org.apache.blur.thirdparty.thrift_0_9_0.TException;
-import org.apache.blur.thrift.BlurClient;
-import org.apache.blur.thrift.generated.Blur.Iface;
-import org.apache.blur.thrift.generated.BlurException;
-import org.apache.blur.thrift.generated.TableDescriptor;
-import org.apache.blur.thrift.generated.TableStats;
-import org.apache.blur.utils.BlurConstants;
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.mapreduce.Cluster;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobID;
-import org.apache.hadoop.mapreduce.JobStatus;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.xml.DOMConfigurator;
-
-public class ClusterDriver extends Configured implements Tool {
-
-  private static final String BLUR_ENV = "blur.env";
-  private static final Log LOG = LogFactory.getLog(ClusterDriver.class);
-  private static final String _SEP = "_";
-  private static final String IMPORT = "import";
-
-  public static void main(String[] args) throws Exception {
-    String logFilePath = System.getenv("BLUR_INDEXER_LOG_FILE");
-    System.out.println("Log file path [" + logFilePath + "]");
-    System.setProperty("BLUR_INDEXER_LOG_FILE", logFilePath);
-    URL url = ClusterDriver.class.getResource("/program-log4j.xml");
-    if (url != null) {
-      LOG.info("Reseting log4j config from classpath resource [{0}]", url);
-      LogManager.resetConfiguration();
-      DOMConfigurator.configure(url);
-    }
-    int res = ToolRunner.run(new Configuration(), new ClusterDriver(), args);
-    System.exit(res);
-  }
-
-  @Override
-  public int run(String[] args) throws Exception {
-    int c = 0;
-    final String blurEnv = args[c++];
-    final String blurZkConnection = args[c++];
-    final String extraConfig = args[c++];
-    final int reducerMultiplier = Integer.parseInt(args[c++]);
-    final Configuration conf = getConf();
-
-    final ExecutorService service = Executors.newCachedThreadPool();
-    final AtomicBoolean running = new AtomicBoolean();
-    running.set(true);
-
-    // Load configs for all filesystems.
-    Path path = new Path(extraConfig);
-    Configuration mergeHdfsConfigs = HdfsConfigurationNamespaceMerge.mergeHdfsConfigs(path.getFileSystem(conf), path);
-    conf.addResource(mergeHdfsConfigs);
-    conf.set(BlurConstants.BLUR_ZOOKEEPER_CONNECTION, blurZkConnection);
-    conf.set(BLUR_ENV, blurEnv);
-
-    final Iface client = BlurClient.getClientFromZooKeeperConnectionStr(blurZkConnection);
-
-    stopAllExistingMRJobs(blurEnv, conf);
-    cleanUpOldImportDirs(client, conf);
-    moveInprogressDirsBackToNew(client, conf);
-    unlockLockedTables(client);
-
-    Map<String, Future<Void>> futures = new HashMap<String, Future<Void>>();
-    while (running.get()) {
-      LOG.debug("Starting index update check for blur cluster [" + blurZkConnection + "].");
-      try {
-        List<String> tableList = client.tableList();
-        startMissingIndexerThreads(tableList, service, futures, blurZkConnection, conf, client, reducerMultiplier);
-      } catch (TException t) {
-        LOG.error("Unknown Blur Thrift Error, Retrying...", t);
-      }
-      Thread.sleep(TimeUnit.SECONDS.toMillis(10));
-    }
-    return 0;
-  }
-
-  private void unlockLockedTables(Iface client) throws BlurException, TException {
-    List<String> tableList = client.tableList();
-    for (String table : tableList) {
-      TableDescriptor tableDescriptor = client.describe(table);
-      if (tableDescriptor.isEnabled()) {
-        unlockLockedTables(client, table);
-      }
-    }
-  }
-
-  private void unlockLockedTables(Iface client, String table) throws BlurException, TException {
-    Map<String, List<String>> listSnapshots = client.listSnapshots(table);
-    for (Entry<String, List<String>> e : listSnapshots.entrySet()) {
-      List<String> value = e.getValue();
-      if (value.contains(FasterDriver.MRUPDATE_SNAPSHOT)) {
-        LOG.info("Unlocking table [{0}]", table);
-        client.removeSnapshot(table, FasterDriver.MRUPDATE_SNAPSHOT);
-        return;
-      }
-    }
-  }
-
-  private void moveInprogressDirsBackToNew(Iface client, Configuration conf) throws BlurException, TException,
-      IOException {
-    List<String> tableList = client.tableList();
-    for (String table : tableList) {
-      String mrIncWorkingPathStr = getMRIncWorkingPathStr(client, table);
-      Path mrIncWorkingPath = new Path(mrIncWorkingPathStr);
-      Path newData = new Path(mrIncWorkingPath, FasterDriver.NEW);
-      Path inprogressData = new Path(mrIncWorkingPath, FasterDriver.INPROGRESS);
-      FileSystem fileSystem = inprogressData.getFileSystem(conf);
-      FileStatus[] listStatus = fileSystem.listStatus(inprogressData);
-      for (FileStatus fileStatus : listStatus) {
-        Path src = fileStatus.getPath();
-        Path dst = new Path(newData, src.getName());
-        if (fileSystem.rename(src, dst)) {
-          LOG.info("Moved [{0}] to [{1}] to be reprocessed.", src, dst);
-        } else {
-          LOG.error("Could not move [{0}] to [{1}] to be reprocessed.", src, dst);
-        }
-      }
-    }
-  }
-
-  private void cleanUpOldImportDirs(Iface client, Configuration conf) throws BlurException, TException, IOException {
-    List<String> tableList = client.tableList();
-    for (String table : tableList) {
-      cleanUpOldImportDirs(client, conf, table);
-    }
-  }
-
-  private void cleanUpOldImportDirs(Iface client, Configuration conf, String table) throws BlurException, TException,
-      IOException {
-    TableDescriptor descriptor = client.describe(table);
-    String tableUri = descriptor.getTableUri();
-    Path tablePath = new Path(tableUri);
-    FileSystem fileSystem = tablePath.getFileSystem(getConf());
-    Path importPath = new Path(tablePath, IMPORT);
-    if (fileSystem.exists(importPath)) {
-      for (FileStatus fileStatus : fileSystem.listStatus(importPath)) {
-        Path path = fileStatus.getPath();
-        LOG.info("Removing failed import [{0}]", path);
-        fileSystem.delete(path, true);
-      }
-    }
-  }
-
-  private void stopAllExistingMRJobs(String blurEnv, Configuration conf) throws YarnException, IOException,
-      InterruptedException {
-    Cluster cluster = new Cluster(conf);
-    JobStatus[] allJobStatuses = cluster.getAllJobStatuses();
-    for (JobStatus jobStatus : allJobStatuses) {
-      if (jobStatus.isJobComplete()) {
-        continue;
-      }
-      String jobFile = jobStatus.getJobFile();
-      JobID jobID = jobStatus.getJobID();
-      Job job = cluster.getJob(jobID);
-      FileSystem fileSystem = FileSystem.get(job.getConfiguration());
-      Configuration configuration = new Configuration(false);
-      Path path = new Path(jobFile);
-      Path makeQualified = path.makeQualified(fileSystem.getUri(), fileSystem.getWorkingDirectory());
-      if (hasReadAccess(fileSystem, makeQualified)) {
-        try (FSDataInputStream in = fileSystem.open(makeQualified)) {
-          configuration.addResource(copy(in));
-        }
-        String jobBlurEnv = configuration.get(BLUR_ENV);
-        LOG.info("Checking job [{0}] has env [{1}] current env set to [{2}]", jobID, jobBlurEnv, blurEnv);
-        if (blurEnv.equals(jobBlurEnv)) {
-          LOG.info("Killing running job [{0}]", jobID);
-          job.killJob();
-        }
-      }
-    }
-  }
-
-  private static InputStream copy(FSDataInputStream input) throws IOException {
-    try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) {
-      IOUtils.copy(input, outputStream);
-      return new ByteArrayInputStream(outputStream.toByteArray());
-    }
-  }
-
-  private static boolean hasReadAccess(FileSystem fileSystem, Path p) {
-    try {
-      fileSystem.access(p, FsAction.READ);
-      return true;
-    } catch (IOException e) {
-      return false;
-    }
-  }
-
-  private Callable<Void> getCallable(final String blurZkConnection, final Configuration conf, final Iface client,
-      final String table, final int reducerMultiplier) {
-    return new Callable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        String originalThreadName = Thread.currentThread().getName();
-        try {
-          Thread.currentThread().setName(table);
-          if (!isEnabled(client, table)) {
-            LOG.info("Table [" + table + "] is not enabled.");
-            return null;
-          }
-          waitForDataToLoad(client, table);
-          LOG.debug("Starting index update for table [" + table + "].");
-          final String mrIncWorkingPathStr = getMRIncWorkingPathStr(client, table);
-          final String outputPathStr = getOutputPathStr(client, table);
-          Path path = new Path(outputPathStr);
-          FileSystem fileSystem = path.getFileSystem(getConf());
-
-          Configuration configuration = new Configuration(conf);
-          BlurInputFormat.setMaxNumberOfMaps(configuration, 10000);
-
-          FasterDriver driver = new FasterDriver();
-          driver.setConf(configuration);
-          try {
-            driver.run(new String[] { table, mrIncWorkingPathStr, outputPathStr, blurZkConnection,
-                Integer.toString(reducerMultiplier) });
-          } finally {
-            if (fileSystem.exists(path)) {
-              fileSystem.delete(path, true);
-            }
-          }
-          return null;
-        } finally {
-          Thread.currentThread().setName(originalThreadName);
-        }
-      }
-    };
-  }
-
-  private void startMissingIndexerThreads(List<String> tableList, ExecutorService service,
-      Map<String, Future<Void>> futures, final String blurZkConnection, final Configuration conf, final Iface client,
-      int reducerMultiplier) throws BlurException, TException {
-    Set<String> tables = new HashSet<String>(tableList);
-
-    // remove futures that are complete
-    for (String table : tables) {
-      Future<Void> future = futures.get(table);
-      if (future != null) {
-        if (future.isDone()) {
-          try {
-            future.get();
-          } catch (InterruptedException e) {
-            LOG.error("Unknown error while processing table [" + table + "].", e);
-          } catch (ExecutionException e) {
-            LOG.error("Unknown error while processing table [" + table + "].", e.getCause());
-          }
-          futures.remove(table);
-        } else {
-          LOG.info("Update for table [" + table + "] still running.");
-        }
-      }
-    }
-
-    // start missing tables
-    for (String table : tables) {
-      if (!futures.containsKey(table)) {
-        if (isEnabled(client, table)) {
-          Future<Void> future = service.submit(getCallable(blurZkConnection, conf, client, table, reducerMultiplier));
-          futures.put(table, future);
-        }
-      }
-    }
-  }
-
-  public static void waitForDataToLoad(Iface client, String table) throws BlurException, TException,
-      InterruptedException {
-    if (isFullyLoaded(client.tableStats(table))) {
-      return;
-    }
-    while (true) {
-      TableStats tableStats = client.tableStats(table);
-      if (isFullyLoaded(tableStats)) {
-        LOG.info("Data load complete in table [" + table + "] [" + tableStats + "]");
-        return;
-      }
-      LOG.info("Waiting for data to load in table [" + table + "] [" + tableStats + "]");
-      Thread.sleep(5000);
-    }
-  }
-
-  private static boolean isFullyLoaded(TableStats tableStats) {
-    if (tableStats.getSegmentImportInProgressCount() == 0 && tableStats.getSegmentImportPendingCount() == 0) {
-      return true;
-    }
-    return false;
-  }
-
-  private boolean isEnabled(Iface client, String table) throws BlurException, TException {
-    TableDescriptor tableDescriptor = client.describe(table);
-    return tableDescriptor.isEnabled();
-  }
-
-  private void mkdirs(FileSystem fileSystem, Path path) throws IOException {
-    if (fileSystem.exists(path)) {
-      return;
-    }
-    LOG.info("Creating path [" + path + "].");
-    if (!fileSystem.mkdirs(path)) {
-      LOG.error("Path [" + path + "] could not be created.");
-    }
-  }
-
-  public static String getMRIncWorkingPathStr(Iface client, String table) throws BlurException, TException, IOException {
-    TableDescriptor descriptor = client.describe(table);
-    Map<String, String> tableProperties = descriptor.getTableProperties();
-    if (tableProperties != null) {
-      String workingPath = tableProperties.get(BlurConstants.BLUR_BULK_UPDATE_WORKING_PATH);
-      if (workingPath != null) {
-        return workingPath;
-      }
-    }
-    throw new IOException("Table [" + table + "] does not have the property ["
-        + BlurConstants.BLUR_BULK_UPDATE_WORKING_PATH + "] setup correctly.");
-  }
-
-  private String getOutputPathStr(Iface client, String table) throws BlurException, TException, IOException {
-    TableDescriptor descriptor = client.describe(table);
-    String tableUri = descriptor.getTableUri();
-    Path tablePath = new Path(tableUri);
-    FileSystem fileSystem = tablePath.getFileSystem(getConf());
-    Path importPath = new Path(tablePath, IMPORT);
-    mkdirs(fileSystem, importPath);
-    return new Path(importPath, IMPORT + _SEP + System.currentTimeMillis() + _SEP + UUID.randomUUID().toString())
-        .toString();
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/FasterDriver.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/FasterDriver.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/FasterDriver.java
deleted file mode 100644
index f43cba5..0000000
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/FasterDriver.java
+++ /dev/null
@@ -1,486 +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.blur.mapreduce.lib.update;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.blur.log.Log;
-import org.apache.blur.log.LogFactory;
-import org.apache.blur.mapreduce.lib.BlurInputFormat;
-import org.apache.blur.mapreduce.lib.BlurOutputFormat;
-import org.apache.blur.thirdparty.thrift_0_9_0.TException;
-import org.apache.blur.thrift.BlurClient;
-import org.apache.blur.thrift.generated.Blur.Iface;
-import org.apache.blur.thrift.generated.BlurException;
-import org.apache.blur.thrift.generated.TableDescriptor;
-import org.apache.blur.thrift.generated.TableStats;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.io.BooleanWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.Counters;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.TaskReport;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-public class FasterDriver extends Configured implements Tool {
-
-  public static final String BLUR_UPDATE_ID = "blur.update.id";
-  private static final String BLUR_EXEC_TYPE = "blur.exec.type";
-  public static final String TMP = "tmp";
-
-  public enum EXEC {
-    MR_ONLY, MR_WITH_LOOKUP, AUTOMATIC
-  }
-
-  public static final String MRUPDATE_SNAPSHOT = "mrupdate-snapshot";
-  public static final String CACHE = "cache";
-  public static final String COMPLETE = "complete";
-  public static final String INPROGRESS = "inprogress";
-  public static final String NEW = "new";
-  private static final Log LOG = LogFactory.getLog(FasterDriver.class);
-
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new FasterDriver(), args);
-    System.exit(res);
-  }
-
-  static class PartitionedInputResult {
-    final Path _partitionedInputData;
-    final Counters _counters;
-    final long[] _rowIdsFromNewData;
-    final long[] _rowIdsToUpdateFromNewData;
-    final long[] _rowIdsFromIndex;
-
-    PartitionedInputResult(Path partitionedInputData, Counters counters, int shards, TaskReport[] taskReports) {
-      _partitionedInputData = partitionedInputData;
-      _counters = counters;
-      _rowIdsFromNewData = new long[shards];
-      _rowIdsToUpdateFromNewData = new long[shards];
-      _rowIdsFromIndex = new long[shards];
-      for (TaskReport tr : taskReports) {
-        int id = tr.getTaskID().getId();
-        Counters taskCounters = tr.getTaskCounters();
-        Counter total = taskCounters.findCounter(BlurIndexCounter.ROW_IDS_FROM_NEW_DATA);
-        _rowIdsFromNewData[id] = total.getValue();
-        Counter update = taskCounters.findCounter(BlurIndexCounter.ROW_IDS_TO_UPDATE_FROM_NEW_DATA);
-        _rowIdsToUpdateFromNewData[id] = update.getValue();
-        Counter index = taskCounters.findCounter(BlurIndexCounter.ROW_IDS_FROM_INDEX);
-        _rowIdsFromIndex[id] = index.getValue();
-      }
-    }
-
-  }
-
-  @Override
-  public int run(String[] args) throws Exception {
-    int c = 0;
-    if (args.length < 5) {
-      System.err
-          .println("Usage Driver <table> <mr inc working path> <output path> <zk connection> <reducer multipler> <extra config files...>");
-      return 1;
-    }
-    String table = args[c++];
-    String mrIncWorkingPathStr = args[c++];
-    String outputPathStr = args[c++];
-    String blurZkConnection = args[c++];
-    int reducerMultipler = Integer.parseInt(args[c++]);
-    for (; c < args.length; c++) {
-      String externalConfigFileToAdd = args[c];
-      getConf().addResource(new Path(externalConfigFileToAdd));
-    }
-
-    Path outputPath = new Path(outputPathStr);
-    Path mrIncWorkingPath = new Path(mrIncWorkingPathStr);
-    FileSystem fileSystem = mrIncWorkingPath.getFileSystem(getConf());
-
-    Path newData = new Path(mrIncWorkingPath, NEW);
-    Path inprogressData = new Path(mrIncWorkingPath, INPROGRESS);
-    Path completeData = new Path(mrIncWorkingPath, COMPLETE);
-    Path fileCache = new Path(mrIncWorkingPath, CACHE);
-    Path tmpPathDontDelete = new Path(mrIncWorkingPath, TMP);
-
-    Path tmpPath = new Path(tmpPathDontDelete, UUID.randomUUID().toString());
-
-    fileSystem.mkdirs(newData);
-    fileSystem.mkdirs(inprogressData);
-    fileSystem.mkdirs(completeData);
-    fileSystem.mkdirs(fileCache);
-
-    List<Path> srcPathList = new ArrayList<Path>();
-    for (FileStatus fileStatus : fileSystem.listStatus(newData)) {
-      srcPathList.add(fileStatus.getPath());
-    }
-    if (srcPathList.isEmpty()) {
-      return 0;
-    }
-
-    List<Path> inprogressPathList = new ArrayList<Path>();
-    boolean success = false;
-    Iface client = null;
-
-    EXEC exec = EXEC.valueOf(getConf().get(BLUR_EXEC_TYPE, EXEC.AUTOMATIC.name()).toUpperCase());
-
-    String uuid = UUID.randomUUID().toString();
-
-    try {
-      client = BlurClient.getClientFromZooKeeperConnectionStr(blurZkConnection);
-      TableDescriptor descriptor = client.describe(table);
-      Map<String, String> tableProperties = descriptor.getTableProperties();
-      String fastDir = tableProperties.get("blur.table.disable.fast.dir");
-      if (fastDir == null || !fastDir.equals("true")) {
-        LOG.error("Table [{0}] has blur.table.disable.fast.dir enabled, not supported in fast MR update.", table);
-        return 1;
-      }
-
-      waitForOtherSnapshotsToBeRemoved(client, table, MRUPDATE_SNAPSHOT);
-      client.createSnapshot(table, MRUPDATE_SNAPSHOT);
-      TableStats tableStats = client.tableStats(table);
-
-      inprogressPathList = movePathList(fileSystem, inprogressData, srcPathList);
-
-      switch (exec) {
-      case MR_ONLY:
-        success = runMrOnly(descriptor, inprogressPathList, table, fileCache, outputPath, reducerMultipler);
-        break;
-      case MR_WITH_LOOKUP:
-        success = runMrWithLookup(uuid, descriptor, inprogressPathList, table, fileCache, outputPath, reducerMultipler,
-            tmpPath, tableStats, MRUPDATE_SNAPSHOT);
-        break;
-      case AUTOMATIC:
-        success = runAutomatic(uuid, descriptor, inprogressPathList, table, fileCache, outputPath, reducerMultipler,
-            tmpPath, tableStats, MRUPDATE_SNAPSHOT);
-        break;
-      default:
-        throw new RuntimeException("Exec type [" + exec + "] not supported.");
-      }
-    } finally {
-      if (success) {
-        LOG.info("Associate lookup cache with new data!");
-        associateLookupCache(uuid, fileCache, outputPath);
-        LOG.info("Indexing job succeeded!");
-        client.loadData(table, outputPathStr);
-        LOG.info("Load data called");
-        movePathList(fileSystem, completeData, inprogressPathList);
-        LOG.info("Input data moved to complete");
-        ClusterDriver.waitForDataToLoad(client, table);
-        LOG.info("Data loaded");
-      } else {
-        LOG.error("Indexing job failed!");
-        movePathList(fileSystem, newData, inprogressPathList);
-      }
-      fileSystem.delete(tmpPath, true);
-      if (client != null) {
-        client.removeSnapshot(table, MRUPDATE_SNAPSHOT);
-      }
-    }
-
-    if (success) {
-      return 0;
-    } else {
-      return 1;
-    }
-  }
-
-  private void associateLookupCache(String uuid, Path fileCache, Path outputPath) throws IOException {
-    FileSystem fileSystem = fileCache.getFileSystem(getConf());
-    cleanupExtraFileFromSpecX(fileSystem, uuid, fileCache);
-    associateLookupCache(fileSystem, uuid, fileSystem.getFileStatus(fileCache), outputPath);
-  }
-
-  private void cleanupExtraFileFromSpecX(FileSystem fileSystem, String uuid, Path fileCache) throws IOException {
-    FileStatus[] listStatus = fileSystem.listStatus(fileCache);
-    List<FileStatus> uuidPaths = new ArrayList<FileStatus>();
-    for (FileStatus fs : listStatus) {
-      Path path = fs.getPath();
-      if (fs.isDirectory()) {
-        cleanupExtraFileFromSpecX(fileSystem, uuid, path);
-      } else if (path.getName().startsWith(uuid)) {
-        uuidPaths.add(fs);
-      }
-    }
-    if (uuidPaths.size() > 1) {
-      deleteIncomplete(fileSystem, uuidPaths);
-    }
-  }
-
-  private void deleteIncomplete(FileSystem fileSystem, List<FileStatus> uuidPaths) throws IOException {
-    long max = 0;
-    FileStatus keeper = null;
-    for (FileStatus fs : uuidPaths) {
-      long len = fs.getLen();
-      if (len > max) {
-        keeper = fs;
-        max = len;
-      }
-    }
-    for (FileStatus fs : uuidPaths) {
-      if (fs != keeper) {
-        LOG.info("Deleteing incomplete cache file [{0}]", fs.getPath());
-        fileSystem.delete(fs.getPath(), false);
-      }
-    }
-  }
-
-  private void associateLookupCache(FileSystem fileSystem, String uuid, FileStatus fileCache, Path outputPath)
-      throws IOException {
-    Path path = fileCache.getPath();
-    if (fileCache.isDirectory()) {
-      FileStatus[] listStatus = fileSystem.listStatus(path);
-      for (FileStatus fs : listStatus) {
-        associateLookupCache(fileSystem, uuid, fs, outputPath);
-      }
-    } else if (path.getName().startsWith(uuid)) {
-      Path parent = path.getParent();
-      String shardName = parent.getName();
-      Path indexPath = findOutputDirPath(outputPath, shardName);
-      LOG.info("Path found for shard [{0}] outputPath [{1}]", shardName, outputPath);
-      String id = MergeSortRowIdMatcher.getIdForSingleSegmentIndex(getConf(), indexPath);
-      Path file = new Path(path.getParent(), id + ".seq");
-      MergeSortRowIdMatcher.commitWriter(getConf(), file, path);
-    }
-  }
-
-  private Path findOutputDirPath(Path outputPath, String shardName) throws IOException {
-    FileSystem fileSystem = outputPath.getFileSystem(getConf());
-    Path shardPath = new Path(outputPath, shardName);
-    if (!fileSystem.exists(shardPath)) {
-      throw new IOException("Shard path [" + shardPath + "]");
-    }
-    FileStatus[] listStatus = fileSystem.listStatus(shardPath, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().endsWith(".commit");          
-      }
-    });
-    if (listStatus.length == 1) {
-      FileStatus fs = listStatus[0];
-      return fs.getPath();
-    } else {
-      throw new IOException("More than one sub dir [" + shardPath + "]");
-    }
-  }
-
-  private boolean runAutomatic(String uuid, TableDescriptor descriptor, List<Path> inprogressPathList, String table,
-      Path fileCache, Path outputPath, int reducerMultipler, Path tmpPath, TableStats tableStats, String snapshot)
-      throws ClassNotFoundException, IOException, InterruptedException {
-    PartitionedInputResult result = buildPartitionedInputData(uuid, tmpPath, descriptor, inprogressPathList, snapshot,
-        fileCache);
-
-    Job job = Job.getInstance(getConf(), "Blur Row Updater for table [" + table + "]");
-
-    InputSplitPruneUtil.setBlurLookupRowIdFromNewDataCounts(job, table, result._rowIdsFromNewData);
-    InputSplitPruneUtil.setBlurLookupRowIdUpdateFromNewDataCounts(job, table, result._rowIdsToUpdateFromNewData);
-    InputSplitPruneUtil.setBlurLookupRowIdFromIndexCounts(job, table, result._rowIdsFromIndex);
-    InputSplitPruneUtil.setTable(job, table);
-
-    BlurInputFormat.setLocalCachePath(job, fileCache);
-
-    // Existing data - This adds the copy data files first open and stream
-    // through all documents.
-    {
-      Path tablePath = new Path(descriptor.getTableUri());
-      BlurInputFormat.addTable(job, descriptor, MRUPDATE_SNAPSHOT);
-      MultipleInputs.addInputPath(job, tablePath, PrunedBlurInputFormat.class, MapperForExistingDataMod.class);
-    }
-
-    // Existing data - This adds the row id lookup
-    {
-      MapperForExistingDataWithIndexLookup.setSnapshot(job, MRUPDATE_SNAPSHOT);
-      FileInputFormat.addInputPath(job, result._partitionedInputData);
-      MultipleInputs.addInputPath(job, result._partitionedInputData, PrunedSequenceFileInputFormat.class,
-          MapperForExistingDataWithIndexLookup.class);
-    }
-
-    // New Data
-    for (Path p : inprogressPathList) {
-      FileInputFormat.addInputPath(job, p);
-      MultipleInputs.addInputPath(job, p, SequenceFileInputFormat.class, MapperForNewDataMod.class);
-    }
-
-    BlurOutputFormat.setOutputPath(job, outputPath);
-    BlurOutputFormat.setupJob(job, descriptor);
-
-    job.setReducerClass(UpdateReducer.class);
-    job.setMapOutputKeyClass(IndexKey.class);
-    job.setMapOutputValueClass(IndexValue.class);
-    job.setPartitionerClass(IndexKeyPartitioner.class);
-    job.setGroupingComparatorClass(IndexKeyWritableComparator.class);
-
-    BlurOutputFormat.setReducerMultiplier(job, reducerMultipler);
-
-    boolean success = job.waitForCompletion(true);
-    Counters counters = job.getCounters();
-    LOG.info("Counters [" + counters + "]");
-    return success;
-  }
-
-  private boolean runMrWithLookup(String uuid, TableDescriptor descriptor, List<Path> inprogressPathList, String table,
-      Path fileCache, Path outputPath, int reducerMultipler, Path tmpPath, TableStats tableStats, String snapshot)
-      throws ClassNotFoundException, IOException, InterruptedException {
-    PartitionedInputResult result = buildPartitionedInputData(uuid, tmpPath, descriptor, inprogressPathList, snapshot,
-        fileCache);
-
-    Job job = Job.getInstance(getConf(), "Blur Row Updater for table [" + table + "]");
-
-    MapperForExistingDataWithIndexLookup.setSnapshot(job, MRUPDATE_SNAPSHOT);
-    FileInputFormat.addInputPath(job, result._partitionedInputData);
-    MultipleInputs.addInputPath(job, result._partitionedInputData, SequenceFileInputFormat.class,
-        MapperForExistingDataWithIndexLookup.class);
-
-    for (Path p : inprogressPathList) {
-      FileInputFormat.addInputPath(job, p);
-      MultipleInputs.addInputPath(job, p, SequenceFileInputFormat.class, MapperForNewDataMod.class);
-    }
-
-    BlurOutputFormat.setOutputPath(job, outputPath);
-    BlurOutputFormat.setupJob(job, descriptor);
-
-    job.setReducerClass(UpdateReducer.class);
-    job.setMapOutputKeyClass(IndexKey.class);
-    job.setMapOutputValueClass(IndexValue.class);
-    job.setPartitionerClass(IndexKeyPartitioner.class);
-    job.setGroupingComparatorClass(IndexKeyWritableComparator.class);
-
-    BlurOutputFormat.setReducerMultiplier(job, reducerMultipler);
-
-    boolean success = job.waitForCompletion(true);
-    Counters counters = job.getCounters();
-    LOG.info("Counters [" + counters + "]");
-    return success;
-  }
-
-  private boolean runMrOnly(TableDescriptor descriptor, List<Path> inprogressPathList, String table, Path fileCache,
-      Path outputPath, int reducerMultipler) throws IOException, ClassNotFoundException, InterruptedException {
-    Job job = Job.getInstance(getConf(), "Blur Row Updater for table [" + table + "]");
-    Path tablePath = new Path(descriptor.getTableUri());
-    BlurInputFormat.setLocalCachePath(job, fileCache);
-    BlurInputFormat.addTable(job, descriptor, MRUPDATE_SNAPSHOT);
-    MultipleInputs.addInputPath(job, tablePath, BlurInputFormat.class, MapperForExistingDataMod.class);
-
-    for (Path p : inprogressPathList) {
-      FileInputFormat.addInputPath(job, p);
-      MultipleInputs.addInputPath(job, p, SequenceFileInputFormat.class, MapperForNewDataMod.class);
-    }
-
-    BlurOutputFormat.setOutputPath(job, outputPath);
-    BlurOutputFormat.setupJob(job, descriptor);
-
-    job.setReducerClass(UpdateReducer.class);
-    job.setMapOutputKeyClass(IndexKey.class);
-    job.setMapOutputValueClass(IndexValue.class);
-    job.setPartitionerClass(IndexKeyPartitioner.class);
-    job.setGroupingComparatorClass(IndexKeyWritableComparator.class);
-
-    BlurOutputFormat.setReducerMultiplier(job, reducerMultipler);
-
-    boolean success = job.waitForCompletion(true);
-    Counters counters = job.getCounters();
-    LOG.info("Counters [" + counters + "]");
-    return success;
-  }
-
-  private PartitionedInputResult buildPartitionedInputData(String uuid, Path tmpPath, TableDescriptor descriptor,
-      List<Path> inprogressPathList, String snapshot, Path fileCachePath) throws IOException, ClassNotFoundException,
-      InterruptedException {
-    Job job = Job.getInstance(getConf(), "Partitioning data for table [" + descriptor.getName() + "]");
-    job.getConfiguration().set(BLUR_UPDATE_ID, uuid);
-
-    // Needed for the bloom filter path information.
-    BlurOutputFormat.setTableDescriptor(job, descriptor);
-    BlurInputFormat.setLocalCachePath(job, fileCachePath);
-    MapperForExistingDataWithIndexLookup.setSnapshot(job, snapshot);
-
-    for (Path p : inprogressPathList) {
-      FileInputFormat.addInputPath(job, p);
-    }
-    Path outputPath = new Path(tmpPath, UUID.randomUUID().toString());
-    job.setJarByClass(getClass());
-    job.setMapperClass(LookupBuilderMapper.class);
-    job.setReducerClass(LookupBuilderReducer.class);
-
-    int shardCount = descriptor.getShardCount();
-    job.setNumReduceTasks(shardCount);
-    job.setInputFormatClass(SequenceFileInputFormat.class);
-    job.setMapOutputKeyClass(Text.class);
-    job.setMapOutputValueClass(NullWritable.class);
-    job.setOutputFormatClass(SequenceFileOutputFormat.class);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(BooleanWritable.class);
-    FileOutputFormat.setOutputPath(job, outputPath);
-    if (job.waitForCompletion(true)) {
-      return new PartitionedInputResult(outputPath, job.getCounters(), shardCount, job.getTaskReports(TaskType.REDUCE));
-    } else {
-      throw new IOException("Partitioning failed!");
-    }
-  }
-
-  private void waitForOtherSnapshotsToBeRemoved(Iface client, String table, String snapshot) throws BlurException,
-      TException, InterruptedException {
-    while (true) {
-      Map<String, List<String>> listSnapshots = client.listSnapshots(table);
-      boolean mrupdateSnapshots = false;
-      for (Entry<String, List<String>> e : listSnapshots.entrySet()) {
-        List<String> value = e.getValue();
-        if (value.contains(snapshot)) {
-          mrupdateSnapshots = true;
-        }
-      }
-      if (!mrupdateSnapshots) {
-        return;
-      } else {
-        LOG.info(snapshot + " Snapshot for table [{0}] already exists", table);
-        Thread.sleep(TimeUnit.SECONDS.toMillis(10));
-        LOG.info("Retrying");
-      }
-    }
-  }
-
-  private List<Path> movePathList(FileSystem fileSystem, Path dstDir, List<Path> lst) throws IOException {
-    List<Path> result = new ArrayList<Path>();
-    for (Path src : lst) {
-      Path dst = new Path(dstDir, src.getName());
-      if (fileSystem.rename(src, dst)) {
-        LOG.info("Moving [{0}] to [{1}]", src, dst);
-        result.add(dst);
-      } else {
-        LOG.error("Could not move [{0}] to [{1}]", src, dst);
-      }
-    }
-    return result;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/HdfsConfigurationNamespaceMerge.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/HdfsConfigurationNamespaceMerge.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/HdfsConfigurationNamespaceMerge.java
deleted file mode 100644
index de96d24..0000000
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/HdfsConfigurationNamespaceMerge.java
+++ /dev/null
@@ -1,131 +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.blur.mapreduce.lib.update;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-
-public class HdfsConfigurationNamespaceMerge {
-
-  private static final String DFS_NAMESERVICES = "dfs.nameservices";
-  private static final Log LOG = LogFactory.getLog(HdfsConfigurationNamespaceMerge.class);
-
-  public static void main(String[] args) throws IOException {
-    Path p = new Path("./src/main/scripts/conf/hdfs");
-
-    Configuration configuration = mergeHdfsConfigs(p.getFileSystem(new Configuration()), p);
-
-    // configuration.writeXml(System.out);
-
-    Collection<String> nameServices = configuration.getStringCollection(DFS_NAMESERVICES);
-    for (String name : nameServices) {
-      Path path = new Path("hdfs://" + name + "/");
-      FileSystem fileSystem = path.getFileSystem(configuration);
-      FileStatus[] listStatus = fileSystem.listStatus(path);
-      for (FileStatus fileStatus : listStatus) {
-        System.out.println(fileStatus.getPath());
-      }
-    }
-  }
-
-  private static boolean checkHostName(String host) {
-    try {
-      InetAddress.getAllByName(host);
-      return true;
-    } catch (UnknownHostException e) {
-      LOG.warn("Host not found [" + host + "]");
-      return false;
-    }
-  }
-
-  public static Configuration mergeHdfsConfigs(FileSystem fs, Path p) throws IOException {
-    List<Configuration> configList = new ArrayList<Configuration>();
-    gatherConfigs(fs, p, configList);
-    return merge(configList);
-  }
-
-  public static Configuration merge(List<Configuration> configList) throws IOException {
-    Configuration merge = new Configuration(false);
-    Set<String> nameServices = new HashSet<String>();
-    for (Configuration configuration : configList) {
-      String nameService = configuration.get(DFS_NAMESERVICES);
-      if (nameServices.contains(nameService)) {
-        throw new IOException("Multiple confs define namespace [" + nameService + "]");
-      }
-      nameServices.add(nameService);
-      if (shouldAdd(configuration, nameService)) {
-        for (Entry<String, String> e : configuration) {
-          String key = e.getKey();
-          if (key.contains(nameService)) {
-            String value = e.getValue();
-            merge.set(key, value);
-          }
-        }
-      }
-    }
-    merge.set(DFS_NAMESERVICES, StringUtils.join(nameServices, ","));
-    return merge;
-  }
-
-  private static boolean shouldAdd(Configuration configuration, String nameService) {
-    for (Entry<String, String> e : configuration) {
-      String key = e.getKey();
-      if (key.contains(nameService) && key.startsWith("dfs.namenode.rpc-address.")) {
-        return checkHostName(getHost(e.getValue()));
-      }
-    }
-    return false;
-  }
-
-  private static String getHost(String host) {
-    return host.substring(0, host.indexOf(":"));
-  }
-
-  public static void gatherConfigs(FileSystem fs, Path p, List<Configuration> configList) throws IOException {
-    if (fs.isFile(p)) {
-      if (p.getName().endsWith(".xml")) {
-        LOG.info("Loading file [" + p + "]");
-        Configuration configuration = new Configuration(false);
-        configuration.addResource(p);
-        configList.add(configuration);
-      } else {
-        LOG.info("Skipping file [" + p + "]");
-      }
-    } else {
-      FileStatus[] listStatus = fs.listStatus(p);
-      for (FileStatus fileStatus : listStatus) {
-        gatherConfigs(fs, fileStatus.getPath(), configList);
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/InputSplitPruneUtil.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/InputSplitPruneUtil.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/InputSplitPruneUtil.java
deleted file mode 100644
index 80d1410..0000000
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/InputSplitPruneUtil.java
+++ /dev/null
@@ -1,149 +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.blur.mapreduce.lib.update;
-
-import org.apache.blur.utils.ShardUtil;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.Job;
-
-public class InputSplitPruneUtil {
-
-  private static final String BLUR_LOOKUP_ROWID_UPDATE_FROM_NEW_DATA_COUNT_PREFIX = "blur.lookup.rowid.update.from.new.data.count";
-  private static final String BLUR_LOOKUP_ROWID_FROM_NEW_DATA_COUNT_PREFIX = "blur.lookup.rowid.from.new.data.count.";
-  private static final String BLUR_LOOKUP_ROWID_FROM_INDEX_COUNT_PREFIX = "blur.lookup.rowid.from.index.count.";
-
-  private static final String BLUR_LOOKUP_TABLE = "blur.lookup.table";
-  private static final String BLUR_LOOKUP_RATIO_PER_SHARD = "blur.lookup.ratio.per.shard";
-  private static final String BLUR_LOOKUP_MAX_TOTAL_PER_SHARD = "blur.lookup.max.total.per.shard";
-
-  private static final double DEFAULT_LOOKUP_RATIO = 0.5;
-  private static final long DEFAULT_LOOKUP_MAX_TOTAL = Long.MAX_VALUE;
-
-  public static boolean shouldLookupExecuteOnShard(Configuration configuration, String table, int shard) {
-    double lookupRatio = getLookupRatio(configuration);
-    long maxLookupCount = getMaxLookupCount(configuration);
-    long rowIdFromNewDataCount = getBlurLookupRowIdFromNewDataCount(configuration, table, shard);
-    long rowIdUpdateFromNewDataCount = getBlurLookupRowIdUpdateFromNewDataCount(configuration, table, shard);
-    long rowIdFromIndexCount = getBlurLookupRowIdFromIndexCount(configuration, table, shard);
-    return shouldLookupRun(rowIdFromIndexCount, rowIdFromNewDataCount, rowIdUpdateFromNewDataCount, lookupRatio,
-        maxLookupCount);
-  }
-
-  private static boolean shouldLookupRun(long rowIdFromIndexCount, long rowIdFromNewDataCount,
-      long rowIdUpdateFromNewDataCount, double lookupRatio, long maxLookupCount) {
-    if (rowIdUpdateFromNewDataCount > maxLookupCount) {
-      return false;
-    }
-    double d = (double) rowIdUpdateFromNewDataCount / (double) rowIdFromIndexCount;
-    if (d <= lookupRatio) {
-      return true;
-    }
-    return false;
-  }
-
-  public static double getLookupRatio(Configuration configuration) {
-    return configuration.getDouble(BLUR_LOOKUP_RATIO_PER_SHARD, DEFAULT_LOOKUP_RATIO);
-  }
-
-  private static long getMaxLookupCount(Configuration configuration) {
-    return configuration.getLong(BLUR_LOOKUP_MAX_TOTAL_PER_SHARD, DEFAULT_LOOKUP_MAX_TOTAL);
-  }
-
-  public static void setTable(Job job, String table) {
-    setTable(job.getConfiguration(), table);
-  }
-
-  public static void setTable(Configuration configuration, String table) {
-    configuration.set(BLUR_LOOKUP_TABLE, table);
-  }
-
-  public static String getTable(Configuration configuration) {
-    return configuration.get(BLUR_LOOKUP_TABLE);
-  }
-
-  public static String getBlurLookupRowIdFromIndexCountName(String table) {
-    return BLUR_LOOKUP_ROWID_FROM_INDEX_COUNT_PREFIX + table;
-  }
-
-  public static String getBlurLookupRowIdFromNewDataCountName(String table) {
-    return BLUR_LOOKUP_ROWID_FROM_NEW_DATA_COUNT_PREFIX + table;
-  }
-
-  public static String getBlurLookupRowIdUpdateFromNewDataCountName(String table) {
-    return BLUR_LOOKUP_ROWID_UPDATE_FROM_NEW_DATA_COUNT_PREFIX + table;
-  }
-
-  public static long getBlurLookupRowIdUpdateFromNewDataCount(Configuration configuration, String table, int shard) {
-    String[] strings = configuration.getStrings(getBlurLookupRowIdUpdateFromNewDataCountName(table));
-    return getCount(strings, shard);
-  }
-
-  public static long getBlurLookupRowIdFromNewDataCount(Configuration configuration, String table, int shard) {
-    String[] strings = configuration.getStrings(getBlurLookupRowIdFromNewDataCountName(table));
-    return getCount(strings, shard);
-  }
-
-  public static long getBlurLookupRowIdFromIndexCount(Configuration configuration, String table, int shard) {
-    String[] strings = configuration.getStrings(getBlurLookupRowIdFromIndexCountName(table));
-    return getCount(strings, shard);
-  }
-
-  public static void setBlurLookupRowIdFromNewDataCounts(Job job, String table, long[] counts) {
-    setBlurLookupRowIdFromNewDataCounts(job.getConfiguration(), table, counts);
-  }
-
-  public static void setBlurLookupRowIdFromNewDataCounts(Configuration configuration, String table, long[] counts) {
-    configuration.setStrings(getBlurLookupRowIdFromNewDataCountName(table), toStrings(counts));
-  }
-
-  public static void setBlurLookupRowIdUpdateFromNewDataCounts(Job job, String table, long[] counts) {
-    setBlurLookupRowIdUpdateFromNewDataCounts(job.getConfiguration(), table, counts);
-  }
-
-  public static void setBlurLookupRowIdUpdateFromNewDataCounts(Configuration configuration, String table, long[] counts) {
-    configuration.setStrings(getBlurLookupRowIdUpdateFromNewDataCountName(table), toStrings(counts));
-  }
-
-  public static void setBlurLookupRowIdFromIndexCounts(Job job, String table, long[] counts) {
-    setBlurLookupRowIdFromIndexCounts(job.getConfiguration(), table, counts);
-  }
-
-  public static void setBlurLookupRowIdFromIndexCounts(Configuration configuration, String table, long[] counts) {
-    configuration.setStrings(getBlurLookupRowIdFromIndexCountName(table), toStrings(counts));
-  }
-
-  public static long getCount(String[] strings, int shard) {
-    return Long.parseLong(strings[shard]);
-  }
-
-  public static int getShardFromDirectoryPath(Path path) {
-    return ShardUtil.getShardIndex(path.getName());
-  }
-
-  public static String[] toStrings(long[] counts) {
-    if (counts == null) {
-      return null;
-    }
-    String[] strs = new String[counts.length];
-    for (int i = 0; i < counts.length; i++) {
-      strs[i] = Long.toString(counts[i]);
-    }
-    return strs;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderMapper.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderMapper.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderMapper.java
deleted file mode 100644
index 87a3a32..0000000
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderMapper.java
+++ /dev/null
@@ -1,34 +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.blur.mapreduce.lib.update;
-
-import java.io.IOException;
-
-import org.apache.blur.mapreduce.lib.BlurRecord;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Mapper;
-
-public class LookupBuilderMapper extends Mapper<Text, BlurRecord, Text, NullWritable> {
-
-  @Override
-  protected void map(Text key, BlurRecord value, Mapper<Text, BlurRecord, Text, NullWritable>.Context context)
-      throws IOException, InterruptedException {
-    context.write(new Text(value.getRowId()), NullWritable.get());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderReducer.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderReducer.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderReducer.java
deleted file mode 100644
index f3a2697..0000000
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderReducer.java
+++ /dev/null
@@ -1,181 +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.blur.mapreduce.lib.update;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.blur.BlurConfiguration;
-import org.apache.blur.manager.BlurPartitioner;
-import org.apache.blur.manager.writer.SnapshotIndexDeletionPolicy;
-import org.apache.blur.mapreduce.lib.BlurInputFormat;
-import org.apache.blur.mapreduce.lib.BlurOutputFormat;
-import org.apache.blur.mapreduce.lib.update.MergeSortRowIdMatcher.Action;
-import org.apache.blur.store.BlockCacheDirectoryFactoryV2;
-import org.apache.blur.store.hdfs.HdfsDirectory;
-import org.apache.blur.thrift.generated.TableDescriptor;
-import org.apache.blur.utils.BlurConstants;
-import org.apache.blur.utils.ShardUtil;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BooleanWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.SequenceFile.Writer;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.lucene.index.AtomicReader;
-import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexCommit;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.store.Directory;
-
-import com.google.common.io.Closer;
-
-public class LookupBuilderReducer extends Reducer<Text, NullWritable, Text, BooleanWritable> {
-
-  public static final String BLUR_CACHE_DIR_TOTAL_BYTES = "blur.cache.dir.total.bytes";
-  private Counter _rowIds;
-  private Counter _rowIdsToUpdate;
-
-  private MergeSortRowIdMatcher _matcher;
-  private int _numberOfShardsInTable;
-  private Configuration _configuration;
-  private String _snapshot;
-  private Path _tablePath;
-  private Counter _rowIdsFromIndex;
-  private long _totalNumberOfBytes;
-  private Action _action;
-  private Closer _closer;
-  private Path _cachePath;
-  private String _table;
-  private Writer _writer;
-
-  @Override
-  protected void setup(Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) throws IOException,
-      InterruptedException {
-    _configuration = context.getConfiguration();
-    _rowIds = context.getCounter(BlurIndexCounter.ROW_IDS_FROM_NEW_DATA);
-    _rowIdsToUpdate = context.getCounter(BlurIndexCounter.ROW_IDS_TO_UPDATE_FROM_NEW_DATA);
-    _rowIdsFromIndex = context.getCounter(BlurIndexCounter.ROW_IDS_FROM_INDEX);
-    TableDescriptor tableDescriptor = BlurOutputFormat.getTableDescriptor(_configuration);
-    _numberOfShardsInTable = tableDescriptor.getShardCount();
-    _tablePath = new Path(tableDescriptor.getTableUri());
-    _snapshot = MapperForExistingDataWithIndexLookup.getSnapshot(_configuration);
-    _totalNumberOfBytes = _configuration.getLong(BLUR_CACHE_DIR_TOTAL_BYTES, 128 * 1024 * 1024);
-    _cachePath = BlurInputFormat.getLocalCachePath(_configuration);
-    _table = tableDescriptor.getName();
-    _closer = Closer.create();
-  }
-
-  @Override
-  protected void reduce(Text rowId, Iterable<NullWritable> nothing,
-      Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) throws IOException, InterruptedException {
-    if (_matcher == null) {
-      _matcher = getMergeSortRowIdMatcher(rowId, context);
-    }
-    if (_writer == null) {
-      _writer = getRowIdWriter(rowId, context);
-    }
-    _writer.append(rowId, NullWritable.get());
-    _rowIds.increment(1);
-    if (_action == null) {
-      _action = new Action() {
-        @Override
-        public void found(Text rowId) throws IOException {
-          _rowIdsToUpdate.increment(1);
-          try {
-            context.write(rowId, new BooleanWritable(true));
-          } catch (InterruptedException e) {
-            throw new IOException(e);
-          }
-        }
-      };
-    }
-    _matcher.lookup(rowId, _action);
-  }
-
-  private Writer getRowIdWriter(Text rowId, Reducer<Text, NullWritable, Text, BooleanWritable>.Context context)
-      throws IOException {
-    BlurPartitioner blurPartitioner = new BlurPartitioner();
-    int shard = blurPartitioner.getShard(rowId, _numberOfShardsInTable);
-    String shardName = ShardUtil.getShardName(shard);
-    Path cachePath = MergeSortRowIdMatcher.getCachePath(_cachePath, _table, shardName);
-    Configuration configuration = context.getConfiguration();
-    String uuid = configuration.get(FasterDriver.BLUR_UPDATE_ID);
-    Path tmpPath = new Path(cachePath, uuid + "_" + getAttemptString(context));
-    return _closer.register(MergeSortRowIdMatcher.createWriter(_configuration, tmpPath));
-  }
-
-  private String getAttemptString(Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) {
-    TaskAttemptID taskAttemptID = context.getTaskAttemptID();
-    return taskAttemptID.toString();
-  }
-
-  @Override
-  protected void cleanup(Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) throws IOException,
-      InterruptedException {
-    _closer.close();
-  }
-
-  private MergeSortRowIdMatcher getMergeSortRowIdMatcher(Text rowId,
-      Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) throws IOException {
-    BlurPartitioner blurPartitioner = new BlurPartitioner();
-    int shard = blurPartitioner.getShard(rowId, _numberOfShardsInTable);
-    String shardName = ShardUtil.getShardName(shard);
-
-    Path shardPath = new Path(_tablePath, shardName);
-    HdfsDirectory hdfsDirectory = new HdfsDirectory(_configuration, shardPath);
-    SnapshotIndexDeletionPolicy policy = new SnapshotIndexDeletionPolicy(_configuration,
-        SnapshotIndexDeletionPolicy.getGenerationsPath(shardPath));
-    Long generation = policy.getGeneration(_snapshot);
-    if (generation == null) {
-      hdfsDirectory.close();
-      throw new IOException("Snapshot [" + _snapshot + "] not found in shard [" + shardPath + "]");
-    }
-
-    BlurConfiguration bc = new BlurConfiguration();
-    BlockCacheDirectoryFactoryV2 blockCacheDirectoryFactoryV2 = new BlockCacheDirectoryFactoryV2(bc,
-        _totalNumberOfBytes);
-    _closer.register(blockCacheDirectoryFactoryV2);
-    Directory dir = blockCacheDirectoryFactoryV2.newDirectory("table", "shard", hdfsDirectory, null);
-    List<IndexCommit> listCommits = DirectoryReader.listCommits(dir);
-    IndexCommit indexCommit = MapperForExistingDataWithIndexLookup.findIndexCommit(listCommits, generation, shardPath);
-    DirectoryReader reader = DirectoryReader.open(indexCommit);
-    _rowIdsFromIndex.setValue(getTotalNumberOfRowIds(reader));
-
-    Path cachePath = MergeSortRowIdMatcher.getCachePath(_cachePath, _table, shardName);
-    return new MergeSortRowIdMatcher(dir, generation, _configuration, cachePath, context);
-  }
-
-  private long getTotalNumberOfRowIds(DirectoryReader reader) throws IOException {
-    long total = 0;
-    List<AtomicReaderContext> leaves = reader.leaves();
-    for (AtomicReaderContext context : leaves) {
-      AtomicReader atomicReader = context.reader();
-      Terms terms = atomicReader.terms(BlurConstants.ROW_ID);
-      long expectedInsertions = terms.size();
-      if (expectedInsertions < 0) {
-        return -1;
-      }
-      total += expectedInsertions;
-    }
-    return total;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataMod.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataMod.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataMod.java
deleted file mode 100644
index bf86e19..0000000
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataMod.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.blur.mapreduce.lib.update;
-
-import java.io.IOException;
-
-import org.apache.blur.mapreduce.lib.BlurRecord;
-import org.apache.blur.mapreduce.lib.TableBlurRecord;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.Mapper;
-
-public class MapperForExistingDataMod extends Mapper<Text, TableBlurRecord, IndexKey, IndexValue> {
-
-  private Counter _existingRecords;
-
-  @Override
-  protected void setup(Context context) throws IOException, InterruptedException {
-    Counter counter = context.getCounter(BlurIndexCounter.INPUT_FORMAT_MAPPER);
-    counter.increment(1);
-    _existingRecords = context.getCounter(BlurIndexCounter.INPUT_FORMAT_EXISTING_RECORDS);
-  }
-
-  @Override
-  protected void map(Text key, TableBlurRecord value, Context context) throws IOException, InterruptedException {
-    BlurRecord blurRecord = value.getBlurRecord();
-    IndexKey oldDataKey = IndexKey.oldData(blurRecord.getRowId(), blurRecord.getRecordId());
-    context.write(oldDataKey, new IndexValue(blurRecord));
-    _existingRecords.increment(1L);
-  }
-
-}


[08/13] git commit: Adding blur indexer project.

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataWithIndexLookup.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataWithIndexLookup.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataWithIndexLookup.java
deleted file mode 100644
index 0e2fe66..0000000
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataWithIndexLookup.java
+++ /dev/null
@@ -1,228 +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.blur.mapreduce.lib.update;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.blur.BlurConfiguration;
-import org.apache.blur.manager.BlurPartitioner;
-import org.apache.blur.manager.writer.SnapshotIndexDeletionPolicy;
-import org.apache.blur.mapreduce.lib.BlurOutputFormat;
-import org.apache.blur.mapreduce.lib.BlurRecord;
-import org.apache.blur.store.BlockCacheDirectoryFactoryV2;
-import org.apache.blur.store.hdfs.HdfsDirectory;
-import org.apache.blur.thrift.generated.Column;
-import org.apache.blur.thrift.generated.FetchRecordResult;
-import org.apache.blur.thrift.generated.Record;
-import org.apache.blur.thrift.generated.TableDescriptor;
-import org.apache.blur.utils.BlurConstants;
-import org.apache.blur.utils.RowDocumentUtil;
-import org.apache.blur.utils.ShardUtil;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BooleanWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.index.AtomicReader;
-import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexCommit;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.Collector;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.store.Directory;
-
-import com.google.common.io.Closer;
-
-public class MapperForExistingDataWithIndexLookup extends Mapper<Text, BooleanWritable, IndexKey, IndexValue> {
-
-  private static final Log LOG = LogFactory.getLog(MapperForExistingDataWithIndexLookup.class);
-
-  private static final String BLUR_SNAPSHOT = "blur.snapshot";
-  private Counter _existingRecords;
-  private Counter _rowLookup;
-  private BlurPartitioner _blurPartitioner;
-  private Path _tablePath;
-  private int _numberOfShardsInTable;
-  private Configuration _configuration;
-  private String _snapshot;
-
-  private int _indexShard = -1;
-  private DirectoryReader _reader;
-  private IndexSearcher _indexSearcher;
-  private long _totalNumberOfBytes;
-  private Closer _closer;
-
-  @Override
-  protected void setup(Context context) throws IOException, InterruptedException {
-    Counter counter = context.getCounter(BlurIndexCounter.LOOKUP_MAPPER);
-    counter.increment(1);
-
-    _configuration = context.getConfiguration();
-    _existingRecords = context.getCounter(BlurIndexCounter.LOOKUP_MAPPER_EXISTING_RECORDS);
-    _rowLookup = context.getCounter(BlurIndexCounter.LOOKUP_MAPPER_ROW_LOOKUP_ATTEMPT);
-    _blurPartitioner = new BlurPartitioner();
-    TableDescriptor tableDescriptor = BlurOutputFormat.getTableDescriptor(_configuration);
-    _numberOfShardsInTable = tableDescriptor.getShardCount();
-    _tablePath = new Path(tableDescriptor.getTableUri());
-    _snapshot = getSnapshot(_configuration);
-    _totalNumberOfBytes = _configuration.getLong(LookupBuilderReducer.BLUR_CACHE_DIR_TOTAL_BYTES, 128 * 1024 * 1024);
-    _closer = Closer.create();
-  }
-
-  @Override
-  protected void map(Text key, BooleanWritable value, Context context) throws IOException, InterruptedException {
-    if (value.get()) {
-      String rowId = key.toString();
-      LOG.debug("Looking up rowid [" + rowId + "]");
-      _rowLookup.increment(1);
-      IndexSearcher indexSearcher = getIndexSearcher(rowId);
-      Term term = new Term(BlurConstants.ROW_ID, rowId);
-      RowCollector collector = getCollector(context);
-      indexSearcher.search(new TermQuery(term), collector);
-      LOG.debug("Looking for rowid [" + rowId + "] has [" + collector.records + "] records");
-    }
-  }
-
-  @Override
-  protected void cleanup(Mapper<Text, BooleanWritable, IndexKey, IndexValue>.Context context) throws IOException,
-      InterruptedException {
-    _closer.close();
-  }
-
-  static class RowCollector extends Collector {
-
-    private AtomicReader reader;
-    private Mapper<Text, BooleanWritable, IndexKey, IndexValue>.Context _context;
-    private Counter _existingRecords;
-    int records;
-
-    RowCollector(Mapper<Text, BooleanWritable, IndexKey, IndexValue>.Context context, Counter existingRecords) {
-      _context = context;
-      _existingRecords = existingRecords;
-    }
-
-    @Override
-    public void setScorer(Scorer scorer) throws IOException {
-
-    }
-
-    @Override
-    public void setNextReader(AtomicReaderContext context) throws IOException {
-      reader = context.reader();
-    }
-
-    @Override
-    public void collect(int doc) throws IOException {
-      Document document = reader.document(doc);
-      FetchRecordResult result = RowDocumentUtil.getRecord(document);
-      String rowid = result.getRowid();
-      Record record = result.getRecord();
-      String recordId = record.getRecordId();
-      IndexKey oldDataKey = IndexKey.oldData(rowid, recordId);
-      try {
-        _context.write(oldDataKey, new IndexValue(toBlurRecord(rowid, record)));
-      } catch (InterruptedException e) {
-        throw new IOException(e);
-      }
-      _existingRecords.increment(1L);
-    }
-
-    private BlurRecord toBlurRecord(String rowId, Record record) {
-      BlurRecord blurRecord = new BlurRecord();
-      blurRecord.setRowId(rowId);
-      blurRecord.setRecordId(record.getRecordId());
-      blurRecord.setFamily(record.getFamily());
-      List<Column> columns = record.getColumns();
-      for (Column column : columns) {
-        blurRecord.addColumn(column.getName(), column.getValue());
-      }
-      return blurRecord;
-    }
-
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-      return false;
-    }
-  }
-
-  private RowCollector getCollector(Mapper<Text, BooleanWritable, IndexKey, IndexValue>.Context context) {
-    return new RowCollector(context, _existingRecords);
-  }
-
-  private IndexSearcher getIndexSearcher(String rowId) throws IOException {
-    int shard = _blurPartitioner.getShard(rowId, _numberOfShardsInTable);
-    if (_indexSearcher != null) {
-      if (shard != _indexShard) {
-        throw new IOException("Input data is not partitioned correctly.");
-      }
-      return _indexSearcher;
-    } else {
-      _indexShard = shard;
-      Path shardPath = new Path(_tablePath, ShardUtil.getShardName(_indexShard));
-      HdfsDirectory hdfsDirectory = new HdfsDirectory(_configuration, shardPath);
-      SnapshotIndexDeletionPolicy policy = new SnapshotIndexDeletionPolicy(_configuration,
-          SnapshotIndexDeletionPolicy.getGenerationsPath(shardPath));
-      Long generation = policy.getGeneration(_snapshot);
-      if (generation == null) {
-        hdfsDirectory.close();
-        throw new IOException("Snapshot [" + _snapshot + "] not found in shard [" + shardPath + "]");
-      }
-
-      BlurConfiguration bc = new BlurConfiguration();
-      BlockCacheDirectoryFactoryV2 blockCacheDirectoryFactoryV2 = new BlockCacheDirectoryFactoryV2(bc,
-          _totalNumberOfBytes);
-      _closer.register(blockCacheDirectoryFactoryV2);
-      Directory dir = blockCacheDirectoryFactoryV2.newDirectory("table", "shard", hdfsDirectory, null);
-
-      List<IndexCommit> listCommits = DirectoryReader.listCommits(dir);
-      IndexCommit indexCommit = findIndexCommit(listCommits, generation, shardPath);
-      _reader = DirectoryReader.open(indexCommit);
-      return _indexSearcher = new IndexSearcher(_reader);
-    }
-  }
-
-  public static IndexCommit findIndexCommit(List<IndexCommit> listCommits, long generation, Path shardDir)
-      throws IOException {
-    for (IndexCommit commit : listCommits) {
-      if (commit.getGeneration() == generation) {
-        return commit;
-      }
-    }
-    throw new IOException("Generation [" + generation + "] not found in shard [" + shardDir + "]");
-  }
-
-  public static void setSnapshot(Job job, String snapshot) {
-    setSnapshot(job.getConfiguration(), snapshot);
-  }
-
-  public static void setSnapshot(Configuration configuration, String snapshot) {
-    configuration.set(BLUR_SNAPSHOT, snapshot);
-  }
-
-  public static String getSnapshot(Configuration configuration) {
-    return configuration.get(BLUR_SNAPSHOT);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForNewDataMod.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForNewDataMod.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForNewDataMod.java
deleted file mode 100644
index d91d1f5..0000000
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForNewDataMod.java
+++ /dev/null
@@ -1,82 +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.blur.mapreduce.lib.update;
-
-import java.io.IOException;
-import java.lang.reflect.Field;
-
-import org.apache.blur.mapreduce.lib.BlurRecord;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-
-public class MapperForNewDataMod extends Mapper<Text, BlurRecord, IndexKey, IndexValue> {
-
-  private static final IndexValue EMPTY_RECORD = new IndexValue();
-  private long _timestamp;
-  private Counter _newRecords;
-
-  @Override
-  protected void setup(Context context) throws IOException, InterruptedException {
-    InputSplit inputSplit = context.getInputSplit();
-    FileSplit fileSplit = getFileSplit(inputSplit);
-    Path path = fileSplit.getPath();
-    FileSystem fileSystem = path.getFileSystem(context.getConfiguration());
-    FileStatus fileStatus = fileSystem.getFileStatus(path);
-    _timestamp = fileStatus.getModificationTime();
-    _newRecords = context.getCounter(BlurIndexCounter.NEW_RECORDS);
-  }
-
-  private FileSplit getFileSplit(InputSplit inputSplit) throws IOException {
-    if (inputSplit instanceof FileSplit) {
-      return (FileSplit) inputSplit;
-    }
-    if (inputSplit.getClass().getName().equals("org.apache.hadoop.mapreduce.lib.input.TaggedInputSplit")) {
-      try {
-        Field declaredField = inputSplit.getClass().getDeclaredField("inputSplit");
-        declaredField.setAccessible(true);
-        return getFileSplit((InputSplit) declaredField.get(inputSplit));
-      } catch (NoSuchFieldException e) {
-        throw new IOException(e);
-      } catch (SecurityException e) {
-        throw new IOException(e);
-      } catch (IllegalArgumentException e) {
-        throw new IOException(e);
-      } catch (IllegalAccessException e) {
-        throw new IOException(e);
-      }
-    } else {
-      throw new IOException("Unknown input split type [" + inputSplit + "] [" + inputSplit.getClass() + "]");
-    }
-  }
-
-  @Override
-  protected void map(Text key, BlurRecord blurRecord, Context context) throws IOException, InterruptedException {
-    IndexKey newDataKey = IndexKey.newData(blurRecord.getRowId(), blurRecord.getRecordId(), _timestamp);
-    context.write(newDataKey, new IndexValue(blurRecord));
-    _newRecords.increment(1L);
-
-    IndexKey newDataMarker = IndexKey.newDataMarker(blurRecord.getRowId());
-    context.write(newDataMarker, EMPTY_RECORD);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MergeSortRowIdMatcher.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MergeSortRowIdMatcher.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MergeSortRowIdMatcher.java
deleted file mode 100644
index bd8580e..0000000
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MergeSortRowIdMatcher.java
+++ /dev/null
@@ -1,388 +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.blur.mapreduce.lib.update;
-
-import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.blur.index.AtomicReaderUtil;
-import org.apache.blur.log.Log;
-import org.apache.blur.log.LogFactory;
-import org.apache.blur.store.hdfs.DirectoryDecorator;
-import org.apache.blur.store.hdfs.HdfsDirectory;
-import org.apache.blur.utils.BlurConstants;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.HdfsBlockLocation;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.io.SequenceFile.Reader;
-import org.apache.hadoop.io.SequenceFile.Writer;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.DeflateCodec;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.hadoop.io.compress.zlib.ZlibFactory;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Progressable;
-import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexCommit;
-import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.SegmentInfoPerCommit;
-import org.apache.lucene.index.SegmentInfos;
-import org.apache.lucene.index.SegmentReader;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
-
-public class MergeSortRowIdMatcher {
-
-  private static final String DEL = ".del";
-  private static final Log LOG = LogFactory.getLog(MergeSortRowIdMatcher.class);
-  private static final Progressable NO_OP = new Progressable() {
-    @Override
-    public void progress() {
-
-    }
-  };
-  private static final long _10_SECONDS = TimeUnit.SECONDS.toNanos(10);
-
-  public interface Action {
-    void found(Text rowId) throws IOException;
-  }
-
-  private final MyReader[] _readers;
-  private final Configuration _configuration;
-  private final Path _cachePath;
-  private final IndexCommit _indexCommit;
-  private final Directory _directory;
-  private final Progressable _progressable;
-
-  private DirectoryReader _reader;
-
-  public MergeSortRowIdMatcher(Directory directory, long generation, Configuration configuration, Path cachePath)
-      throws IOException {
-    this(directory, generation, configuration, cachePath, null);
-  }
-
-  public MergeSortRowIdMatcher(Directory directory, long generation, Configuration configuration, Path cachePath,
-      Progressable progressable) throws IOException {
-    List<IndexCommit> listCommits = DirectoryReader.listCommits(directory);
-    _indexCommit = findIndexCommit(listCommits, generation);
-    _configuration = configuration;
-    _cachePath = cachePath;
-    _directory = directory;
-    _progressable = progressable == null ? NO_OP : progressable;
-    _readers = openReaders();
-  }
-
-  public void lookup(Text rowId, Action action) throws IOException {
-    if (lookup(rowId)) {
-      action.found(rowId);
-    }
-  }
-
-  private boolean lookup(Text rowId) throws IOException {
-    advanceReadersIfNeeded(rowId);
-    sortReaders();
-    return checkReaders(rowId);
-  }
-
-  private boolean checkReaders(Text rowId) {
-    for (MyReader reader : _readers) {
-      int compareTo = reader.getCurrentRowId().compareTo(rowId);
-      if (compareTo == 0) {
-        return true;
-      } else if (compareTo > 0) {
-        return false;
-      }
-    }
-    return false;
-  }
-
-  private void advanceReadersIfNeeded(Text rowId) throws IOException {
-    _progressable.progress();
-    for (MyReader reader : _readers) {
-      if (rowId.compareTo(reader.getCurrentRowId()) > 0) {
-        advanceReader(reader, rowId);
-      }
-    }
-  }
-
-  private void advanceReader(MyReader reader, Text rowId) throws IOException {
-    while (reader.next()) {
-      if (rowId.compareTo(reader.getCurrentRowId()) <= 0) {
-        return;
-      }
-    }
-  }
-
-  private static final Comparator<MyReader> COMP = new Comparator<MyReader>() {
-    @Override
-    public int compare(MyReader o1, MyReader o2) {
-      return o1.getCurrentRowId().compareTo(o2.getCurrentRowId());
-    }
-  };
-
-  private void sortReaders() {
-    Arrays.sort(_readers, COMP);
-  }
-
-  private MyReader[] openReaders() throws IOException {
-    Collection<SegmentKey> segmentKeys = getSegmentKeys();
-    MyReader[] readers = new MyReader[segmentKeys.size()];
-    int i = 0;
-    for (SegmentKey segmentKey : segmentKeys) {
-      readers[i++] = openReader(segmentKey);
-    }
-    return readers;
-  }
-
-  private MyReader openReader(SegmentKey segmentKey) throws IOException {
-    Path file = getCacheFilePath(segmentKey);
-    FileSystem fileSystem = _cachePath.getFileSystem(_configuration);
-    if (!fileSystem.exists(file)) {
-      createCacheFile(file, segmentKey);
-    }
-    Reader reader = new SequenceFile.Reader(_configuration, SequenceFile.Reader.file(file));
-    return new MyReader(reader);
-  }
-
-  private void createCacheFile(Path file, SegmentKey segmentKey) throws IOException {
-    LOG.info("Building cache for segment [{0}] to [{1}]", segmentKey, file);
-    Path tmpPath = getTmpWriterPath(file.getParent());
-    try (Writer writer = createWriter(_configuration, tmpPath)) {
-      DirectoryReader reader = getReader();
-      for (AtomicReaderContext context : reader.leaves()) {
-        SegmentReader segmentReader = AtomicReaderUtil.getSegmentReader(context.reader());
-        if (segmentReader.getSegmentName().equals(segmentKey.getSegmentName())) {
-          writeRowIds(writer, segmentReader);
-          break;
-        }
-      }
-    }
-    commitWriter(_configuration, file, tmpPath);
-  }
-
-  public static void commitWriter(Configuration configuration, Path file, Path tmpPath) throws IOException {
-    FileSystem fileSystem = tmpPath.getFileSystem(configuration);
-    LOG.info("Commit tmp [{0}] to file [{1}]", tmpPath, file);
-    if (!fileSystem.rename(tmpPath, file)) {
-      LOG.warn("Could not commit tmp file [{0}] to file [{1}]", tmpPath, file);
-    }
-  }
-
-  public static Path getTmpWriterPath(Path dir) {
-    return new Path(dir, UUID.randomUUID().toString() + ".tmp");
-  }
-
-  public static Writer createWriter(Configuration configuration, Path tmpPath) throws IOException {
-    return SequenceFile.createWriter(configuration, SequenceFile.Writer.file(tmpPath),
-        SequenceFile.Writer.keyClass(Text.class), SequenceFile.Writer.valueClass(NullWritable.class),
-        SequenceFile.Writer.compression(CompressionType.BLOCK, getCodec(configuration)));
-  }
-
-  private static CompressionCodec getCodec(Configuration configuration) {
-    if (ZlibFactory.isNativeZlibLoaded(configuration)) {
-      return new GzipCodec();
-    }
-    return new DeflateCodec();
-  }
-
-  private void writeRowIds(Writer writer, SegmentReader segmentReader) throws IOException {
-    Terms terms = segmentReader.terms(BlurConstants.ROW_ID);
-    if (terms == null) {
-      return;
-    }
-    TermsEnum termsEnum = terms.iterator(null);
-    BytesRef rowId;
-    long s = System.nanoTime();
-    while ((rowId = termsEnum.next()) != null) {
-      long n = System.nanoTime();
-      if (n + _10_SECONDS > s) {
-        _progressable.progress();
-        s = System.nanoTime();
-      }
-      writer.append(new Text(rowId.utf8ToString()), NullWritable.get());
-    }
-  }
-
-  private IndexCommit findIndexCommit(List<IndexCommit> listCommits, long generation) throws IOException {
-    for (IndexCommit commit : listCommits) {
-      if (commit.getGeneration() == generation) {
-        return commit;
-      }
-    }
-    throw new IOException("Generation [" + generation + "] not found.");
-  }
-
-  static class SegmentKey {
-
-    final String _segmentName;
-    final String _id;
-
-    SegmentKey(String segmentName, String id) throws IOException {
-      _segmentName = segmentName;
-      _id = id;
-    }
-
-    String getSegmentName() {
-      return _segmentName;
-    }
-
-    @Override
-    public String toString() {
-      return _id;
-    }
-  }
-
-  private DirectoryReader getReader() throws IOException {
-    if (_reader == null) {
-      _reader = DirectoryReader.open(_indexCommit);
-    }
-    return _reader;
-  }
-
-  private Collection<SegmentKey> getSegmentKeys() throws IOException {
-    List<SegmentKey> keys = new ArrayList<SegmentKey>();
-    SegmentInfos segmentInfos = new SegmentInfos();
-    segmentInfos.read(_directory, _indexCommit.getSegmentsFileName());
-    for (SegmentInfoPerCommit segmentInfoPerCommit : segmentInfos) {
-      String name = segmentInfoPerCommit.info.name;
-      String id = getId(segmentInfoPerCommit.info);
-      keys.add(new SegmentKey(name, id));
-    }
-    return keys;
-  }
-
-  private String getId(SegmentInfo si) throws IOException {
-    HdfsDirectory dir = getHdfsDirectory(si.dir);
-    Set<String> files = new TreeSet<String>(si.files());
-    return getId(_configuration, dir, files);
-  }
-
-  private static String getId(Configuration configuration, HdfsDirectory dir, Set<String> files) throws IOException {
-    long ts = 0;
-    String file = null;
-    for (String f : files) {
-      if (f.endsWith(DEL)) {
-        continue;
-      }
-      long fileModified = dir.getFileModified(f);
-      if (fileModified > ts) {
-        ts = fileModified;
-        file = f;
-      }
-    }
-
-    Path path = dir.getPath();
-    FileSystem fileSystem = path.getFileSystem(configuration);
-    Path realFile = new Path(path, file);
-    if (!fileSystem.exists(realFile)) {
-      realFile = dir.getRealFilePathFromSymlink(file);
-      if (!fileSystem.exists(realFile)) {
-        throw new IOException("Lucene file [" + file + "] for dir [" + path + "] can not be found.");
-      }
-    }
-    return getFirstBlockId(fileSystem, realFile);
-  }
-
-  public static String getIdForSingleSegmentIndex(Configuration configuration, Path indexPath) throws IOException {
-    HdfsDirectory dir = new HdfsDirectory(configuration, indexPath);
-    Set<String> files = new TreeSet<String>(Arrays.asList(dir.listAll()));
-    return getId(configuration, dir, files);
-  }
-
-  private static String getFirstBlockId(FileSystem fileSystem, Path realFile) throws IOException {
-    FileStatus fileStatus = fileSystem.getFileStatus(realFile);
-    BlockLocation[] locations = fileSystem.getFileBlockLocations(fileStatus, 0, 1);
-    HdfsBlockLocation location = (HdfsBlockLocation) locations[0];
-    LocatedBlock locatedBlock = location.getLocatedBlock();
-    ExtendedBlock block = locatedBlock.getBlock();
-    return toNiceString(block.getBlockId());
-  }
-
-  private static String toNiceString(long blockId) {
-    return "b" + blockId;
-  }
-
-  private static HdfsDirectory getHdfsDirectory(Directory dir) {
-    if (dir instanceof HdfsDirectory) {
-      return (HdfsDirectory) dir;
-    } else if (dir instanceof DirectoryDecorator) {
-      DirectoryDecorator dd = (DirectoryDecorator) dir;
-      return getHdfsDirectory(dd.getOriginalDirectory());
-    } else {
-      throw new RuntimeException("Unknown directory type.");
-    }
-  }
-
-  private Path getCacheFilePath(SegmentKey segmentKey) {
-    return new Path(_cachePath, segmentKey + ".seq");
-  }
-
-  static class MyReader {
-
-    final Reader _reader;
-    final Text _rowId = new Text();
-    boolean _finished = false;
-
-    public MyReader(Reader reader) {
-      _reader = reader;
-    }
-
-    public Text getCurrentRowId() {
-      return _rowId;
-    }
-
-    public boolean next() throws IOException {
-      if (_finished) {
-        return false;
-      }
-      if (_reader.next(_rowId)) {
-        return true;
-      }
-      _finished = true;
-      return false;
-    }
-
-    public boolean isFinished() {
-      return _finished;
-    }
-  }
-
-  public static Path getCachePath(Path cachePath, String table, String shardName) {
-    return new Path(new Path(cachePath, table), shardName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedBlurInputFormat.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedBlurInputFormat.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedBlurInputFormat.java
deleted file mode 100644
index 8738c5a..0000000
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedBlurInputFormat.java
+++ /dev/null
@@ -1,73 +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.blur.mapreduce.lib.update;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.blur.mapreduce.lib.BlurInputFormat;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-
-public class PrunedBlurInputFormat extends BlurInputFormat {
-
-  private static final Log LOG = LogFactory.getLog(PrunedBlurInputFormat.class);
-
-  @Override
-  public List<InputSplit> getSplits(JobContext context) throws IOException {
-    Path[] dirs = getInputPaths(context);
-    Configuration configuration = context.getConfiguration();
-    List<BlurInputSplit> splits = getSplits(configuration, dirs);
-    Map<Path, List<BlurInputSplit>> splitMap = new TreeMap<Path, List<BlurInputSplit>>();
-    for (BlurInputSplit split : splits) {
-      Path path = split.getDir();
-      String table = split.getTable().toString();
-      int shard = InputSplitPruneUtil.getShardFromDirectoryPath(path);
-      long rowIdUpdateFromNewDataCount = InputSplitPruneUtil.getBlurLookupRowIdUpdateFromNewDataCount(configuration,
-          table, shard);
-      long indexCount = InputSplitPruneUtil.getBlurLookupRowIdFromIndexCount(configuration, table, shard);
-      if (rowIdUpdateFromNewDataCount == 0 || indexCount == 0) {
-        LOG.info("Pruning id lookup input path [" + path + "] no overlapping ids.");
-      } else if (InputSplitPruneUtil.shouldLookupExecuteOnShard(configuration, table, shard)) {
-        LOG.info("Pruning blur input path [" + split.getDir() + "]");
-      } else {
-        LOG.debug("Keeping blur input path [" + split.getDir() + "]");
-        List<BlurInputSplit> list = splitMap.get(path);
-        if (list == null) {
-          splitMap.put(path, list = new ArrayList<BlurInputSplit>());
-        }
-        list.add(split);
-      }
-    }
-    List<InputSplit> result = new ArrayList<InputSplit>();
-    for (List<BlurInputSplit> lst : splitMap.values()) {
-      BlurInputSplitColletion blurInputSplitColletion = new BlurInputSplitColletion();
-      for (BlurInputSplit blurInputSplit : lst) {
-        blurInputSplitColletion.add(blurInputSplit);
-      }
-      result.add(blurInputSplitColletion);
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedSequenceFileInputFormat.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedSequenceFileInputFormat.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedSequenceFileInputFormat.java
deleted file mode 100644
index 58e9800..0000000
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedSequenceFileInputFormat.java
+++ /dev/null
@@ -1,75 +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.blur.mapreduce.lib.update;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-
-import com.google.common.base.Splitter;
-
-public class PrunedSequenceFileInputFormat<K, V> extends SequenceFileInputFormat<K, V> {
-
-  private static final Log LOG = LogFactory.getLog(PrunedSequenceFileInputFormat.class);
-
-  @Override
-  public List<InputSplit> getSplits(JobContext job) throws IOException {
-    List<InputSplit> splits = super.getSplits(job);
-    List<InputSplit> results = new ArrayList<InputSplit>();
-    Configuration configuration = job.getConfiguration();
-    String table = InputSplitPruneUtil.getTable(configuration);
-    for (InputSplit inputSplit : splits) {
-      FileSplit fileSplit = (FileSplit) inputSplit;
-      Path path = fileSplit.getPath();
-      LOG.debug("Getting shard index from path [" + path + "]");
-      String name = path.getName();
-      int shard = getShardIndex(name);
-      long rowIdUpdateFromNewDataCount = InputSplitPruneUtil.getBlurLookupRowIdUpdateFromNewDataCount(configuration,
-          table, shard);
-      long indexCount = InputSplitPruneUtil.getBlurLookupRowIdFromIndexCount(configuration, table, shard);
-      if (rowIdUpdateFromNewDataCount == 0 || indexCount == 0) {
-        LOG.info("Pruning id lookup input path [" + path + "] no overlapping ids.");
-      } else if (InputSplitPruneUtil.shouldLookupExecuteOnShard(configuration, table, shard)) {
-        LOG.debug("Keeping id lookup input path [" + path + "]");
-        results.add(inputSplit);
-      } else {
-        LOG.info("Pruning id lookup input path [" + path + "]");
-      }
-    }
-    return results;
-  }
-
-  private int getShardIndex(String name) {
-    // based on file format of "part-r-00000", etc
-    Iterable<String> split = Splitter.on('-').split(name);
-    List<String> parts = new ArrayList<String>();
-    for (String s : split) {
-      parts.add(s);
-    }
-    return Integer.parseInt(parts.get(2));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/distribution-bin/pom.xml
----------------------------------------------------------------------
diff --git a/distribution-bin/pom.xml b/distribution-bin/pom.xml
index 4eeedc9..6bdc405 100644
--- a/distribution-bin/pom.xml
+++ b/distribution-bin/pom.xml
@@ -40,6 +40,11 @@ under the License.
 		</dependency>
 		<dependency>
 			<groupId>org.apache.blur</groupId>
+			<artifactId>blur-indexer</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.blur</groupId>
 			<artifactId>blur-shell</artifactId>
 			<version>${project.version}</version>
 		</dependency>


[12/13] git commit: Removing thread local variables.

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-store/src/main/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
----------------------------------------------------------------------
diff --git a/blur-store/src/main/java/org/apache/lucene/codecs/BlockTreeTermsReader.java b/blur-store/src/main/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
new file mode 100644
index 0000000..4ef65ff
--- /dev/null
+++ b/blur-store/src/main/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
@@ -0,0 +1,2939 @@
+package org.apache.lucene.codecs;
+
+/*
+ * 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.
+ */
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.UnsupportedEncodingException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Locale;
+import java.util.TreeMap;
+
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.RunAutomaton;
+import org.apache.lucene.util.automaton.Transition;
+import org.apache.lucene.util.fst.ByteSequenceOutputs;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.Outputs;
+import org.apache.lucene.util.fst.Util;
+
+/** A block-based terms index and dictionary that assigns
+ *  terms to variable length blocks according to how they
+ *  share prefixes.  The terms index is a prefix trie
+ *  whose leaves are term blocks.  The advantage of this
+ *  approach is that seekExact is often able to
+ *  determine a term cannot exist without doing any IO, and
+ *  intersection with Automata is very fast.  Note that this
+ *  terms dictionary has it's own fixed terms index (ie, it
+ *  does not support a pluggable terms index
+ *  implementation).
+ *
+ *  <p><b>NOTE</b>: this terms dictionary does not support
+ *  index divisor when opening an IndexReader.  Instead, you
+ *  can change the min/maxItemsPerBlock during indexing.</p>
+ *
+ *  <p>The data structure used by this implementation is very
+ *  similar to a burst trie
+ *  (http://citeseer.ist.psu.edu/viewdoc/summary?doi=10.1.1.18.3499),
+ *  but with added logic to break up too-large blocks of all
+ *  terms sharing a given prefix into smaller ones.</p>
+ *
+ *  <p>Use {@link org.apache.lucene.index.CheckIndex} with the <code>-verbose</code>
+ *  option to see summary statistics on the blocks in the
+ *  dictionary.
+ *
+ *  See {@link BlockTreeTermsWriter}.
+ *
+ * @lucene.experimental
+ */
+
+public class BlockTreeTermsReader extends FieldsProducer {
+
+  // Open input to the main terms dict file (_X.tib)
+  private final IndexInput in;
+
+  //private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+
+  // Reads the terms dict entries, to gather state to
+  // produce DocsEnum on demand
+  private final PostingsReaderBase postingsReader;
+
+  private final TreeMap<String,FieldReader> fields = new TreeMap<String,FieldReader>();
+
+  /** File offset where the directory starts in the terms file. */
+  private long dirOffset;
+
+  /** File offset where the directory starts in the index file. */
+  private long indexDirOffset;
+
+  private String segment;
+  
+  private final int version;
+
+  private final IndexInput indexIn;
+
+  /** Sole constructor. */
+  public BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info,
+                              PostingsReaderBase postingsReader, IOContext ioContext,
+                              String segmentSuffix, int indexDivisor)
+    throws IOException {
+    
+    this.postingsReader = postingsReader;
+
+    this.segment = info.name;
+    in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_EXTENSION),
+                       ioContext);
+
+    boolean success = false;
+
+    try {
+      version = readHeader(in);
+      if (indexDivisor != -1) {
+        indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
+                                ioContext);
+        int indexVersion = readIndexHeader(indexIn);
+        if (indexVersion != version) {
+          throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion);
+        }
+      } else {
+        indexIn = null;
+      }
+
+      // Have PostingsReader init itself
+      postingsReader.init(in);
+
+      // Read per-field details
+      seekDir(in, dirOffset);
+      if (indexDivisor != -1) {
+        seekDir(indexIn, indexDirOffset);
+      }
+
+      final int numFields = in.readVInt();
+      if (numFields < 0) {
+        throw new CorruptIndexException("invalid numFields: " + numFields + " (resource=" + in + ")");
+      }
+
+      for(int i=0;i<numFields;i++) {
+        final int field = in.readVInt();
+        final long numTerms = in.readVLong();
+        assert numTerms >= 0;
+        final int numBytes = in.readVInt();
+        final BytesRef rootCode = new BytesRef(new byte[numBytes]);
+        in.readBytes(rootCode.bytes, 0, numBytes);
+        rootCode.length = numBytes;
+        final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+        assert fieldInfo != null: "field=" + field;
+        final long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : in.readVLong();
+        final long sumDocFreq = in.readVLong();
+        final int docCount = in.readVInt();
+        if (docCount < 0 || docCount > info.getDocCount()) { // #docs with field must be <= #docs
+          throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + info.getDocCount() + " (resource=" + in + ")");
+        }
+        if (sumDocFreq < docCount) {  // #postings must be >= #docs with field
+          throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount + " (resource=" + in + ")");
+        }
+        if (sumTotalTermFreq != -1 && sumTotalTermFreq < sumDocFreq) { // #positions must be >= #postings
+          throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq + " (resource=" + in + ")");
+        }
+        final long indexStartFP = indexDivisor != -1 ? indexIn.readVLong() : 0;
+        FieldReader previous = fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount, indexStartFP, indexIn));
+        if (previous != null) {
+          throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
+        }
+      }
+
+      success = true;
+    } finally {
+      if (!success) {
+        // this.close() will close in:
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  /** Reads terms file header. */
+  protected int readHeader(IndexInput input) throws IOException {
+    int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_CODEC_NAME,
+                          BlockTreeTermsWriter.TERMS_VERSION_START,
+                          BlockTreeTermsWriter.TERMS_VERSION_CURRENT);
+    if (version < BlockTreeTermsWriter.TERMS_VERSION_APPEND_ONLY) {
+      dirOffset = input.readLong();
+    }
+    return version;
+  }
+
+  /** Reads index file header. */
+  protected int readIndexHeader(IndexInput input) throws IOException {
+    int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME,
+                          BlockTreeTermsWriter.TERMS_INDEX_VERSION_START,
+                          BlockTreeTermsWriter.TERMS_INDEX_VERSION_CURRENT);
+    if (version < BlockTreeTermsWriter.TERMS_INDEX_VERSION_APPEND_ONLY) {
+      indexDirOffset = input.readLong(); 
+    }
+    return version;
+  }
+
+  /** Seek {@code input} to the directory offset. */
+  protected void seekDir(IndexInput input, long dirOffset)
+      throws IOException {
+    if (version >= BlockTreeTermsWriter.TERMS_INDEX_VERSION_APPEND_ONLY) {
+      input.seek(input.length() - 8);
+      dirOffset = input.readLong();
+    }
+    input.seek(dirOffset);
+  }
+
+  // for debugging
+  // private static String toHex(int v) {
+  //   return "0x" + Integer.toHexString(v);
+  // }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      IOUtils.close(indexIn, in, postingsReader);
+    } finally { 
+      // Clear so refs to terms index is GCable even if
+      // app hangs onto us:
+      fields.clear();
+    }
+  }
+
+  @Override
+  public Iterator<String> iterator() {
+    return Collections.unmodifiableSet(fields.keySet()).iterator();
+  }
+
+  @Override
+  public Terms terms(String field) throws IOException {
+    assert field != null;
+    return fields.get(field);
+  }
+
+  @Override
+  public int size() {
+    return fields.size();
+  }
+
+  // for debugging
+  String brToString(BytesRef b) {
+    if (b == null) {
+      return "null";
+    } else {
+      try {
+        return b.utf8ToString() + " " + b;
+      } catch (Throwable t) {
+        // If BytesRef isn't actually UTF8, or it's eg a
+        // prefix of UTF8 that ends mid-unicode-char, we
+        // fallback to hex:
+        return b.toString();
+      }
+    }
+  }
+
+  /**
+   * BlockTree statistics for a single field 
+   * returned by {@link FieldReader#computeStats()}.
+   */
+  public static class Stats {
+    /** How many nodes in the index FST. */
+    public long indexNodeCount;
+
+    /** How many arcs in the index FST. */
+    public long indexArcCount;
+
+    /** Byte size of the index. */
+    public long indexNumBytes;
+
+    /** Total number of terms in the field. */
+    public long totalTermCount;
+
+    /** Total number of bytes (sum of term lengths) across all terms in the field. */
+    public long totalTermBytes;
+
+    /** The number of normal (non-floor) blocks in the terms file. */
+    public int nonFloorBlockCount;
+
+    /** The number of floor blocks (meta-blocks larger than the
+     *  allowed {@code maxItemsPerBlock}) in the terms file. */
+    public int floorBlockCount;
+    
+    /** The number of sub-blocks within the floor blocks. */
+    public int floorSubBlockCount;
+
+    /** The number of "internal" blocks (that have both
+     *  terms and sub-blocks). */
+    public int mixedBlockCount;
+
+    /** The number of "leaf" blocks (blocks that have only
+     *  terms). */
+    public int termsOnlyBlockCount;
+
+    /** The number of "internal" blocks that do not contain
+     *  terms (have only sub-blocks). */
+    public int subBlocksOnlyBlockCount;
+
+    /** Total number of blocks. */
+    public int totalBlockCount;
+
+    /** Number of blocks at each prefix depth. */
+    public int[] blockCountByPrefixLen = new int[10];
+    private int startBlockCount;
+    private int endBlockCount;
+
+    /** Total number of bytes used to store term suffixes. */
+    public long totalBlockSuffixBytes;
+
+    /** Total number of bytes used to store term stats (not
+     *  including what the {@link PostingsBaseFormat}
+     *  stores. */
+    public long totalBlockStatsBytes;
+
+    /** Total bytes stored by the {@link PostingsBaseFormat},
+     *  plus the other few vInts stored in the frame. */
+    public long totalBlockOtherBytes;
+
+    /** Segment name. */
+    public final String segment;
+
+    /** Field name. */
+    public final String field;
+
+    Stats(String segment, String field) {
+      this.segment = segment;
+      this.field = field;
+    }
+
+    void startBlock(FieldReader.SegmentTermsEnum.Frame frame, boolean isFloor) {
+      totalBlockCount++;
+      if (isFloor) {
+        if (frame.fp == frame.fpOrig) {
+          floorBlockCount++;
+        }
+        floorSubBlockCount++;
+      } else {
+        nonFloorBlockCount++;
+      }
+
+      if (blockCountByPrefixLen.length <= frame.prefix) {
+        blockCountByPrefixLen = ArrayUtil.grow(blockCountByPrefixLen, 1+frame.prefix);
+      }
+      blockCountByPrefixLen[frame.prefix]++;
+      startBlockCount++;
+      totalBlockSuffixBytes += frame.suffixesReader.length();
+      totalBlockStatsBytes += frame.statsReader.length();
+    }
+
+    void endBlock(FieldReader.SegmentTermsEnum.Frame frame) {
+      final int termCount = frame.isLeafBlock ? frame.entCount : frame.state.termBlockOrd;
+      final int subBlockCount = frame.entCount - termCount;
+      totalTermCount += termCount;
+      if (termCount != 0 && subBlockCount != 0) {
+        mixedBlockCount++;
+      } else if (termCount != 0) {
+        termsOnlyBlockCount++;
+      } else if (subBlockCount != 0) {
+        subBlocksOnlyBlockCount++;
+      } else {
+        throw new IllegalStateException();
+      }
+      endBlockCount++;
+      final long otherBytes = frame.fpEnd - frame.fp - frame.suffixesReader.length() - frame.statsReader.length();
+      assert otherBytes > 0 : "otherBytes=" + otherBytes + " frame.fp=" + frame.fp + " frame.fpEnd=" + frame.fpEnd;
+      totalBlockOtherBytes += otherBytes;
+    }
+
+    void term(BytesRef term) {
+      totalTermBytes += term.length;
+    }
+
+    void finish() {
+      assert startBlockCount == endBlockCount: "startBlockCount=" + startBlockCount + " endBlockCount=" + endBlockCount;
+      assert totalBlockCount == floorSubBlockCount + nonFloorBlockCount: "floorSubBlockCount=" + floorSubBlockCount + " nonFloorBlockCount=" + nonFloorBlockCount + " totalBlockCount=" + totalBlockCount;
+      assert totalBlockCount == mixedBlockCount + termsOnlyBlockCount + subBlocksOnlyBlockCount: "totalBlockCount=" + totalBlockCount + " mixedBlockCount=" + mixedBlockCount + " subBlocksOnlyBlockCount=" + subBlocksOnlyBlockCount + " termsOnlyBlockCount=" + termsOnlyBlockCount;
+    }
+
+    @Override
+    public String toString() {
+      final ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
+      PrintStream out;
+      try {
+        out = new PrintStream(bos, false, "UTF-8");
+      } catch (UnsupportedEncodingException bogus) {
+        throw new RuntimeException(bogus);
+      }
+      
+      out.println("  index FST:");
+      out.println("    " + indexNodeCount + " nodes");
+      out.println("    " + indexArcCount + " arcs");
+      out.println("    " + indexNumBytes + " bytes");
+      out.println("  terms:");
+      out.println("    " + totalTermCount + " terms");
+      out.println("    " + totalTermBytes + " bytes" + (totalTermCount != 0 ? " (" + String.format(Locale.ROOT, "%.1f", ((double) totalTermBytes)/totalTermCount) + " bytes/term)" : ""));
+      out.println("  blocks:");
+      out.println("    " + totalBlockCount + " blocks");
+      out.println("    " + termsOnlyBlockCount + " terms-only blocks");
+      out.println("    " + subBlocksOnlyBlockCount + " sub-block-only blocks");
+      out.println("    " + mixedBlockCount + " mixed blocks");
+      out.println("    " + floorBlockCount + " floor blocks");
+      out.println("    " + (totalBlockCount-floorSubBlockCount) + " non-floor blocks");
+      out.println("    " + floorSubBlockCount + " floor sub-blocks");
+      out.println("    " + totalBlockSuffixBytes + " term suffix bytes" + (totalBlockCount != 0 ? " (" + String.format(Locale.ROOT, "%.1f", ((double) totalBlockSuffixBytes)/totalBlockCount) + " suffix-bytes/block)" : ""));
+      out.println("    " + totalBlockStatsBytes + " term stats bytes" + (totalBlockCount != 0 ? " (" + String.format(Locale.ROOT, "%.1f", ((double) totalBlockStatsBytes)/totalBlockCount) + " stats-bytes/block)" : ""));
+      out.println("    " + totalBlockOtherBytes + " other bytes" + (totalBlockCount != 0 ? " (" + String.format(Locale.ROOT, "%.1f", ((double) totalBlockOtherBytes)/totalBlockCount) + " other-bytes/block)" : ""));
+      if (totalBlockCount != 0) {
+        out.println("    by prefix length:");
+        int total = 0;
+        for(int prefix=0;prefix<blockCountByPrefixLen.length;prefix++) {
+          final int blockCount = blockCountByPrefixLen[prefix];
+          total += blockCount;
+          if (blockCount != 0) {
+            out.println("      " + String.format(Locale.ROOT, "%2d", prefix) + ": " + blockCount);
+          }
+        }
+        assert totalBlockCount == total;
+      }
+
+      try {
+        return bos.toString("UTF-8");
+      } catch (UnsupportedEncodingException bogus) {
+        throw new RuntimeException(bogus);
+      }
+    }
+  }
+
+  final Outputs<BytesRef> fstOutputs = ByteSequenceOutputs.getSingleton();
+  final BytesRef NO_OUTPUT = fstOutputs.getNoOutput();
+
+  /** BlockTree's implementation of {@link Terms}. */
+  public final class FieldReader extends Terms {
+    final long numTerms;
+    final FieldInfo fieldInfo;
+    final long sumTotalTermFreq;
+    final long sumDocFreq;
+    final int docCount;
+    final long indexStartFP;
+    final long rootBlockFP;
+    final BytesRef rootCode;
+    private final FST<BytesRef> index;
+
+    //private boolean DEBUG;
+
+    FieldReader(FieldInfo fieldInfo, long numTerms, BytesRef rootCode, long sumTotalTermFreq, long sumDocFreq, int docCount, long indexStartFP, IndexInput indexIn) throws IOException {
+      assert numTerms > 0;
+      this.fieldInfo = fieldInfo;
+      //DEBUG = BlockTreeTermsReader.DEBUG && fieldInfo.name.equals("id");
+      this.numTerms = numTerms;
+      this.sumTotalTermFreq = sumTotalTermFreq; 
+      this.sumDocFreq = sumDocFreq; 
+      this.docCount = docCount;
+      this.indexStartFP = indexStartFP;
+      this.rootCode = rootCode;
+      // if (DEBUG) {
+      //   System.out.println("BTTR: seg=" + segment + " field=" + fieldInfo.name + " rootBlockCode=" + rootCode + " divisor=" + indexDivisor);
+      // }
+
+      rootBlockFP = (new ByteArrayDataInput(rootCode.bytes, rootCode.offset, rootCode.length)).readVLong() >>> BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS;
+
+      if (indexIn != null) {
+        final IndexInput clone = indexIn.clone();
+        //System.out.println("start=" + indexStartFP + " field=" + fieldInfo.name);
+        clone.seek(indexStartFP);
+        index = new FST<BytesRef>(clone, ByteSequenceOutputs.getSingleton());
+        
+        /*
+        if (false) {
+          final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
+          Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
+          Util.toDot(index, w, false, false);
+          System.out.println("FST INDEX: SAVED to " + dotFileName);
+          w.close();
+        }
+        */
+      } else {
+        index = null;
+      }
+    }
+
+    /** For debugging -- used by CheckIndex too*/
+    // TODO: maybe push this into Terms?
+    public Stats computeStats() throws IOException {
+      return new SegmentTermsEnum().computeBlockStats();
+    }
+
+    @Override
+    public Comparator<BytesRef> getComparator() {
+      return BytesRef.getUTF8SortedAsUnicodeComparator();
+    }
+
+    @Override
+    public boolean hasOffsets() {
+      return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+    }
+
+    @Override
+    public boolean hasPositions() {
+      return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+    }
+    
+    @Override
+    public boolean hasPayloads() {
+      return fieldInfo.hasPayloads();
+    }
+
+    @Override
+    public TermsEnum iterator(TermsEnum reuse) throws IOException {
+      return new SegmentTermsEnum();
+    }
+
+    @Override
+    public long size() {
+      return numTerms;
+    }
+
+    @Override
+    public long getSumTotalTermFreq() {
+      return sumTotalTermFreq;
+    }
+
+    @Override
+    public long getSumDocFreq() {
+      return sumDocFreq;
+    }
+
+    @Override
+    public int getDocCount() {
+      return docCount;
+    }
+
+    @Override
+    public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
+      if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+        throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
+      }
+      return new IntersectEnum(compiled, startTerm);
+    }
+    
+    // NOTE: cannot seek!
+    private final class IntersectEnum extends TermsEnum {
+      private final IndexInput in;
+
+      private Frame[] stack;
+      
+      @SuppressWarnings({"rawtypes","unchecked"}) private FST.Arc<BytesRef>[] arcs = new FST.Arc[5];
+
+      private final RunAutomaton runAutomaton;
+      private final CompiledAutomaton compiledAutomaton;
+
+      private Frame currentFrame;
+
+      private final BytesRef term = new BytesRef();
+
+      private final FST.BytesReader fstReader;
+
+      // TODO: can we share this with the frame in STE?
+      private final class Frame {
+        final int ord;
+        long fp;
+        long fpOrig;
+        long fpEnd;
+        long lastSubFP;
+
+        // State in automaton
+        int state;
+
+        int metaDataUpto;
+
+        byte[] suffixBytes = new byte[128];
+        final ByteArrayDataInput suffixesReader = new ByteArrayDataInput();
+
+        byte[] statBytes = new byte[64];
+        final ByteArrayDataInput statsReader = new ByteArrayDataInput();
+
+        byte[] floorData = new byte[32];
+        final ByteArrayDataInput floorDataReader = new ByteArrayDataInput();
+
+        // Length of prefix shared by all terms in this block
+        int prefix;
+
+        // Number of entries (term or sub-block) in this block
+        int entCount;
+
+        // Which term we will next read
+        int nextEnt;
+
+        // True if this block is either not a floor block,
+        // or, it's the last sub-block of a floor block
+        boolean isLastInFloor;
+
+        // True if all entries are terms
+        boolean isLeafBlock;
+
+        int numFollowFloorBlocks;
+        int nextFloorLabel;
+        
+        Transition[] transitions;
+        int curTransitionMax;
+        int transitionIndex;
+
+        FST.Arc<BytesRef> arc;
+
+        final BlockTermState termState;
+
+        // Cumulative output so far
+        BytesRef outputPrefix;
+
+        private int startBytePos;
+        private int suffix;
+
+        public Frame(int ord) throws IOException {
+          this.ord = ord;
+          termState = postingsReader.newTermState();
+          termState.totalTermFreq = -1;
+        }
+
+        void loadNextFloorBlock() throws IOException {
+          assert numFollowFloorBlocks > 0;
+          //if (DEBUG) System.out.println("    loadNextFoorBlock trans=" + transitions[transitionIndex]);
+
+          do {
+            fp = fpOrig + (floorDataReader.readVLong() >>> 1);
+            numFollowFloorBlocks--;
+            // if (DEBUG) System.out.println("    skip floor block2!  nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[transitionIndex].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);
+            if (numFollowFloorBlocks != 0) {
+              nextFloorLabel = floorDataReader.readByte() & 0xff;
+            } else {
+              nextFloorLabel = 256;
+            }
+            // if (DEBUG) System.out.println("    nextFloorLabel=" + (char) nextFloorLabel);
+          } while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[transitionIndex].getMin());
+
+          load(null);
+        }
+
+        public void setState(int state) {
+          this.state = state;
+          transitionIndex = 0;
+          transitions = compiledAutomaton.sortedTransitions[state];
+          if (transitions.length != 0) {
+            curTransitionMax = transitions[0].getMax();
+          } else {
+            curTransitionMax = -1;
+          }
+        }
+
+        void load(BytesRef frameIndexData) throws IOException {
+
+          // if (DEBUG) System.out.println("    load fp=" + fp + " fpOrig=" + fpOrig + " frameIndexData=" + frameIndexData + " trans=" + (transitions.length != 0 ? transitions[0] : "n/a" + " state=" + state));
+
+          if (frameIndexData != null && transitions.length != 0) {
+            // Floor frame
+            if (floorData.length < frameIndexData.length) {
+              this.floorData = new byte[ArrayUtil.oversize(frameIndexData.length, 1)];
+            }
+            System.arraycopy(frameIndexData.bytes, frameIndexData.offset, floorData, 0, frameIndexData.length);
+            floorDataReader.reset(floorData, 0, frameIndexData.length);
+            // Skip first long -- has redundant fp, hasTerms
+            // flag, isFloor flag
+            final long code = floorDataReader.readVLong();
+            if ((code & BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0) {
+              numFollowFloorBlocks = floorDataReader.readVInt();
+              nextFloorLabel = floorDataReader.readByte() & 0xff;
+              // if (DEBUG) System.out.println("    numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + nextFloorLabel);
+
+              // If current state is accept, we must process
+              // first block in case it has empty suffix:
+              if (!runAutomaton.isAccept(state)) {
+                // Maybe skip floor blocks:
+                while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[0].getMin()) {
+                  fp = fpOrig + (floorDataReader.readVLong() >>> 1);
+                  numFollowFloorBlocks--;
+                  // if (DEBUG) System.out.println("    skip floor block!  nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[0].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);
+                  if (numFollowFloorBlocks != 0) {
+                    nextFloorLabel = floorDataReader.readByte() & 0xff;
+                  } else {
+                    nextFloorLabel = 256;
+                  }
+                }
+              }
+            }
+          }
+
+          in.seek(fp);
+          int code = in.readVInt();
+          entCount = code >>> 1;
+          assert entCount > 0;
+          isLastInFloor = (code & 1) != 0;
+
+          // term suffixes:
+          code = in.readVInt();
+          isLeafBlock = (code & 1) != 0;
+          int numBytes = code >>> 1;
+          // if (DEBUG) System.out.println("      entCount=" + entCount + " lastInFloor?=" + isLastInFloor + " leafBlock?=" + isLeafBlock + " numSuffixBytes=" + numBytes);
+          if (suffixBytes.length < numBytes) {
+            suffixBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+          }
+          in.readBytes(suffixBytes, 0, numBytes);
+          suffixesReader.reset(suffixBytes, 0, numBytes);
+
+          // stats
+          numBytes = in.readVInt();
+          if (statBytes.length < numBytes) {
+            statBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+          }
+          in.readBytes(statBytes, 0, numBytes);
+          statsReader.reset(statBytes, 0, numBytes);
+          metaDataUpto = 0;
+
+          termState.termBlockOrd = 0;
+          nextEnt = 0;
+          
+          postingsReader.readTermsBlock(in, fieldInfo, termState);
+
+          if (!isLastInFloor) {
+            // Sub-blocks of a single floor block are always
+            // written one after another -- tail recurse:
+            fpEnd = in.getFilePointer();
+          }
+        }
+
+        // TODO: maybe add scanToLabel; should give perf boost
+
+        public boolean next() {
+          return isLeafBlock ? nextLeaf() : nextNonLeaf();
+        }
+
+        // Decodes next entry; returns true if it's a sub-block
+        public boolean nextLeaf() {
+          //if (DEBUG) System.out.println("  frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+          assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
+          nextEnt++;
+          suffix = suffixesReader.readVInt();
+          startBytePos = suffixesReader.getPosition();
+          suffixesReader.skipBytes(suffix);
+          return false;
+        }
+
+        public boolean nextNonLeaf() {
+          //if (DEBUG) System.out.println("  frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+          assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
+          nextEnt++;
+          final int code = suffixesReader.readVInt();
+          suffix = code >>> 1;
+          startBytePos = suffixesReader.getPosition();
+          suffixesReader.skipBytes(suffix);
+          if ((code & 1) == 0) {
+            // A normal term
+            termState.termBlockOrd++;
+            return false;
+          } else {
+            // A sub-block; make sub-FP absolute:
+            lastSubFP = fp - suffixesReader.readVLong();
+            return true;
+          }
+        }
+
+        public int getTermBlockOrd() {
+          return isLeafBlock ? nextEnt : termState.termBlockOrd;
+        }
+
+        public void decodeMetaData() throws IOException {
+
+          // lazily catch up on metadata decode:
+          final int limit = getTermBlockOrd();
+          assert limit > 0;
+
+          // We must set/incr state.termCount because
+          // postings impl can look at this
+          termState.termBlockOrd = metaDataUpto;
+      
+          // TODO: better API would be "jump straight to term=N"???
+          while (metaDataUpto < limit) {
+
+            // TODO: we could make "tiers" of metadata, ie,
+            // decode docFreq/totalTF but don't decode postings
+            // metadata; this way caller could get
+            // docFreq/totalTF w/o paying decode cost for
+            // postings
+
+            // TODO: if docFreq were bulk decoded we could
+            // just skipN here:
+            termState.docFreq = statsReader.readVInt();
+            //if (DEBUG) System.out.println("    dF=" + state.docFreq);
+            if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+              termState.totalTermFreq = termState.docFreq + statsReader.readVLong();
+              //if (DEBUG) System.out.println("    totTF=" + state.totalTermFreq);
+            }
+
+            postingsReader.nextTerm(fieldInfo, termState);
+            metaDataUpto++;
+            termState.termBlockOrd++;
+          }
+        }
+      }
+
+      private BytesRef savedStartTerm;
+      
+      // TODO: in some cases we can filter by length?  eg
+      // regexp foo*bar must be at least length 6 bytes
+      public IntersectEnum(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
+        // if (DEBUG) {
+        //   System.out.println("\nintEnum.init seg=" + segment + " commonSuffix=" + brToString(compiled.commonSuffixRef));
+        // }
+        runAutomaton = compiled.runAutomaton;
+        compiledAutomaton = compiled;
+        in = BlockTreeTermsReader.this.in.clone();
+        stack = new Frame[5];
+        for(int idx=0;idx<stack.length;idx++) {
+          stack[idx] = new Frame(idx);
+        }
+        for(int arcIdx=0;arcIdx<arcs.length;arcIdx++) {
+          arcs[arcIdx] = new FST.Arc<BytesRef>();
+        }
+
+        if (index == null) {
+          fstReader = null;
+        } else {
+          fstReader = index.getBytesReader();
+        }
+
+        // TODO: if the automaton is "smallish" we really
+        // should use the terms index to seek at least to
+        // the initial term and likely to subsequent terms
+        // (or, maybe just fallback to ATE for such cases).
+        // Else the seek cost of loading the frames will be
+        // too costly.
+
+        final FST.Arc<BytesRef> arc = index.getFirstArc(arcs[0]);
+        // Empty string prefix must have an output in the index!
+        assert arc.isFinal();
+
+        // Special pushFrame since it's the first one:
+        final Frame f = stack[0];
+        f.fp = f.fpOrig = rootBlockFP;
+        f.prefix = 0;
+        f.setState(runAutomaton.getInitialState());
+        f.arc = arc;
+        f.outputPrefix = arc.output;
+        f.load(rootCode);
+
+        // for assert:
+        assert setSavedStartTerm(startTerm);
+
+        currentFrame = f;
+        if (startTerm != null) {
+          seekToStartTerm(startTerm);
+        }
+      }
+
+      // only for assert:
+      private boolean setSavedStartTerm(BytesRef startTerm) {
+        savedStartTerm = startTerm == null ? null : BytesRef.deepCopyOf(startTerm);
+        return true;
+      }
+
+      @Override
+      public TermState termState() throws IOException {
+        currentFrame.decodeMetaData();
+        return currentFrame.termState.clone();
+      }
+
+      private Frame getFrame(int ord) throws IOException {
+        if (ord >= stack.length) {
+          final Frame[] next = new Frame[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+          System.arraycopy(stack, 0, next, 0, stack.length);
+          for(int stackOrd=stack.length;stackOrd<next.length;stackOrd++) {
+            next[stackOrd] = new Frame(stackOrd);
+          }
+          stack = next;
+        }
+        assert stack[ord].ord == ord;
+        return stack[ord];
+      }
+
+      private FST.Arc<BytesRef> getArc(int ord) {
+        if (ord >= arcs.length) {
+          @SuppressWarnings({"rawtypes","unchecked"}) final FST.Arc<BytesRef>[] next =
+            new FST.Arc[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+          System.arraycopy(arcs, 0, next, 0, arcs.length);
+          for(int arcOrd=arcs.length;arcOrd<next.length;arcOrd++) {
+            next[arcOrd] = new FST.Arc<BytesRef>();
+          }
+          arcs = next;
+        }
+        return arcs[ord];
+      }
+
+      private Frame pushFrame(int state) throws IOException {
+        final Frame f = getFrame(currentFrame == null ? 0 : 1+currentFrame.ord);
+        
+        f.fp = f.fpOrig = currentFrame.lastSubFP;
+        f.prefix = currentFrame.prefix + currentFrame.suffix;
+        // if (DEBUG) System.out.println("    pushFrame state=" + state + " prefix=" + f.prefix);
+        f.setState(state);
+
+        // Walk the arc through the index -- we only
+        // "bother" with this so we can get the floor data
+        // from the index and skip floor blocks when
+        // possible:
+        FST.Arc<BytesRef> arc = currentFrame.arc;
+        int idx = currentFrame.prefix;
+        assert currentFrame.suffix > 0;
+        BytesRef output = currentFrame.outputPrefix;
+        while (idx < f.prefix) {
+          final int target = term.bytes[idx] & 0xff;
+          // TODO: we could be more efficient for the next()
+          // case by using current arc as starting point,
+          // passed to findTargetArc
+          arc = index.findTargetArc(target, arc, getArc(1+idx), fstReader);
+          assert arc != null;
+          output = fstOutputs.add(output, arc.output);
+          idx++;
+        }
+
+        f.arc = arc;
+        f.outputPrefix = output;
+        assert arc.isFinal();
+        f.load(fstOutputs.add(output, arc.nextFinalOutput));
+        return f;
+      }
+
+      @Override
+      public BytesRef term() {
+        return term;
+      }
+
+      @Override
+      public int docFreq() throws IOException {
+        //if (DEBUG) System.out.println("BTIR.docFreq");
+        currentFrame.decodeMetaData();
+        //if (DEBUG) System.out.println("  return " + currentFrame.termState.docFreq);
+        return currentFrame.termState.docFreq;
+      }
+
+      @Override
+      public long totalTermFreq() throws IOException {
+        currentFrame.decodeMetaData();
+        return currentFrame.termState.totalTermFreq;
+      }
+
+      @Override
+      public DocsEnum docs(Bits skipDocs, DocsEnum reuse, int flags) throws IOException {
+        currentFrame.decodeMetaData();
+        return postingsReader.docs(fieldInfo, currentFrame.termState, skipDocs, reuse, flags);
+      }
+
+      @Override
+      public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
+        if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
+          // Positions were not indexed:
+          return null;
+        }
+
+        currentFrame.decodeMetaData();
+        return postingsReader.docsAndPositions(fieldInfo, currentFrame.termState, skipDocs, reuse, flags);
+      }
+
+      private int getState() {
+        int state = currentFrame.state;
+        for(int idx=0;idx<currentFrame.suffix;idx++) {
+          state = runAutomaton.step(state,  currentFrame.suffixBytes[currentFrame.startBytePos+idx] & 0xff);
+          assert state != -1;
+        }
+        return state;
+      }
+
+      // NOTE: specialized to only doing the first-time
+      // seek, but we could generalize it to allow
+      // arbitrary seekExact/Ceil.  Note that this is a
+      // seekFloor!
+      private void seekToStartTerm(BytesRef target) throws IOException {
+        //if (DEBUG) System.out.println("seek to startTerm=" + target.utf8ToString());
+        assert currentFrame.ord == 0;
+        if (term.length < target.length) {
+          term.bytes = ArrayUtil.grow(term.bytes, target.length);
+        }
+        FST.Arc<BytesRef> arc = arcs[0];
+        assert arc == currentFrame.arc;
+
+        for(int idx=0;idx<=target.length;idx++) {
+
+          while (true) {
+            final int savePos = currentFrame.suffixesReader.getPosition();
+            final int saveStartBytePos = currentFrame.startBytePos;
+            final int saveSuffix = currentFrame.suffix;
+            final long saveLastSubFP = currentFrame.lastSubFP;
+            final int saveTermBlockOrd = currentFrame.termState.termBlockOrd;
+
+            final boolean isSubBlock = currentFrame.next();
+
+            //if (DEBUG) System.out.println("    cycle ent=" + currentFrame.nextEnt + " (of " + currentFrame.entCount + ") prefix=" + currentFrame.prefix + " suffix=" + currentFrame.suffix + " isBlock=" + isSubBlock + " firstLabel=" + (currentFrame.suffix == 0 ? "" : (currentFrame.suffixBytes[currentFrame.startBytePos])&0xff));
+            term.length = currentFrame.prefix + currentFrame.suffix;
+            if (term.bytes.length < term.length) {
+              term.bytes = ArrayUtil.grow(term.bytes, term.length);
+            }
+            System.arraycopy(currentFrame.suffixBytes, currentFrame.startBytePos, term.bytes, currentFrame.prefix, currentFrame.suffix);
+
+            if (isSubBlock && StringHelper.startsWith(target, term)) {
+              // Recurse
+              //if (DEBUG) System.out.println("      recurse!");
+              currentFrame = pushFrame(getState());
+              break;
+            } else {
+              final int cmp = term.compareTo(target);
+              if (cmp < 0) {
+                if (currentFrame.nextEnt == currentFrame.entCount) {
+                  if (!currentFrame.isLastInFloor) {
+                    //if (DEBUG) System.out.println("  load floorBlock");
+                    currentFrame.loadNextFloorBlock();
+                    continue;
+                  } else {
+                    //if (DEBUG) System.out.println("  return term=" + brToString(term));
+                    return;
+                  }
+                }
+                continue;
+              } else if (cmp == 0) {
+                //if (DEBUG) System.out.println("  return term=" + brToString(term));
+                return;
+              } else {
+                // Fallback to prior entry: the semantics of
+                // this method is that the first call to
+                // next() will return the term after the
+                // requested term
+                currentFrame.nextEnt--;
+                currentFrame.lastSubFP = saveLastSubFP;
+                currentFrame.startBytePos = saveStartBytePos;
+                currentFrame.suffix = saveSuffix;
+                currentFrame.suffixesReader.setPosition(savePos);
+                currentFrame.termState.termBlockOrd = saveTermBlockOrd;
+                System.arraycopy(currentFrame.suffixBytes, currentFrame.startBytePos, term.bytes, currentFrame.prefix, currentFrame.suffix);
+                term.length = currentFrame.prefix + currentFrame.suffix;
+                // If the last entry was a block we don't
+                // need to bother recursing and pushing to
+                // the last term under it because the first
+                // next() will simply skip the frame anyway
+                return;
+              }
+            }
+          }
+        }
+
+        assert false;
+      }
+
+      @Override
+      public BytesRef next() throws IOException {
+
+        // if (DEBUG) {
+        //   System.out.println("\nintEnum.next seg=" + segment);
+        //   System.out.println("  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
+        // }
+
+        nextTerm:
+        while(true) {
+          // Pop finished frames
+          while (currentFrame.nextEnt == currentFrame.entCount) {
+            if (!currentFrame.isLastInFloor) {
+              //if (DEBUG) System.out.println("    next-floor-block");
+              currentFrame.loadNextFloorBlock();
+              //if (DEBUG) System.out.println("\n  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
+            } else {
+              //if (DEBUG) System.out.println("  pop frame");
+              if (currentFrame.ord == 0) {
+                return null;
+              }
+              final long lastFP = currentFrame.fpOrig;
+              currentFrame = stack[currentFrame.ord-1];
+              assert currentFrame.lastSubFP == lastFP;
+              //if (DEBUG) System.out.println("\n  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
+            }
+          }
+
+          final boolean isSubBlock = currentFrame.next();
+          // if (DEBUG) {
+          //   final BytesRef suffixRef = new BytesRef();
+          //   suffixRef.bytes = currentFrame.suffixBytes;
+          //   suffixRef.offset = currentFrame.startBytePos;
+          //   suffixRef.length = currentFrame.suffix;
+          //   System.out.println("    " + (isSubBlock ? "sub-block" : "term") + " " + currentFrame.nextEnt + " (of " + currentFrame.entCount + ") suffix=" + brToString(suffixRef));
+          // }
+
+          if (currentFrame.suffix != 0) {
+            final int label = currentFrame.suffixBytes[currentFrame.startBytePos] & 0xff;
+            while (label > currentFrame.curTransitionMax) {
+              if (currentFrame.transitionIndex >= currentFrame.transitions.length-1) {
+                // Stop processing this frame -- no further
+                // matches are possible because we've moved
+                // beyond what the max transition will allow
+                //if (DEBUG) System.out.println("      break: trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]));
+
+                // sneaky!  forces a pop above
+                currentFrame.isLastInFloor = true;
+                currentFrame.nextEnt = currentFrame.entCount;
+                continue nextTerm;
+              }
+              currentFrame.transitionIndex++;
+              currentFrame.curTransitionMax = currentFrame.transitions[currentFrame.transitionIndex].getMax();
+              //if (DEBUG) System.out.println("      next trans=" + currentFrame.transitions[currentFrame.transitionIndex]);
+            }
+          }
+
+          // First test the common suffix, if set:
+          if (compiledAutomaton.commonSuffixRef != null && !isSubBlock) {
+            final int termLen = currentFrame.prefix + currentFrame.suffix;
+            if (termLen < compiledAutomaton.commonSuffixRef.length) {
+              // No match
+              // if (DEBUG) {
+              //   System.out.println("      skip: common suffix length");
+              // }
+              continue nextTerm;
+            }
+
+            final byte[] suffixBytes = currentFrame.suffixBytes;
+            final byte[] commonSuffixBytes = compiledAutomaton.commonSuffixRef.bytes;
+
+            final int lenInPrefix = compiledAutomaton.commonSuffixRef.length - currentFrame.suffix;
+            assert compiledAutomaton.commonSuffixRef.offset == 0;
+            int suffixBytesPos;
+            int commonSuffixBytesPos = 0;
+
+            if (lenInPrefix > 0) {
+              // A prefix of the common suffix overlaps with
+              // the suffix of the block prefix so we first
+              // test whether the prefix part matches:
+              final byte[] termBytes = term.bytes;
+              int termBytesPos = currentFrame.prefix - lenInPrefix;
+              assert termBytesPos >= 0;
+              final int termBytesPosEnd = currentFrame.prefix;
+              while (termBytesPos < termBytesPosEnd) {
+                if (termBytes[termBytesPos++] != commonSuffixBytes[commonSuffixBytesPos++]) {
+                  // if (DEBUG) {
+                  //   System.out.println("      skip: common suffix mismatch (in prefix)");
+                  // }
+                  continue nextTerm;
+                }
+              }
+              suffixBytesPos = currentFrame.startBytePos;
+            } else {
+              suffixBytesPos = currentFrame.startBytePos + currentFrame.suffix - compiledAutomaton.commonSuffixRef.length;
+            }
+
+            // Test overlapping suffix part:
+            final int commonSuffixBytesPosEnd = compiledAutomaton.commonSuffixRef.length;
+            while (commonSuffixBytesPos < commonSuffixBytesPosEnd) {
+              if (suffixBytes[suffixBytesPos++] != commonSuffixBytes[commonSuffixBytesPos++]) {
+                // if (DEBUG) {
+                //   System.out.println("      skip: common suffix mismatch");
+                // }
+                continue nextTerm;
+              }
+            }
+          }
+
+          // TODO: maybe we should do the same linear test
+          // that AutomatonTermsEnum does, so that if we
+          // reach a part of the automaton where .* is
+          // "temporarily" accepted, we just blindly .next()
+          // until the limit
+
+          // See if the term prefix matches the automaton:
+          int state = currentFrame.state;
+          for (int idx=0;idx<currentFrame.suffix;idx++) {
+            state = runAutomaton.step(state,  currentFrame.suffixBytes[currentFrame.startBytePos+idx] & 0xff);
+            if (state == -1) {
+              // No match
+              //System.out.println("    no s=" + state);
+              continue nextTerm;
+            } else {
+              //System.out.println("    c s=" + state);
+            }
+          }
+
+          if (isSubBlock) {
+            // Match!  Recurse:
+            //if (DEBUG) System.out.println("      sub-block match to state=" + state + "; recurse fp=" + currentFrame.lastSubFP);
+            copyTerm();
+            currentFrame = pushFrame(state);
+            //if (DEBUG) System.out.println("\n  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
+          } else if (runAutomaton.isAccept(state)) {
+            copyTerm();
+            //if (DEBUG) System.out.println("      term match to state=" + state + "; return term=" + brToString(term));
+            assert savedStartTerm == null || term.compareTo(savedStartTerm) > 0: "saveStartTerm=" + savedStartTerm.utf8ToString() + " term=" + term.utf8ToString();
+            return term;
+          } else {
+            //System.out.println("    no s=" + state);
+          }
+        }
+      }
+
+      private void copyTerm() {
+        //System.out.println("      copyTerm cur.prefix=" + currentFrame.prefix + " cur.suffix=" + currentFrame.suffix + " first=" + (char) currentFrame.suffixBytes[currentFrame.startBytePos]);
+        final int len = currentFrame.prefix + currentFrame.suffix;
+        if (term.bytes.length < len) {
+          term.bytes = ArrayUtil.grow(term.bytes, len);
+        }
+        System.arraycopy(currentFrame.suffixBytes, currentFrame.startBytePos, term.bytes, currentFrame.prefix, currentFrame.suffix);
+        term.length = len;
+      }
+
+      @Override
+      public Comparator<BytesRef> getComparator() {
+        return BytesRef.getUTF8SortedAsUnicodeComparator();
+      }
+
+      @Override
+      public boolean seekExact(BytesRef text, boolean useCache) {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public void seekExact(long ord) {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public long ord() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public SeekStatus seekCeil(BytesRef text, boolean useCache) {
+        throw new UnsupportedOperationException();
+      }
+    }
+
+    // Iterates through terms in this field
+    private final class SegmentTermsEnum extends TermsEnum {
+      private IndexInput in;
+
+      private Frame[] stack;
+      private final Frame staticFrame;
+      private Frame currentFrame;
+      private boolean termExists;
+
+      private int targetBeforeCurrentLength;
+
+      private final ByteArrayDataInput scratchReader = new ByteArrayDataInput();
+
+      // What prefix of the current term was present in the index:
+      private int validIndexPrefix;
+
+      // assert only:
+      private boolean eof;
+
+      final BytesRef term = new BytesRef();
+      private final FST.BytesReader fstReader;
+
+      @SuppressWarnings({"rawtypes","unchecked"}) private FST.Arc<BytesRef>[] arcs =
+          new FST.Arc[1];
+
+      public SegmentTermsEnum() throws IOException {
+        //if (DEBUG) System.out.println("BTTR.init seg=" + segment);
+        stack = new Frame[0];
+        
+        // Used to hold seek by TermState, or cached seek
+        staticFrame = new Frame(-1);
+
+        if (index == null) {
+          fstReader = null;
+        } else {
+          fstReader = index.getBytesReader();
+        }
+
+        // Init w/ root block; don't use index since it may
+        // not (and need not) have been loaded
+        for(int arcIdx=0;arcIdx<arcs.length;arcIdx++) {
+          arcs[arcIdx] = new FST.Arc<BytesRef>();
+        }
+
+        currentFrame = staticFrame;
+        final FST.Arc<BytesRef> arc;
+        if (index != null) {
+          arc = index.getFirstArc(arcs[0]);
+          // Empty string prefix must have an output in the index!
+          assert arc.isFinal();
+        } else {
+          arc = null;
+        }
+        currentFrame = staticFrame;
+        //currentFrame = pushFrame(arc, rootCode, 0);
+        //currentFrame.loadBlock();
+        validIndexPrefix = 0;
+        // if (DEBUG) {
+        //   System.out.println("init frame state " + currentFrame.ord);
+        //   printSeekState();
+        // }
+
+        //System.out.println();
+        // computeBlockStats().print(System.out);
+      }
+      
+      // Not private to avoid synthetic access$NNN methods
+      void initIndexInput() {
+        if (this.in == null) {
+          this.in = BlockTreeTermsReader.this.in.clone();
+        }
+      }
+
+      /** Runs next() through the entire terms dict,
+       *  computing aggregate statistics. */
+      public Stats computeBlockStats() throws IOException {
+
+        Stats stats = new Stats(segment, fieldInfo.name);
+        if (index != null) {
+          stats.indexNodeCount = index.getNodeCount();
+          stats.indexArcCount = index.getArcCount();
+          stats.indexNumBytes = index.sizeInBytes();
+        }
+        
+        currentFrame = staticFrame;
+        FST.Arc<BytesRef> arc;
+        if (index != null) {
+          arc = index.getFirstArc(arcs[0]);
+          // Empty string prefix must have an output in the index!
+          assert arc.isFinal();
+        } else {
+          arc = null;
+        }
+
+        // Empty string prefix must have an output in the
+        // index!
+        currentFrame = pushFrame(arc, rootCode, 0);
+        currentFrame.fpOrig = currentFrame.fp;
+        currentFrame.loadBlock();
+        validIndexPrefix = 0;
+
+        stats.startBlock(currentFrame, !currentFrame.isLastInFloor);
+
+        allTerms:
+        while (true) {
+
+          // Pop finished blocks
+          while (currentFrame.nextEnt == currentFrame.entCount) {
+            stats.endBlock(currentFrame);
+            if (!currentFrame.isLastInFloor) {
+              currentFrame.loadNextFloorBlock();
+              stats.startBlock(currentFrame, true);
+            } else {
+              if (currentFrame.ord == 0) {
+                break allTerms;
+              }
+              final long lastFP = currentFrame.fpOrig;
+              currentFrame = stack[currentFrame.ord-1];
+              assert lastFP == currentFrame.lastSubFP;
+              // if (DEBUG) {
+              //   System.out.println("  reset validIndexPrefix=" + validIndexPrefix);
+              // }
+            }
+          }
+
+          while(true) {
+            if (currentFrame.next()) {
+              // Push to new block:
+              currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length);
+              currentFrame.fpOrig = currentFrame.fp;
+              // This is a "next" frame -- even if it's
+              // floor'd we must pretend it isn't so we don't
+              // try to scan to the right floor frame:
+              currentFrame.isFloor = false;
+              //currentFrame.hasTerms = true;
+              currentFrame.loadBlock();
+              stats.startBlock(currentFrame, !currentFrame.isLastInFloor);
+            } else {
+              stats.term(term);
+              break;
+            }
+          }
+        }
+
+        stats.finish();
+
+        // Put root frame back:
+        currentFrame = staticFrame;
+        if (index != null) {
+          arc = index.getFirstArc(arcs[0]);
+          // Empty string prefix must have an output in the index!
+          assert arc.isFinal();
+        } else {
+          arc = null;
+        }
+        currentFrame = pushFrame(arc, rootCode, 0);
+        currentFrame.rewind();
+        currentFrame.loadBlock();
+        validIndexPrefix = 0;
+        term.length = 0;
+
+        return stats;
+      }
+
+      private Frame getFrame(int ord) throws IOException {
+        if (ord >= stack.length) {
+          final Frame[] next = new Frame[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+          System.arraycopy(stack, 0, next, 0, stack.length);
+          for(int stackOrd=stack.length;stackOrd<next.length;stackOrd++) {
+            next[stackOrd] = new Frame(stackOrd);
+          }
+          stack = next;
+        }
+        assert stack[ord].ord == ord;
+        return stack[ord];
+      }
+
+      private FST.Arc<BytesRef> getArc(int ord) {
+        if (ord >= arcs.length) {
+          @SuppressWarnings({"rawtypes","unchecked"}) final FST.Arc<BytesRef>[] next =
+              new FST.Arc[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+          System.arraycopy(arcs, 0, next, 0, arcs.length);
+          for(int arcOrd=arcs.length;arcOrd<next.length;arcOrd++) {
+            next[arcOrd] = new FST.Arc<BytesRef>();
+          }
+          arcs = next;
+        }
+        return arcs[ord];
+      }
+
+      @Override
+      public Comparator<BytesRef> getComparator() {
+        return BytesRef.getUTF8SortedAsUnicodeComparator();
+      }
+
+      // Pushes a frame we seek'd to
+      Frame pushFrame(FST.Arc<BytesRef> arc, BytesRef frameData, int length) throws IOException {
+        scratchReader.reset(frameData.bytes, frameData.offset, frameData.length);
+        final long code = scratchReader.readVLong();
+        final long fpSeek = code >>> BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS;
+        final Frame f = getFrame(1+currentFrame.ord);
+        f.hasTerms = (code & BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS) != 0;
+        f.hasTermsOrig = f.hasTerms;
+        f.isFloor = (code & BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0;
+        if (f.isFloor) {
+          f.setFloorData(scratchReader, frameData);
+        }
+        pushFrame(arc, fpSeek, length);
+
+        return f;
+      }
+
+      // Pushes next'd frame or seek'd frame; we later
+      // lazy-load the frame only when needed
+      Frame pushFrame(FST.Arc<BytesRef> arc, long fp, int length) throws IOException {
+        final Frame f = getFrame(1+currentFrame.ord);
+        f.arc = arc;
+        if (f.fpOrig == fp && f.nextEnt != -1) {
+          //if (DEBUG) System.out.println("      push reused frame ord=" + f.ord + " fp=" + f.fp + " isFloor?=" + f.isFloor + " hasTerms=" + f.hasTerms + " pref=" + term + " nextEnt=" + f.nextEnt + " targetBeforeCurrentLength=" + targetBeforeCurrentLength + " term.length=" + term.length + " vs prefix=" + f.prefix);
+          if (f.prefix > targetBeforeCurrentLength) {
+            f.rewind();
+          } else {
+            // if (DEBUG) {
+            //   System.out.println("        skip rewind!");
+            // }
+          }
+          assert length == f.prefix;
+        } else {
+          f.nextEnt = -1;
+          f.prefix = length;
+          f.state.termBlockOrd = 0;
+          f.fpOrig = f.fp = fp;
+          f.lastSubFP = -1;
+          // if (DEBUG) {
+          //   final int sav = term.length;
+          //   term.length = length;
+          //   System.out.println("      push new frame ord=" + f.ord + " fp=" + f.fp + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " pref=" + brToString(term));
+          //   term.length = sav;
+          // }
+        }
+
+        return f;
+      }
+
+      // asserts only
+      private boolean clearEOF() {
+        eof = false;
+        return true;
+      }
+
+      // asserts only
+      private boolean setEOF() {
+        eof = true;
+        return true;
+      }
+
+      @Override
+      public boolean seekExact(final BytesRef target, final boolean useCache) throws IOException {
+
+        if (index == null) {
+          throw new IllegalStateException("terms index was not loaded");
+        }
+
+        if (term.bytes.length <= target.length) {
+          term.bytes = ArrayUtil.grow(term.bytes, 1+target.length);
+        }
+
+        assert clearEOF();
+
+        // if (DEBUG) {
+        //   System.out.println("\nBTTR.seekExact seg=" + segment + " target=" + fieldInfo.name + ":" + brToString(target) + " current=" + brToString(term) + " (exists?=" + termExists + ") validIndexPrefix=" + validIndexPrefix);
+        //   printSeekState();
+        // }
+
+        FST.Arc<BytesRef> arc;
+        int targetUpto;
+        BytesRef output;
+
+        targetBeforeCurrentLength = currentFrame.ord;
+
+        if (currentFrame != staticFrame) {
+
+          // We are already seek'd; find the common
+          // prefix of new seek term vs current term and
+          // re-use the corresponding seek state.  For
+          // example, if app first seeks to foobar, then
+          // seeks to foobaz, we can re-use the seek state
+          // for the first 5 bytes.
+
+          // if (DEBUG) {
+          //   System.out.println("  re-use current seek state validIndexPrefix=" + validIndexPrefix);
+          // }
+
+          arc = arcs[0];
+          assert arc.isFinal();
+          output = arc.output;
+          targetUpto = 0;
+          
+          Frame lastFrame = stack[0];
+          assert validIndexPrefix <= term.length;
+
+          final int targetLimit = Math.min(target.length, validIndexPrefix);
+
+          int cmp = 0;
+
+          // TODO: reverse vLong byte order for better FST
+          // prefix output sharing
+
+          // First compare up to valid seek frames:
+          while (targetUpto < targetLimit) {
+            cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+            // if (DEBUG) {
+            //   System.out.println("    cycle targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")"   + " arc.output=" + arc.output + " output=" + output);
+            // }
+            if (cmp != 0) {
+              break;
+            }
+            arc = arcs[1+targetUpto];
+            //if (arc.label != (target.bytes[target.offset + targetUpto] & 0xFF)) {
+            //System.out.println("FAIL: arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF));
+            //}
+            assert arc.label == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
+            if (arc.output != NO_OUTPUT) {
+              output = fstOutputs.add(output, arc.output);
+            }
+            if (arc.isFinal()) {
+              lastFrame = stack[1+lastFrame.ord];
+            }
+            targetUpto++;
+          }
+
+          if (cmp == 0) {
+            final int targetUptoMid = targetUpto;
+
+            // Second compare the rest of the term, but
+            // don't save arc/output/frame; we only do this
+            // to find out if the target term is before,
+            // equal or after the current term
+            final int targetLimit2 = Math.min(target.length, term.length);
+            while (targetUpto < targetLimit2) {
+              cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+              // if (DEBUG) {
+              //   System.out.println("    cycle2 targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")");
+              // }
+              if (cmp != 0) {
+                break;
+              }
+              targetUpto++;
+            }
+
+            if (cmp == 0) {
+              cmp = term.length - target.length;
+            }
+            targetUpto = targetUptoMid;
+          }
+
+          if (cmp < 0) {
+            // Common case: target term is after current
+            // term, ie, app is seeking multiple terms
+            // in sorted order
+            // if (DEBUG) {
+            //   System.out.println("  target is after current (shares prefixLen=" + targetUpto + "); frame.ord=" + lastFrame.ord);
+            // }
+            currentFrame = lastFrame;
+
+          } else if (cmp > 0) {
+            // Uncommon case: target term
+            // is before current term; this means we can
+            // keep the currentFrame but we must rewind it
+            // (so we scan from the start)
+            targetBeforeCurrentLength = 0;
+            // if (DEBUG) {
+            //   System.out.println("  target is before current (shares prefixLen=" + targetUpto + "); rewind frame ord=" + lastFrame.ord);
+            // }
+            currentFrame = lastFrame;
+            currentFrame.rewind();
+          } else {
+            // Target is exactly the same as current term
+            assert term.length == target.length;
+            if (termExists) {
+              // if (DEBUG) {
+              //   System.out.println("  target is same as current; return true");
+              // }
+              return true;
+            } else {
+              // if (DEBUG) {
+              //   System.out.println("  target is same as current but term doesn't exist");
+              // }
+            }
+            //validIndexPrefix = currentFrame.depth;
+            //term.length = target.length;
+            //return termExists;
+          }
+
+        } else {
+
+          targetBeforeCurrentLength = -1;
+          arc = index.getFirstArc(arcs[0]);
+
+          // Empty string prefix must have an output (block) in the index!
+          assert arc.isFinal();
+          assert arc.output != null;
+
+          // if (DEBUG) {
+          //   System.out.println("    no seek state; push root frame");
+          // }
+
+          output = arc.output;
+
+          currentFrame = staticFrame;
+
+          //term.length = 0;
+          targetUpto = 0;
+          currentFrame = pushFrame(arc, fstOutputs.add(output, arc.nextFinalOutput), 0);
+        }
+
+        // if (DEBUG) {
+        //   System.out.println("  start index loop targetUpto=" + targetUpto + " output=" + output + " currentFrame.ord=" + currentFrame.ord + " targetBeforeCurrentLength=" + targetBeforeCurrentLength);
+        // }
+
+        while (targetUpto < target.length) {
+
+          final int targetLabel = target.bytes[target.offset + targetUpto] & 0xFF;
+
+          final FST.Arc<BytesRef> nextArc = index.findTargetArc(targetLabel, arc, getArc(1+targetUpto), fstReader);
+
+          if (nextArc == null) {
+
+            // Index is exhausted
+            // if (DEBUG) {
+            //   System.out.println("    index: index exhausted label=" + ((char) targetLabel) + " " + toHex(targetLabel));
+            // }
+            
+            validIndexPrefix = currentFrame.prefix;
+            //validIndexPrefix = targetUpto;
+
+            currentFrame.scanToFloorFrame(target);
+
+            if (!currentFrame.hasTerms) {
+              termExists = false;
+              term.bytes[targetUpto] = (byte) targetLabel;
+              term.length = 1+targetUpto;
+              // if (DEBUG) {
+              //   System.out.println("  FAST NOT_FOUND term=" + brToString(term));
+              // }
+              return false;
+            }
+
+            currentFrame.loadBlock();
+
+            final SeekStatus result = currentFrame.scanToTerm(target, true);            
+            if (result == SeekStatus.FOUND) {
+              // if (DEBUG) {
+              //   System.out.println("  return FOUND term=" + term.utf8ToString() + " " + term);
+              // }
+              return true;
+            } else {
+              // if (DEBUG) {
+              //   System.out.println("  got " + result + "; return NOT_FOUND term=" + brToString(term));
+              // }
+              return false;
+            }
+          } else {
+            // Follow this arc
+            arc = nextArc;
+            term.bytes[targetUpto] = (byte) targetLabel;
+            // Aggregate output as we go:
+            assert arc.output != null;
+            if (arc.output != NO_OUTPUT) {
+              output = fstOutputs.add(output, arc.output);
+            }
+
+            // if (DEBUG) {
+            //   System.out.println("    index: follow label=" + toHex(target.bytes[target.offset + targetUpto]&0xff) + " arc.output=" + arc.output + " arc.nfo=" + arc.nextFinalOutput);
+            // }
+            targetUpto++;
+
+            if (arc.isFinal()) {
+              //if (DEBUG) System.out.println("    arc is final!");
+              currentFrame = pushFrame(arc, fstOutputs.add(output, arc.nextFinalOutput), targetUpto);
+              //if (DEBUG) System.out.println("    curFrame.ord=" + currentFrame.ord + " hasTerms=" + currentFrame.hasTerms);
+            }
+          }
+        }
+
+        //validIndexPrefix = targetUpto;
+        validIndexPrefix = currentFrame.prefix;
+
+        currentFrame.scanToFloorFrame(target);
+
+        // Target term is entirely contained in the index:
+        if (!currentFrame.hasTerms) {
+          termExists = false;
+          term.length = targetUpto;
+          // if (DEBUG) {
+          //   System.out.println("  FAST NOT_FOUND term=" + brToString(term));
+          // }
+          return false;
+        }
+
+        currentFrame.loadBlock();
+
+        final SeekStatus result = currentFrame.scanToTerm(target, true);            
+        if (result == SeekStatus.FOUND) {
+          // if (DEBUG) {
+          //   System.out.println("  return FOUND term=" + term.utf8ToString() + " " + term);
+          // }
+          return true;
+        } else {
+          // if (DEBUG) {
+          //   System.out.println("  got result " + result + "; return NOT_FOUND term=" + term.utf8ToString());
+          // }
+
+          return false;
+        }
+      }
+
+      @Override
+      public SeekStatus seekCeil(final BytesRef target, final boolean useCache) throws IOException {
+        if (index == null) {
+          throw new IllegalStateException("terms index was not loaded");
+        }
+   
+        if (term.bytes.length <= target.length) {
+          term.bytes = ArrayUtil.grow(term.bytes, 1+target.length);
+        }
+
+        assert clearEOF();
+
+        //if (DEBUG) {
+        //System.out.println("\nBTTR.seekCeil seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + brToString(term) + " (exists?=" + termExists + ") validIndexPrefix=  " + validIndexPrefix);
+        //printSeekState();
+        //}
+
+        FST.Arc<BytesRef> arc;
+        int targetUpto;
+        BytesRef output;
+
+        targetBeforeCurrentLength = currentFrame.ord;
+
+        if (currentFrame != staticFrame) {
+
+          // We are already seek'd; find the common
+          // prefix of new seek term vs current term and
+          // re-use the corresponding seek state.  For
+          // example, if app first seeks to foobar, then
+          // seeks to foobaz, we can re-use the seek state
+          // for the first 5 bytes.
+
+          //if (DEBUG) {
+          //System.out.println("  re-use current seek state validIndexPrefix=" + validIndexPrefix);
+          //}
+
+          arc = arcs[0];
+          assert arc.isFinal();
+          output = arc.output;
+          targetUpto = 0;
+          
+          Frame lastFrame = stack[0];
+          assert validIndexPrefix <= term.length;
+
+          final int targetLimit = Math.min(target.length, validIndexPrefix);
+
+          int cmp = 0;
+
+          // TOOD: we should write our vLong backwards (MSB
+          // first) to get better sharing from the FST
+
+          // First compare up to valid seek frames:
+          while (targetUpto < targetLimit) {
+            cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+            //if (DEBUG) {
+            //System.out.println("    cycle targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")"   + " arc.output=" + arc.output + " output=" + output);
+            //}
+            if (cmp != 0) {
+              break;
+            }
+            arc = arcs[1+targetUpto];
+            assert arc.label == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
+            // TOOD: we could save the outputs in local
+            // byte[][] instead of making new objs ever
+            // seek; but, often the FST doesn't have any
+            // shared bytes (but this could change if we
+            // reverse vLong byte order)
+            if (arc.output != NO_OUTPUT) {
+              output = fstOutputs.add(output, arc.output);
+            }
+            if (arc.isFinal()) {
+              lastFrame = stack[1+lastFrame.ord];
+            }
+            targetUpto++;
+          }
+
+
+          if (cmp == 0) {
+            final int targetUptoMid = targetUpto;
+            // Second compare the rest of the term, but
+            // don't save arc/output/frame:
+            final int targetLimit2 = Math.min(target.length, term.length);
+            while (targetUpto < targetLimit2) {
+              cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+              //if (DEBUG) {
+              //System.out.println("    cycle2 targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")");
+              //}
+              if (cmp != 0) {
+                break;
+              }
+              targetUpto++;
+            }
+
+            if (cmp == 0) {
+              cmp = term.length - target.length;
+            }
+            targetUpto = targetUptoMid;
+          }
+
+          if (cmp < 0) {
+            // Common case: target term is after current
+            // term, ie, app is seeking multiple terms
+            // in sorted order
+            //if (DEBUG) {
+            //System.out.println("  target is after current (shares prefixLen=" + targetUpto + "); clear frame.scanned ord=" + lastFrame.ord);
+            //}
+            currentFrame = lastFrame;
+
+          } else if (cmp > 0) {
+            // Uncommon case: target term
+            // is before current term; this means we can
+            // keep the currentFrame but we must rewind it
+            // (so we scan from the start)
+            targetBeforeCurrentLength = 0;
+            //if (DEBUG) {
+            //System.out.println("  target is before current (shares prefixLen=" + targetUpto + "); rewind frame ord=" + lastFrame.ord);
+            //}
+            currentFrame = lastFrame;
+            currentFrame.rewind();
+          } else {
+            // Target is exactly the same as current term
+            assert term.length == target.length;
+            if (termExists) {
+              //if (DEBUG) {
+              //System.out.println("  target is same as current; return FOUND");
+              //}
+              return SeekStatus.FOUND;
+            } else {
+              //if (DEBUG) {
+              //System.out.println("  target is same as current but term doesn't exist");
+              //}
+            }
+          }
+
+        } else {
+
+          targetBeforeCurrentLength = -1;
+          arc = index.getFirstArc(arcs[0]);
+
+          // Empty string prefix must have an output (block) in the index!
+          assert arc.isFinal();
+          assert arc.output != null;
+
+          //if (DEBUG) {
+          //System.out.println("    no seek state; push root frame");
+          //}
+
+          output = arc.output;
+
+          currentFrame = staticFrame;
+
+          //term.length = 0;
+          targetUpto = 0;
+          currentFrame = pushFrame(arc, fstOutputs.add(output, arc.nextFinalOutput), 0);
+        }
+
+        //if (DEBUG) {
+        //System.out.println("  start index loop targetUpto=" + targetUpto + " output=" + output + " currentFrame.ord+1=" + currentFrame.ord + " targetBeforeCurrentLength=" + targetBeforeCurrentLength);
+        //}
+
+        while (targetUpto < target.length) {
+
+          final int targetLabel = target.bytes[target.offset + targetUpto] & 0xFF;
+
+          final FST.Arc<BytesRef> nextArc = index.findTargetArc(targetLabel, arc, getArc(1+targetUpto), fstReader);
+
+          if (nextArc == null) {
+
+            // Index is exhausted
+            // if (DEBUG) {
+            //   System.out.println("    index: index exhausted label=" + ((char) targetLabel) + " " + toHex(targetLabel));
+            // }
+            
+            validIndexPrefix = currentFrame.prefix;
+            //validIndexPrefix = targetUpto;
+
+            currentFrame.scanToFloorFrame(target);
+
+            currentFrame.loadBlock();
+
+            final SeekStatus result = currentFrame.scanToTerm(target, false);
+            if (result == SeekStatus.END) {
+              term.copyBytes(target);
+              termExists = false;
+
+              if (next() != null) {
+                //if (DEBUG) {
+                //System.out.println("  return NOT_FOUND term=" + brToString(term) + " " + term);
+                //}
+                return SeekStatus.NOT_FOUND;
+              } else {
+                //if (DEBUG) {
+                //System.out.println("  return END");
+                //}
+                return SeekStatus.END;
+              }
+            } else {
+              //if (DEBUG) {
+              //System.out.println("  return " + result + " term=" + brToString(term) + " " + term);
+              //}
+              return result;
+            }
+          } else {
+            // Follow this arc
+            term.bytes[targetUpto] = (byte) targetLabel;
+            arc = nextArc;
+            // Aggregate output as we go:
+            assert arc.output != null;
+            if (arc.output != NO_OUTPUT) {
+              output = fstOutputs.add(output, arc.output);
+            }
+
+            //if (DEBUG) {
+            //System.out.println("    index: follow label=" + toHex(target.bytes[target.offset + targetUpto]&0xff) + " arc.output=" + arc.output + " arc.nfo=" + arc.nextFinalOutput);
+            //}
+            targetUpto++;
+
+            if (arc.isFinal()) {
+              //if (DEBUG) System.out.println("    arc is final!");
+              currentFrame = pushFrame(arc, fstOutputs.add(output, arc.nextFinalOutput), targetUpto);
+              //if (DEBUG) System.out.println("    curFrame.ord=" + currentFrame.ord + " hasTerms=" + currentFrame.hasTerms);
+            }
+          }
+        }
+
+        //validIndexPrefix = targetUpto;
+        validIndexPrefix = currentFrame.prefix;
+
+        currentFrame.scanToFloorFrame(target);
+
+        currentFrame.loadBlock();
+
+        final SeekStatus result = currentFrame.scanToTerm(target, false);
+
+        if (result == SeekStatus.END) {
+          term.copyBytes(target);
+          termExists = false;
+          if (next() != null) {
+            //if (DEBUG) {
+            //System.out.println("  return NOT_FOUND term=" + term.utf8ToString() + " " + term);
+            //}
+            return SeekStatus.NOT_FOUND;
+          } else {
+            //if (DEBUG) {
+            //System.out.println("  return END");
+            //}
+            return SeekStatus.END;
+          }
+        } else {
+          return result;
+        }
+      }
+
+      @SuppressWarnings("unused")
+      private void printSeekState(PrintStream out) throws IOException {
+        if (currentFrame == staticFrame) {
+          out.println("  no prior seek");
+        } else {
+          out.println("  prior seek state:");
+          int ord = 0;
+          boolean isSeekFrame = true;
+          while(true) {
+            Frame f = getFrame(ord);
+            assert f != null;
+            final BytesRef prefix = new BytesRef(term.bytes, 0, f.prefix);
+            if (f.nextEnt == -1) {
+              out.println("    frame " + (isSeekFrame ? "(seek)" : "(next)") + " ord=" + ord + " fp=" + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" + f.prefix + " prefix=" + prefix + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp<<BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0)) + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd());
+            } else {
+              out.println("    frame " + (isSeekFrame ? "(seek, loaded)" : "(next, loaded)") + " ord=" + ord + " fp=" + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" + f.prefix + " prefix=" + prefix + " nextEnt=" + f.nextEnt + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp<<BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0)) + " lastSubFP=" + f.lastSubFP + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd());
+            }
+            if (index != null) {
+              assert !isSeekFrame || f.arc != null: "isSeekFrame=" + isSeekFrame + " f.arc=" + f.arc;
+              if (f.prefix > 0 && isSeekFrame && f.arc.label != (term.bytes[f.prefix-1]&0xFF)) {
+                out.println("      broken seek state: arc.label=" + (char) f.arc.label + " vs term byte=" + (char) (term.bytes[f.prefix-1]&0xFF));
+                throw new RuntimeException("seek state is broken");
+              }
+              BytesRef output = Util.get(index, prefix);
+              if (output == null) {
+                out.println("      broken seek state: prefix is not final in index");
+                throw new RuntimeException("seek state is broken");
+              } else if (isSeekFrame && !f.isFloor) {
+                final ByteArrayDataInput reader = new ByteArrayDataInput(output.bytes, output.offset, output.length);
+                final long codeOrig = reader.readVLong();
+                final long code = (f.fp << BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) | (f.hasTerms ? BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) | (f.isFloor ? BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0);
+                if (codeOrig != code) {
+                  out.println("      broken seek state: output code=" + codeOrig + " doesn't match frame code=" + code);
+                  throw new RuntimeException("seek state is broken");
+                }
+              }
+            }
+            if (f == currentFrame) {
+              break;
+            }
+            if (f.prefix == validIndexPrefix) {
+              isSeekFrame = false;
+            }
+            ord++;
+          }
+        }
+      }
+
+      /* Decodes only the term bytes of the next term.  If caller then asks for
+         metadata, ie docFreq, totalTermFreq or pulls a D/&PEnum, we then (lazily)
+         decode all metadata up to the current term. */
+      @Override
+      public BytesRef next() throws IOException {
+
+        if (in == null) {
+          // Fresh TermsEnum; seek to first term:
+          final FST.Arc<BytesRef> arc;
+          if (index != null) {
+            arc = index.getFirstArc(arcs[0]);
+            // Empty string prefix must have an output in the index!
+            assert arc.isFinal();
+          } else {
+            arc = null;
+          }
+          currentFrame = pushFrame(arc, rootCode, 0);
+          currentFrame.loadBlock();
+        }
+
+        targetBeforeCurrentLength = currentFrame.ord;
+
+        assert !eof;
+        //if (DEBUG) {
+        //System.out.println("\nBTTR.next seg=" + segment + " term=" + brToString(term) + " termExists?=" + termExists + " field=" + fieldInfo.name + " termBlockOrd=" + currentFrame.state.termBlockOrd + " validIndexPrefix=" + validIndexPrefix);
+        //printSeekState();
+        //}
+
+        if (currentFrame == staticFrame) {
+          // If seek was previously called and the term was
+          // cached, or seek(TermState) was called, usually
+          // caller is just going to pull a D/&PEnum or get
+          // docFreq, etc.  But, if they then call next(),
+          // this method catches up all internal state so next()
+          // works properly:
+          //if (DEBUG) System.out.println("  re-seek to pending term=" + term.utf8ToString() + " " + term);
+          final boolean result = seekExact(term, false);
+          assert result;
+        }
+
+        // Pop finished blocks
+        while (currentFrame.nextEnt == currentFrame.entCount) {
+          if (!currentFrame.isLastInFloor) {
+            currentFrame.loadNextFloorBlock();
+          } else {
+            //if (DEBUG) System.out.println("  pop frame");
+            if (currentFrame.ord == 0) {
+              //if (DEBUG) System.out.println("  return null");
+              assert setEOF();
+              term.length = 0;
+              validIndexPrefix = 0;
+              currentFrame.rewind();
+              termExists = false;
+              return null;
+            }
+            final long lastFP = currentFrame.fpOrig;
+            currentFrame = stack[currentFrame.ord-1];
+
+            if (currentFrame.nextEnt == -1 || currentFrame.lastSubFP != lastFP) {
+              // We popped into a frame that's not loaded
+              // yet or not scan'd to the right entry
+              currentFrame.scanToFloorFrame(term);
+              currentFrame.loadBlock();
+              currentFrame.scanToSubBlock(lastFP);
+            }
+
+            // Note that the seek state (last seek) has been
+            // invalidated beyond this depth
+            validIndexPrefix = Math.min(validIndexPrefix, currentFrame.prefix);
+            //if (DEBUG) {
+            //System.out.println("  reset validIndexPrefix=" + validIndexPrefix);
+            //}
+          }
+        }
+
+        while(true) {
+          if (currentFrame.next()) {
+            // Push to new block:
+            //if (DEBUG) System.out.println("  push frame");
+            currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length);
+            // This is a "next" frame -- even if it's
+            // floor'd we must pretend it isn't so we don't
+            // try to scan to the right floor frame:
+            currentFrame.isFloor = false;
+            //currentFrame.hasTerms = true;
+            currentFrame.loadBlock();
+          } else {
+            //if (DEBUG) System.out.println("  return term=" + term.utf8ToString() + " " + term + " currentFrame.ord=" + currentFrame.ord);
+            return term;
+          }
+        }
+      }
+
+      @Override
+      public BytesRef term() {
+        assert !eof;
+        return term;
+      }
+
+      @Override
+      public int docFreq() throws IOException {
+        assert !eof;
+        //if (DEBUG) System.out.println("BTR.docFreq");
+        currentFrame.decodeMetaData();
+        //if (DEBUG) System.out.println("  return " + currentFrame.state.docFreq);
+        return currentFrame.state.docFreq;
+      }
+
+      @Override
+      public long totalTermFreq() throws IOException {
+        assert !eof;
+        currentFrame.decodeMetaData();
+        return currentFrame.state.totalTermFreq;
+      }
+
+      @Override
+      public DocsEnum docs(Bits skipDocs, DocsEnum reuse, int flags) throws IOException {
+        assert !eof;
+        //if (DEBUG) {
+        //System.out.println("BTTR.docs seg=" + segment);
+      

<TRUNCATED>

[13/13] git commit: Removing thread local variables.

Posted by am...@apache.org.
Removing thread local variables.


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

Branch: refs/heads/master
Commit: 991fb0435a373283b2fb05ef29c711c916f0aec9
Parents: 98359a4
Author: Aaron McCurry <am...@gmail.com>
Authored: Mon Aug 29 21:49:31 2016 -0400
Committer: Aaron McCurry <am...@gmail.com>
Committed: Mon Aug 29 21:49:31 2016 -0400

----------------------------------------------------------------------
 .../apache/blur/command/BaseCommandManager.java |   15 +-
 .../MasterBasedDistributedLayoutFactory.java    |    3 +-
 .../java/org/apache/blur/utils/BlurUtil.java    |    2 +-
 .../blur/command/ShardCommandManagerTest.java   |    2 +-
 .../org/apache/blur/hive/BlurSerializer.java    |    7 +-
 .../blur/mapreduce/lib/BlurOutputFormat.java    |    5 +-
 .../mapreduce/lib/GenericBlurRecordWriter.java  |    3 +-
 .../analysis/type/DateFieldTypeDefinition.java  |    5 +-
 .../java/org/apache/blur/index/ExitObject.java  |    4 +-
 .../blur/lucene/codec/CachedDecompressor.java   |    3 +-
 .../lucene/codec/DiskDocValuesProducer.java     |    5 +-
 .../blur/store/blockcache/BlockCache.java       |    4 +-
 .../blur/store/blockcache_v2/MeterWrapper.java  |    3 +-
 .../blur/store/hdfs_v2/StoreDirection.java      |    4 +-
 .../lucene/codecs/BlockTreeTermsReader.java     | 2939 ++++++++++++++++++
 .../apache/blur/thrift/BlurClientManager.java   |    3 +-
 .../apache/blur/thrift/sasl/TSaslTransport.java |    3 +-
 .../main/java/org/apache/blur/trace/Trace.java  |    5 +-
 .../java/org/apache/blur/user/UserContext.java  |    4 +-
 .../java/org/apache/blur/utils/ThreadValue.java |   56 +
 .../apache/blur/utils/BlurConstantsTest.java    |    2 +-
 21 files changed, 3048 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-core/src/main/java/org/apache/blur/command/BaseCommandManager.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/command/BaseCommandManager.java b/blur-core/src/main/java/org/apache/blur/command/BaseCommandManager.java
index be92e34..b91e378 100644
--- a/blur-core/src/main/java/org/apache/blur/command/BaseCommandManager.java
+++ b/blur-core/src/main/java/org/apache/blur/command/BaseCommandManager.java
@@ -390,7 +390,9 @@ public abstract class BaseCommandManager implements Closeable {
       throw new IOException("Execution instance id [" + instanceExecutionId + "] did not find any executing commands.");
     }
     try {
-      return future.get(_connectionTimeout, TimeUnit.MILLISECONDS);
+      Response response = future.get(_connectionTimeout, TimeUnit.MILLISECONDS);
+      _driverRunningMap.remove(instanceExecutionId);
+      return response;
     } catch (CancellationException e) {
       throw new IOException(e);
     } catch (InterruptedException e) {
@@ -411,7 +413,9 @@ public abstract class BaseCommandManager implements Closeable {
     _driverRunningMap.put(instanceExecutionId, new ResponseFuture<Response>(_runningCacheTombstoneTime, future,
         commandExecuting, originalCommandStatusObject, running));
     try {
-      return future.get(_connectionTimeout, TimeUnit.MILLISECONDS);
+      Response response = future.get(_connectionTimeout, TimeUnit.MILLISECONDS);
+      _driverRunningMap.remove(instanceExecutionId);
+      return response;
     } catch (CancellationException e) {
       throw new IOException(e);
     } catch (InterruptedException e) {
@@ -447,9 +451,10 @@ public abstract class BaseCommandManager implements Closeable {
       CommandStatus originalCommandStatusObject, AtomicBoolean running) {
     Future<T> future = _executorServiceWorker.submit(callable);
     Long instanceExecutionId = getInstanceExecutionId();
-    _workerRunningMap.put(instanceExecutionId, new ResponseFuture<T>(_runningCacheTombstoneTime, future,
-        commandExecuting, originalCommandStatusObject, running));
-    return future;
+    ResponseFuture<T> responseFuture = new ResponseFuture<T>(_runningCacheTombstoneTime, future,
+        commandExecuting, originalCommandStatusObject, running);
+    _workerRunningMap.put(instanceExecutionId, responseFuture);
+    return responseFuture;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-core/src/main/java/org/apache/blur/manager/indexserver/MasterBasedDistributedLayoutFactory.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/manager/indexserver/MasterBasedDistributedLayoutFactory.java b/blur-core/src/main/java/org/apache/blur/manager/indexserver/MasterBasedDistributedLayoutFactory.java
index 6fff994..1a2e916 100644
--- a/blur-core/src/main/java/org/apache/blur/manager/indexserver/MasterBasedDistributedLayoutFactory.java
+++ b/blur-core/src/main/java/org/apache/blur/manager/indexserver/MasterBasedDistributedLayoutFactory.java
@@ -38,6 +38,7 @@ import java.util.concurrent.ConcurrentMap;
 
 import org.apache.blur.log.Log;
 import org.apache.blur.log.LogFactory;
+import org.apache.blur.utils.ThreadValue;
 import org.apache.blur.zookeeper.ZkUtils;
 import org.apache.blur.zookeeper.ZooKeeperLockManager;
 import org.apache.blur.zookeeper.ZookeeperPathConstants;
@@ -57,7 +58,7 @@ public class MasterBasedDistributedLayoutFactory implements DistributedLayoutFac
   private final String _storagePath;
   private final ZooKeeperLockManager _zooKeeperLockManager;
   private final String _locksStoragePath;
-  private final ThreadLocal<Random> _random = new ThreadLocal<Random>() {
+  private final ThreadValue<Random> _random = new ThreadValue<Random>() {
     @Override
     protected Random initialValue() {
       return new Random();

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-core/src/main/java/org/apache/blur/utils/BlurUtil.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/utils/BlurUtil.java b/blur-core/src/main/java/org/apache/blur/utils/BlurUtil.java
index 9406e66..5c0022c 100644
--- a/blur-core/src/main/java/org/apache/blur/utils/BlurUtil.java
+++ b/blur-core/src/main/java/org/apache/blur/utils/BlurUtil.java
@@ -221,7 +221,7 @@ public class BlurUtil {
     final String prefix = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
     InvocationHandler handler = new InvocationHandler() {
       private final AtomicLong _requestCounter = new AtomicLong();
-      private ThreadLocal<LoggerArgsState> _loggerArgsState = new ThreadLocal<LoggerArgsState>() {
+      private ThreadValue<LoggerArgsState> _loggerArgsState = new ThreadValue<LoggerArgsState>() {
         @Override
         protected LoggerArgsState initialValue() {
           return new LoggerArgsState(1024);

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-core/src/test/java/org/apache/blur/command/ShardCommandManagerTest.java
----------------------------------------------------------------------
diff --git a/blur-core/src/test/java/org/apache/blur/command/ShardCommandManagerTest.java b/blur-core/src/test/java/org/apache/blur/command/ShardCommandManagerTest.java
index f5fe4ea..0fb966a 100644
--- a/blur-core/src/test/java/org/apache/blur/command/ShardCommandManagerTest.java
+++ b/blur-core/src/test/java/org/apache/blur/command/ShardCommandManagerTest.java
@@ -96,7 +96,7 @@ public class ShardCommandManagerTest {
   @Test
   public void testGetCommands() {
     Map<String, BigInteger> commands = _manager.getCommands();
-    assertEquals(4, commands.size());
+    assertEquals(5, commands.size());
     assertTrue(commands.containsKey("wait"));
     assertTrue(commands.containsKey("error"));
     assertTrue(commands.containsKey("RunSlowForTesting"));

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-hive/src/main/java/org/apache/blur/hive/BlurSerializer.java
----------------------------------------------------------------------
diff --git a/blur-hive/src/main/java/org/apache/blur/hive/BlurSerializer.java b/blur-hive/src/main/java/org/apache/blur/hive/BlurSerializer.java
index 460c7f9..bd5ae74 100644
--- a/blur-hive/src/main/java/org/apache/blur/hive/BlurSerializer.java
+++ b/blur-hive/src/main/java/org/apache/blur/hive/BlurSerializer.java
@@ -26,6 +26,7 @@ import java.util.Set;
 
 import org.apache.blur.mapreduce.lib.BlurRecord;
 import org.apache.blur.thrift.generated.ColumnDefinition;
+import org.apache.blur.utils.ThreadValue;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -39,7 +40,7 @@ public class BlurSerializer {
 
   private static final String DATE_FORMAT = "dateFormat";
   private static final String DATE = "date";
-  private Map<String, ThreadLocal<SimpleDateFormat>> _dateFormat = new HashMap<String, ThreadLocal<SimpleDateFormat>>();
+  private Map<String, ThreadValue<SimpleDateFormat>> _dateFormat = new HashMap<String, ThreadValue<SimpleDateFormat>>();
   private BlurColumnNameResolver _columnNameResolver;
 
   public BlurSerializer(Map<String, ColumnDefinition> colDefs, BlurColumnNameResolver columnNameResolver) {
@@ -52,7 +53,7 @@ public class BlurSerializer {
       if (fieldType.equals(DATE)) {
         Map<String, String> properties = columnDefinition.getProperties();
         final String dateFormat = properties.get(DATE_FORMAT);
-        ThreadLocal<SimpleDateFormat> threadLocal = new ThreadLocal<SimpleDateFormat>() {
+        ThreadValue<SimpleDateFormat> threadLocal = new ThreadValue<SimpleDateFormat>() {
           @Override
           protected SimpleDateFormat initialValue() {
             return new SimpleDateFormat(dateFormat);
@@ -184,7 +185,7 @@ public class BlurSerializer {
   }
 
   private SimpleDateFormat getSimpleDateFormat(String columnName) throws SerDeException {
-    ThreadLocal<SimpleDateFormat> threadLocal = _dateFormat.get(columnName);
+    ThreadValue<SimpleDateFormat> threadLocal = _dateFormat.get(columnName);
     if (threadLocal == null) {
       throw new SerDeException("Date format missing for column [" + columnName + "]");
     }

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/BlurOutputFormat.java
----------------------------------------------------------------------
diff --git a/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/BlurOutputFormat.java b/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/BlurOutputFormat.java
index 52900cf..3e72772 100644
--- a/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/BlurOutputFormat.java
+++ b/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/BlurOutputFormat.java
@@ -27,6 +27,7 @@ import org.apache.blur.thirdparty.thrift_0_9_0.transport.TIOStreamTransport;
 import org.apache.blur.thrift.BlurClient;
 import org.apache.blur.thrift.generated.Blur.Iface;
 import org.apache.blur.thrift.generated.TableDescriptor;
+import org.apache.blur.utils.ThreadValue;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -77,8 +78,8 @@ public class BlurOutputFormat extends OutputFormat<Text, BlurMutate> {
   public static final String BLUR_OUTPUT_PATH = "blur.output.path";
 
   private static final String MAPRED_OUTPUT_COMMITTER_CLASS = "mapred.output.committer.class";
-  private static ThreadLocal<Progressable> _progressable = new ThreadLocal<Progressable>();
-  private static ThreadLocal<GetCounter> _getCounter = new ThreadLocal<GetCounter>();
+  private static ThreadValue<Progressable> _progressable = new ThreadValue<Progressable>();
+  private static ThreadValue<GetCounter> _getCounter = new ThreadValue<GetCounter>();
 
   public static void setProgressable(Progressable progressable) {
     _progressable.set(progressable);

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/GenericBlurRecordWriter.java
----------------------------------------------------------------------
diff --git a/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/GenericBlurRecordWriter.java b/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/GenericBlurRecordWriter.java
index 8828f85..fba819e 100644
--- a/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/GenericBlurRecordWriter.java
+++ b/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/GenericBlurRecordWriter.java
@@ -39,6 +39,7 @@ import org.apache.blur.thrift.generated.TableDescriptor;
 import org.apache.blur.utils.BlurConstants;
 import org.apache.blur.utils.RowDocumentUtil;
 import org.apache.blur.utils.ShardUtil;
+import org.apache.blur.utils.ThreadValue;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -278,7 +279,7 @@ public class GenericBlurRecordWriter {
     return record;
   }
 
-  private static ThreadLocal<AtomicBoolean> _existingRow = new ThreadLocal<AtomicBoolean>() {
+  private static ThreadValue<AtomicBoolean> _existingRow = new ThreadValue<AtomicBoolean>() {
     @Override
     protected AtomicBoolean initialValue() {
       return new AtomicBoolean();

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-query/src/main/java/org/apache/blur/analysis/type/DateFieldTypeDefinition.java
----------------------------------------------------------------------
diff --git a/blur-query/src/main/java/org/apache/blur/analysis/type/DateFieldTypeDefinition.java b/blur-query/src/main/java/org/apache/blur/analysis/type/DateFieldTypeDefinition.java
index 695cdc3..0e35125 100644
--- a/blur-query/src/main/java/org/apache/blur/analysis/type/DateFieldTypeDefinition.java
+++ b/blur-query/src/main/java/org/apache/blur/analysis/type/DateFieldTypeDefinition.java
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.blur.thrift.generated.Column;
+import org.apache.blur.utils.ThreadValue;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
@@ -44,7 +45,7 @@ public class DateFieldTypeDefinition extends NumericFieldTypeDefinition {
   public static final String DATE_FORMAT = "dateFormat";
   public static final String NAME = "date";
   private FieldType _typeNotStored;
-  private ThreadLocal<SimpleDateFormat> _simpleDateFormat;
+  private ThreadValue<SimpleDateFormat> _simpleDateFormat;
   private TimeUnit _timeUnit = TimeUnit.SECONDS;
 
   @Override
@@ -62,7 +63,7 @@ public class DateFieldTypeDefinition extends NumericFieldTypeDefinition {
     if (timeUnitStr != null) {
       _timeUnit = TimeUnit.valueOf(timeUnitStr.trim().toUpperCase());
     }
-    _simpleDateFormat = new ThreadLocal<SimpleDateFormat>() {
+    _simpleDateFormat = new ThreadValue<SimpleDateFormat>() {
       @Override
       protected SimpleDateFormat initialValue() {
         return new SimpleDateFormat(dateFormat);

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-store/src/main/java/org/apache/blur/index/ExitObject.java
----------------------------------------------------------------------
diff --git a/blur-store/src/main/java/org/apache/blur/index/ExitObject.java b/blur-store/src/main/java/org/apache/blur/index/ExitObject.java
index 92cd219..d2ad352 100644
--- a/blur-store/src/main/java/org/apache/blur/index/ExitObject.java
+++ b/blur-store/src/main/java/org/apache/blur/index/ExitObject.java
@@ -18,9 +18,11 @@ package org.apache.blur.index;
 
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.blur.utils.ThreadValue;
+
 public class ExitObject {
 
-  private final ThreadLocal<AtomicBoolean> _running = new ThreadLocal<AtomicBoolean>() {
+  private final ThreadValue<AtomicBoolean> _running = new ThreadValue<AtomicBoolean>() {
     @Override
     protected AtomicBoolean initialValue() {
       return new AtomicBoolean(true);

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-store/src/main/java/org/apache/blur/lucene/codec/CachedDecompressor.java
----------------------------------------------------------------------
diff --git a/blur-store/src/main/java/org/apache/blur/lucene/codec/CachedDecompressor.java b/blur-store/src/main/java/org/apache/blur/lucene/codec/CachedDecompressor.java
index 18c4cc1..1a438da 100644
--- a/blur-store/src/main/java/org/apache/blur/lucene/codec/CachedDecompressor.java
+++ b/blur-store/src/main/java/org/apache/blur/lucene/codec/CachedDecompressor.java
@@ -18,6 +18,7 @@ package org.apache.blur.lucene.codec;
 
 import java.io.IOException;
 
+import org.apache.blur.utils.ThreadValue;
 import org.apache.lucene.codecs.compressing.Decompressor;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.IndexInput;
@@ -27,7 +28,7 @@ import org.apache.lucene.util.BytesRef;
 public class CachedDecompressor extends Decompressor {
 
   private final Decompressor _decompressor;
-  private final ThreadLocal<Entry> _entry = new ThreadLocal<Entry>() {
+  private final ThreadValue<Entry> _entry = new ThreadValue<Entry>() {
     @Override
     protected Entry initialValue() {
       return new Entry();

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-store/src/main/java/org/apache/blur/lucene/codec/DiskDocValuesProducer.java
----------------------------------------------------------------------
diff --git a/blur-store/src/main/java/org/apache/blur/lucene/codec/DiskDocValuesProducer.java b/blur-store/src/main/java/org/apache/blur/lucene/codec/DiskDocValuesProducer.java
index fd617b6..ff51ddd 100644
--- a/blur-store/src/main/java/org/apache/blur/lucene/codec/DiskDocValuesProducer.java
+++ b/blur-store/src/main/java/org/apache/blur/lucene/codec/DiskDocValuesProducer.java
@@ -23,6 +23,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.blur.trace.Trace;
 import org.apache.blur.trace.Tracer;
+import org.apache.blur.utils.ThreadValue;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.index.BinaryDocValues;
@@ -236,7 +237,7 @@ class DiskDocValuesProducer extends DocValuesProducer {
 
     return new LongBinaryDocValues() {
 
-      private final ThreadLocal<IndexInput> in = new ThreadLocal<IndexInput>() {
+      private final ThreadValue<IndexInput> in = new ThreadValue<IndexInput>() {
         @Override
         protected IndexInput initialValue() {
           return data.clone();
@@ -277,7 +278,7 @@ class DiskDocValuesProducer extends DocValuesProducer {
     }
     return new LongBinaryDocValues() {
       
-      private final ThreadLocal<IndexInput> _input = new ThreadLocal<IndexInput>() {
+      private final ThreadValue<IndexInput> _input = new ThreadValue<IndexInput>() {
         @Override
         protected IndexInput initialValue() {
           return data.clone();

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-store/src/main/java/org/apache/blur/store/blockcache/BlockCache.java
----------------------------------------------------------------------
diff --git a/blur-store/src/main/java/org/apache/blur/store/blockcache/BlockCache.java b/blur-store/src/main/java/org/apache/blur/store/blockcache/BlockCache.java
index 4597ffa..1c6d9a1 100644
--- a/blur-store/src/main/java/org/apache/blur/store/blockcache/BlockCache.java
+++ b/blur-store/src/main/java/org/apache/blur/store/blockcache/BlockCache.java
@@ -33,6 +33,8 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.blur.utils.ThreadValue;
+
 import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap;
 import com.googlecode.concurrentlinkedhashmap.EvictionListener;
 import com.yammer.metrics.Metrics;
@@ -71,7 +73,7 @@ public class BlockCache implements Closeable {
   private final Meter evictions;
   private final int _numberOfSlabs;
   private final boolean _directAllocation;
-  private final ThreadLocal<ByteBuffer[]> _threadLocalSlabs = new ThreadLocal<ByteBuffer[]>() {
+  private final ThreadValue<ByteBuffer[]> _threadLocalSlabs = new ThreadValue<ByteBuffer[]>() {
     @Override
     protected ByteBuffer[] initialValue() {
       return new ByteBuffer[_numberOfSlabs];

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/MeterWrapper.java
----------------------------------------------------------------------
diff --git a/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/MeterWrapper.java b/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/MeterWrapper.java
index 269245a..5a7102d 100644
--- a/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/MeterWrapper.java
+++ b/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/MeterWrapper.java
@@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.blur.log.Log;
 import org.apache.blur.log.LogFactory;
+import org.apache.blur.utils.ThreadValue;
 
 import com.yammer.metrics.core.Meter;
 
@@ -75,7 +76,7 @@ public abstract class MeterWrapper implements Closeable {
 
   public static MeterWrapper wrap(final SimpleMeter meter) {
     final String id = UUID.randomUUID().toString();
-    final ThreadLocal<AtomicLong> countThreadLocal = new ThreadLocal<AtomicLong>() {
+    final ThreadValue<AtomicLong> countThreadLocal = new ThreadValue<AtomicLong>() {
       @Override
       protected AtomicLong initialValue() {
         AtomicLong counter = new AtomicLong();

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-store/src/main/java/org/apache/blur/store/hdfs_v2/StoreDirection.java
----------------------------------------------------------------------
diff --git a/blur-store/src/main/java/org/apache/blur/store/hdfs_v2/StoreDirection.java b/blur-store/src/main/java/org/apache/blur/store/hdfs_v2/StoreDirection.java
index 2dacc8a..2ee6e3c 100644
--- a/blur-store/src/main/java/org/apache/blur/store/hdfs_v2/StoreDirection.java
+++ b/blur-store/src/main/java/org/apache/blur/store/hdfs_v2/StoreDirection.java
@@ -16,8 +16,10 @@
  */
 package org.apache.blur.store.hdfs_v2;
 
+import org.apache.blur.utils.ThreadValue;
+
 public class StoreDirection {
-  public static ThreadLocal<Boolean> LONG_TERM = new ThreadLocal<Boolean>() {
+  public static ThreadValue<Boolean> LONG_TERM = new ThreadValue<Boolean>() {
     @Override
     protected Boolean initialValue() {
       return false;


Re: [03/13] git commit: Third round of updates.

Posted by Aaron McCurry <am...@gmail.com>.
Will do.  :-)

On Tue, Aug 30, 2016 at 9:10 AM, Tim Williams <wi...@gmail.com> wrote:

> No worries, just a friendly reminder:)  If you get time, I think it'd
> be helpful to a couple sentences about any new stuff/big changes...
> seems like there's a new project for example...
>
> Thanks,
> --tim
>
>
> On Tue, Aug 30, 2016 at 7:49 AM, Aaron McCurry <am...@gmail.com> wrote:
> > I apologize for the big commits without proper messaging.  It was
> difficult
> > to remember the changs and the original commit messages were lost due to
> an
> > offline git repo (which is no longer is use).  I only had the diff
> between
> > the original git repo and everything after the changes.  Plus the diff
> > didn't apply cleanly so that's why I broke it up in to different
> sections.
> >
> > I suppose I should have broke up the changes manually out of the diff and
> > applied them separately and recreated all the commit messages but I
> didn't
> > have the time to work through all of them.  Sorry.
> >
> > Aaron
> >
> >
> > On Tuesday, August 30, 2016, Tim Williams <wi...@gmail.com> wrote:
> >
> >> NoNot sure what this is yet but itPlease be more considerate with your
> >> commit messages... it's a lot of code to look through without having
> >> any context besides "N round of updates."
> >>
> >>
> >> On Mon, Aug 29, 2016 at 9:57 PM,  <amccurry@apache.org <javascript:;>>
> >> wrote:
> >> > Third round of updates.
> >> >
> >> >
> >> > Project: http://git-wip-us.apache.org/repos/asf/incubator-blur/repo
> >> > Commit: http://git-wip-us.apache.org/repos/asf/incubator-blur/
> >> commit/ea50630a
> >> > Tree: http://git-wip-us.apache.org/repos/asf/incubator-blur/tree/
> >> ea50630a
> >> > Diff: http://git-wip-us.apache.org/repos/asf/incubator-blur/diff/
> >> ea50630a
> >> >
> >> > Branch: refs/heads/master
> >> > Commit: ea50630a38d67675a61a916b144f3c0ce85d7f7a
> >> > Parents: 0141656
> >> > Author: Aaron McCurry <amccurry@gmail.com <javascript:;>>
> >> > Authored: Sat May 7 13:11:54 2016 -0400
> >> > Committer: Aaron McCurry <amccurry@gmail.com <javascript:;>>
> >> > Committed: Sat May 7 13:11:54 2016 -0400
> >> >
> >> > ------------------------------------------------------------
> ----------
> >> >  blur-indexer/pom.xml                            |  58 +++
> >> >  blur-indexer/src/main/assemble/bin.xml          |  45 ++
> >> >  .../mapreduce/lib/update/BlurIndexCounter.java  |  17 +
> >> >  .../mapreduce/lib/update/ClusterDriver.java     | 362 ++++++++++++++
> >> >  .../blur/mapreduce/lib/update/FasterDriver.java | 486
> >> +++++++++++++++++++
> >> >  .../update/HdfsConfigurationNamespaceMerge.java | 115 +++++
> >> >  .../lib/update/InputSplitPruneUtil.java         | 133 +++++
> >> >  .../lib/update/LookupBuilderMapper.java         |  18 +
> >> >  .../lib/update/LookupBuilderReducer.java        | 165 +++++++
> >> >  .../lib/update/MapperForExistingDataMod.java    |  46 ++
> >> >  .../MapperForExistingDataWithIndexLookup.java   | 228 +++++++++
> >> >  .../lib/update/MapperForNewDataMod.java         |  82 ++++
> >> >  .../lib/update/MergeSortRowIdMatcher.java       | 372 ++++++++++++++
> >> >  .../lib/update/PrunedBlurInputFormat.java       |  57 +++
> >> >  .../update/PrunedSequenceFileInputFormat.java   |  59 +++
> >> >  .../src/main/resources/blur-site.properties     |   1 +
> >> >  .../src/main/resources/program-log4j.xml        |  29 ++
> >> >  blur-indexer/src/main/resources/test-log4j.xml  |  46 ++
> >> >  18 files changed, 2319 insertions(+)
> >> > ------------------------------------------------------------
> ----------
> >> >
> >> >
> >> > http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/
> >> ea50630a/blur-indexer/pom.xml
> >> > ------------------------------------------------------------
> ----------
> >> > diff --git a/blur-indexer/pom.xml b/blur-indexer/pom.xml
> >> > new file mode 100644
> >> > index 0000000..c7c1753
> >> > --- /dev/null
> >> > +++ b/blur-indexer/pom.xml
> >> > @@ -0,0 +1,58 @@
> >> > +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="
> >> http://www.w3.org/2001/XMLSchema-instance"
> >> > +       xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
> >> http://maven.apache.org/xsd/maven-4.0.0.xsd">
> >> > +       <modelVersion>4.0.0</modelVersion>
> >> > +       <groupId>org.apache.blur</groupId>
> >> > +       <artifactId>blur-indexer</artifactId>
> >> > +       <version>0.2.8</version>
> >> > +       <name>blur-indexer</name>
> >> > +       <packaging>jar</packaging>
> >> > +
> >> > +       <properties>
> >> > +               <blur.version>0.3.0.incubating.2.5.0.cdh5.3.3-
> >> SNAPSHOT</blur.version>
> >> > +       </properties>
> >> > +       <dependencies>
> >> > +               <dependency>
> >> > +                       <groupId>org.apache.blur</groupId>
> >> > +                       <artifactId>blur-mapred</artifactId>
> >> > +                       <version>${blur.version}</version>
> >> > +               </dependency>
> >> > +               <dependency>
> >> > +                       <groupId>junit</groupId>
> >> > +                       <artifactId>junit</artifactId>
> >> > +                       <version>4.9</version>
> >> > +                       <scope>test</scope>
> >> > +               </dependency>
> >> > +       </dependencies>
> >> > +
> >> > +       <build>
> >> > +               <pluginManagement>
> >> > +                       <plugins>
> >> > +                               <plugin>
> >> > +                                       <groupId>org.apache.maven.
> >> plugins</groupId>
> >> > +                                       <artifactId>maven-compiler-
> >> plugin</artifactId>
> >> > +                                       <configuration>
> >> > +                                               <source>1.8</source>
> >> > +                                               <target>1.8</target>
> >> > +                                       </configuration>
> >> > +                               </plugin>
> >> > +                       </plugins>
> >> > +               </pluginManagement>
> >> > +               <plugins>
> >> > +                       <plugin>
> >> > +                               <artifactId>maven-assembly-
> >> plugin</artifactId>
> >> > +                               <configuration>
> >> > +                                       <descriptor>src/main/assemble/
> >> bin.xml</descriptor>
> >> > +                                       <finalName>blur-indexer-${
> >> project.version}</finalName>
> >> > +                               </configuration>
> >> > +                               <executions>
> >> > +                                       <execution>
> >> > +                                               <phase>package</phase>
> >> > +                                               <goals>
> >> > +
> >>  <goal>single</goal>
> >> > +                                               </goals>
> >> > +                                       </execution>
> >> > +                               </executions>
> >> > +                       </plugin>
> >> > +               </plugins>
> >> > +       </build>
> >> > +</project>
> >> >
> >> > http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/
> >> ea50630a/blur-indexer/src/main/assemble/bin.xml
> >> > ------------------------------------------------------------
> ----------
> >> > diff --git a/blur-indexer/src/main/assemble/bin.xml
> >> b/blur-indexer/src/main/assemble/bin.xml
> >> > new file mode 100644
> >> > index 0000000..5fddd56
> >> > --- /dev/null
> >> > +++ b/blur-indexer/src/main/assemble/bin.xml
> >> > @@ -0,0 +1,45 @@
> >> > +<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-
> >> plugin/assembly/1.1.2"
> >> > +    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
> >> > +           xsi:schemaLocation="http://maven.apache.org/plugins/
> >> maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/
> >> assembly-1.1.2.xsd">
> >> > +  <formats>
> >> > +    <format>tar.gz</format>
> >> > +  </formats>
> >> > +  <includeBaseDirectory>false</includeBaseDirectory>
> >> > +
> >> > +  <dependencySets>
> >> > +    <dependencySet>
> >> > +      <useProjectArtifact>true</useProjectArtifact>
> >> > +      <outputDirectory>blur-indexer-${project.version}/lib</
> >> outputDirectory>
> >> > +      <unpack>false</unpack>
> >> > +      <includes>
> >> > +        <include>org.apache.blur:blur-indexer</include>
> >> > +        <include>org.apache.blur:*</include>
> >> > +        <include>org.apache.zookeeper:zookeeper</include>
> >> > +        <include>org.slf4j:slf4j-api</include>
> >> > +        <include>org.slf4j:slf4j-log4j12</include>
> >> > +        <include>org.json:json</include>
> >> > +        <include>log4j:log4j</include>
> >> > +        <include>com.yammer.metrics:*</include>
> >> > +        <include>com.google.guava:guava</include>
> >> > +        <include>org.apache.httpcomponents:*</include>
> >> > +        <include>org.apache.lucene:*</include>
> >> > +        <include>com.spatial4j:spatial4j</include>
> >> > +        <include>commons-cli:commons-cli</include>
> >> > +        <include>org.eclipse.jetty:*</include>
> >> > +        <include>com.googlecode.concurrentlinkedhashmap:
> >> concurrentlinkedhashmap-lru</include>
> >> > +        <include>jline:jline</include>
> >> > +        <include>com.fasterxml.jackson.core:*</include>
> >> > +      </includes>
> >> > +    </dependencySet>
> >> > +  </dependencySets>
> >> > +
> >> > +  <fileSets>
> >> > +    <fileSet>
> >> > +      <directory>${project.build.scriptSourceDirectory}</directory>
> >> > +      <outputDirectory>blur-indexer-${project.version}</
> >> outputDirectory>
> >> > +      <excludes>
> >> > +        <exclude>**/.empty</exclude>
> >> > +      </excludes>
> >> > +    </fileSet>
> >> > +  </fileSets>
> >> > +</assembly>
> >> >
> >> > http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/
> >> ea50630a/blur-indexer/src/main/java/org/apache/blur/
> mapreduce/lib/update/
> >> BlurIndexCounter.java
> >> > ------------------------------------------------------------
> ----------
> >> > diff --git a/blur-indexer/src/main/java/
> org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
> >> b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/
> >> update/BlurIndexCounter.java
> >> > new file mode 100644
> >> > index 0000000..a9caabb
> >> > --- /dev/null
> >> > +++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/
> >> update/BlurIndexCounter.java
> >> > @@ -0,0 +1,17 @@
> >> > +package org.apache.blur.mapreduce.lib.update;
> >> > +
> >> > +public enum BlurIndexCounter {
> >> > +
> >> > +  NEW_RECORDS,
> >> > +  ROW_IDS_FROM_INDEX,
> >> > +  ROW_IDS_TO_UPDATE_FROM_NEW_DATA,
> >> > +  ROW_IDS_FROM_NEW_DATA,
> >> > +
> >> > +  INPUT_FORMAT_MAPPER,
> >> > +  INPUT_FORMAT_EXISTING_RECORDS,
> >> > +
> >> > +  LOOKUP_MAPPER,
> >> > +  LOOKUP_MAPPER_EXISTING_RECORDS,
> >> > +  LOOKUP_MAPPER_ROW_LOOKUP_ATTEMPT
> >> > +
> >> > +}
> >> >
> >> > http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/
> >> ea50630a/blur-indexer/src/main/java/org/apache/blur/
> mapreduce/lib/update/
> >> ClusterDriver.java
> >> > ------------------------------------------------------------
> ----------
> >> > diff --git a/blur-indexer/src/main/java/
> org/apache/blur/mapreduce/lib/update/ClusterDriver.java
> >> b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/
> >> update/ClusterDriver.java
> >> > new file mode 100644
> >> > index 0000000..d44adf1
> >> > --- /dev/null
> >> > +++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/
> >> update/ClusterDriver.java
> >> > @@ -0,0 +1,362 @@
> >> > +package org.apache.blur.mapreduce.lib.update;
> >> > +
> >> > +import java.io.ByteArrayInputStream;
> >> > +import java.io.ByteArrayOutputStream;
> >> > +import java.io.IOException;
> >> > +import java.io.InputStream;
> >> > +import java.net.URL;
> >> > +import java.util.HashMap;
> >> > +import java.util.HashSet;
> >> > +import java.util.List;
> >> > +import java.util.Map;
> >> > +import java.util.Map.Entry;
> >> > +import java.util.Set;
> >> > +import java.util.UUID;
> >> > +import java.util.concurrent.Callable;
> >> > +import java.util.concurrent.ExecutionException;
> >> > +import java.util.concurrent.ExecutorService;
> >> > +import java.util.concurrent.Executors;
> >> > +import java.util.concurrent.Future;
> >> > +import java.util.concurrent.TimeUnit;
> >> > +import java.util.concurrent.atomic.AtomicBoolean;
> >> > +
> >> > +import org.apache.blur.log.Log;
> >> > +import org.apache.blur.log.LogFactory;
> >> > +import org.apache.blur.mapreduce.lib.BlurInputFormat;
> >> > +import org.apache.blur.thirdparty.thrift_0_9_0.TException;
> >> > +import org.apache.blur.thrift.BlurClient;
> >> > +import org.apache.blur.thrift.generated.Blur.Iface;
> >> > +import org.apache.blur.thrift.generated.BlurException;
> >> > +import org.apache.blur.thrift.generated.TableDescriptor;
> >> > +import org.apache.blur.thrift.generated.TableStats;
> >> > +import org.apache.blur.utils.BlurConstants;
> >> > +import org.apache.commons.io.IOUtils;
> >> > +import org.apache.hadoop.conf.Configuration;
> >> > +import org.apache.hadoop.conf.Configured;
> >> > +import org.apache.hadoop.fs.FSDataInputStream;
> >> > +import org.apache.hadoop.fs.FileStatus;
> >> > +import org.apache.hadoop.fs.FileSystem;
> >> > +import org.apache.hadoop.fs.Path;
> >> > +import org.apache.hadoop.fs.permission.FsAction;
> >> > +import org.apache.hadoop.mapreduce.Cluster;
> >> > +import org.apache.hadoop.mapreduce.Job;
> >> > +import org.apache.hadoop.mapreduce.JobID;
> >> > +import org.apache.hadoop.mapreduce.JobStatus;
> >> > +import org.apache.hadoop.util.Tool;
> >> > +import org.apache.hadoop.util.ToolRunner;
> >> > +import org.apache.hadoop.yarn.exceptions.YarnException;
> >> > +import org.apache.log4j.LogManager;
> >> > +import org.apache.log4j.xml.DOMConfigurator;
> >> > +
> >> > +public class ClusterDriver extends Configured implements Tool {
> >> > +
> >> > +  private static final String BLUR_ENV = "blur.env";
> >> > +  private static final Log LOG = LogFactory.getLog(
> >> ClusterDriver.class);
> >> > +  private static final String _SEP = "_";
> >> > +  private static final String IMPORT = "import";
> >> > +
> >> > +  public static void main(String[] args) throws Exception {
> >> > +    String logFilePath = System.getenv("BLUR_INDEXER_LOG_FILE");
> >> > +    System.out.println("Log file path [" + logFilePath + "]");
> >> > +    System.setProperty("BLUR_INDEXER_LOG_FILE", logFilePath);
> >> > +    URL url = ClusterDriver.class.getResource("/program-log4j.xml");
> >> > +    if (url != null) {
> >> > +      LOG.info("Reseting log4j config from classpath resource [{0}]",
> >> url);
> >> > +      LogManager.resetConfiguration();
> >> > +      DOMConfigurator.configure(url);
> >> > +    }
> >> > +    int res = ToolRunner.run(new Configuration(), new
> ClusterDriver(),
> >> args);
> >>
> >> Not sure what this thing does yet but it seems we should validate
> >> those args since their accessed blindly in run...
> >>
> >> --tim
> >>
>

Re: [03/13] git commit: Third round of updates.

Posted by Tim Williams <wi...@gmail.com>.
No worries, just a friendly reminder:)  If you get time, I think it'd
be helpful to a couple sentences about any new stuff/big changes...
seems like there's a new project for example...

Thanks,
--tim


On Tue, Aug 30, 2016 at 7:49 AM, Aaron McCurry <am...@gmail.com> wrote:
> I apologize for the big commits without proper messaging.  It was difficult
> to remember the changs and the original commit messages were lost due to an
> offline git repo (which is no longer is use).  I only had the diff between
> the original git repo and everything after the changes.  Plus the diff
> didn't apply cleanly so that's why I broke it up in to different sections.
>
> I suppose I should have broke up the changes manually out of the diff and
> applied them separately and recreated all the commit messages but I didn't
> have the time to work through all of them.  Sorry.
>
> Aaron
>
>
> On Tuesday, August 30, 2016, Tim Williams <wi...@gmail.com> wrote:
>
>> NoNot sure what this is yet but itPlease be more considerate with your
>> commit messages... it's a lot of code to look through without having
>> any context besides "N round of updates."
>>
>>
>> On Mon, Aug 29, 2016 at 9:57 PM,  <amccurry@apache.org <javascript:;>>
>> wrote:
>> > Third round of updates.
>> >
>> >
>> > Project: http://git-wip-us.apache.org/repos/asf/incubator-blur/repo
>> > Commit: http://git-wip-us.apache.org/repos/asf/incubator-blur/
>> commit/ea50630a
>> > Tree: http://git-wip-us.apache.org/repos/asf/incubator-blur/tree/
>> ea50630a
>> > Diff: http://git-wip-us.apache.org/repos/asf/incubator-blur/diff/
>> ea50630a
>> >
>> > Branch: refs/heads/master
>> > Commit: ea50630a38d67675a61a916b144f3c0ce85d7f7a
>> > Parents: 0141656
>> > Author: Aaron McCurry <amccurry@gmail.com <javascript:;>>
>> > Authored: Sat May 7 13:11:54 2016 -0400
>> > Committer: Aaron McCurry <amccurry@gmail.com <javascript:;>>
>> > Committed: Sat May 7 13:11:54 2016 -0400
>> >
>> > ----------------------------------------------------------------------
>> >  blur-indexer/pom.xml                            |  58 +++
>> >  blur-indexer/src/main/assemble/bin.xml          |  45 ++
>> >  .../mapreduce/lib/update/BlurIndexCounter.java  |  17 +
>> >  .../mapreduce/lib/update/ClusterDriver.java     | 362 ++++++++++++++
>> >  .../blur/mapreduce/lib/update/FasterDriver.java | 486
>> +++++++++++++++++++
>> >  .../update/HdfsConfigurationNamespaceMerge.java | 115 +++++
>> >  .../lib/update/InputSplitPruneUtil.java         | 133 +++++
>> >  .../lib/update/LookupBuilderMapper.java         |  18 +
>> >  .../lib/update/LookupBuilderReducer.java        | 165 +++++++
>> >  .../lib/update/MapperForExistingDataMod.java    |  46 ++
>> >  .../MapperForExistingDataWithIndexLookup.java   | 228 +++++++++
>> >  .../lib/update/MapperForNewDataMod.java         |  82 ++++
>> >  .../lib/update/MergeSortRowIdMatcher.java       | 372 ++++++++++++++
>> >  .../lib/update/PrunedBlurInputFormat.java       |  57 +++
>> >  .../update/PrunedSequenceFileInputFormat.java   |  59 +++
>> >  .../src/main/resources/blur-site.properties     |   1 +
>> >  .../src/main/resources/program-log4j.xml        |  29 ++
>> >  blur-indexer/src/main/resources/test-log4j.xml  |  46 ++
>> >  18 files changed, 2319 insertions(+)
>> > ----------------------------------------------------------------------
>> >
>> >
>> > http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/
>> ea50630a/blur-indexer/pom.xml
>> > ----------------------------------------------------------------------
>> > diff --git a/blur-indexer/pom.xml b/blur-indexer/pom.xml
>> > new file mode 100644
>> > index 0000000..c7c1753
>> > --- /dev/null
>> > +++ b/blur-indexer/pom.xml
>> > @@ -0,0 +1,58 @@
>> > +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="
>> http://www.w3.org/2001/XMLSchema-instance"
>> > +       xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
>> http://maven.apache.org/xsd/maven-4.0.0.xsd">
>> > +       <modelVersion>4.0.0</modelVersion>
>> > +       <groupId>org.apache.blur</groupId>
>> > +       <artifactId>blur-indexer</artifactId>
>> > +       <version>0.2.8</version>
>> > +       <name>blur-indexer</name>
>> > +       <packaging>jar</packaging>
>> > +
>> > +       <properties>
>> > +               <blur.version>0.3.0.incubating.2.5.0.cdh5.3.3-
>> SNAPSHOT</blur.version>
>> > +       </properties>
>> > +       <dependencies>
>> > +               <dependency>
>> > +                       <groupId>org.apache.blur</groupId>
>> > +                       <artifactId>blur-mapred</artifactId>
>> > +                       <version>${blur.version}</version>
>> > +               </dependency>
>> > +               <dependency>
>> > +                       <groupId>junit</groupId>
>> > +                       <artifactId>junit</artifactId>
>> > +                       <version>4.9</version>
>> > +                       <scope>test</scope>
>> > +               </dependency>
>> > +       </dependencies>
>> > +
>> > +       <build>
>> > +               <pluginManagement>
>> > +                       <plugins>
>> > +                               <plugin>
>> > +                                       <groupId>org.apache.maven.
>> plugins</groupId>
>> > +                                       <artifactId>maven-compiler-
>> plugin</artifactId>
>> > +                                       <configuration>
>> > +                                               <source>1.8</source>
>> > +                                               <target>1.8</target>
>> > +                                       </configuration>
>> > +                               </plugin>
>> > +                       </plugins>
>> > +               </pluginManagement>
>> > +               <plugins>
>> > +                       <plugin>
>> > +                               <artifactId>maven-assembly-
>> plugin</artifactId>
>> > +                               <configuration>
>> > +                                       <descriptor>src/main/assemble/
>> bin.xml</descriptor>
>> > +                                       <finalName>blur-indexer-${
>> project.version}</finalName>
>> > +                               </configuration>
>> > +                               <executions>
>> > +                                       <execution>
>> > +                                               <phase>package</phase>
>> > +                                               <goals>
>> > +
>>  <goal>single</goal>
>> > +                                               </goals>
>> > +                                       </execution>
>> > +                               </executions>
>> > +                       </plugin>
>> > +               </plugins>
>> > +       </build>
>> > +</project>
>> >
>> > http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/
>> ea50630a/blur-indexer/src/main/assemble/bin.xml
>> > ----------------------------------------------------------------------
>> > diff --git a/blur-indexer/src/main/assemble/bin.xml
>> b/blur-indexer/src/main/assemble/bin.xml
>> > new file mode 100644
>> > index 0000000..5fddd56
>> > --- /dev/null
>> > +++ b/blur-indexer/src/main/assemble/bin.xml
>> > @@ -0,0 +1,45 @@
>> > +<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-
>> plugin/assembly/1.1.2"
>> > +    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
>> > +           xsi:schemaLocation="http://maven.apache.org/plugins/
>> maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/
>> assembly-1.1.2.xsd">
>> > +  <formats>
>> > +    <format>tar.gz</format>
>> > +  </formats>
>> > +  <includeBaseDirectory>false</includeBaseDirectory>
>> > +
>> > +  <dependencySets>
>> > +    <dependencySet>
>> > +      <useProjectArtifact>true</useProjectArtifact>
>> > +      <outputDirectory>blur-indexer-${project.version}/lib</
>> outputDirectory>
>> > +      <unpack>false</unpack>
>> > +      <includes>
>> > +        <include>org.apache.blur:blur-indexer</include>
>> > +        <include>org.apache.blur:*</include>
>> > +        <include>org.apache.zookeeper:zookeeper</include>
>> > +        <include>org.slf4j:slf4j-api</include>
>> > +        <include>org.slf4j:slf4j-log4j12</include>
>> > +        <include>org.json:json</include>
>> > +        <include>log4j:log4j</include>
>> > +        <include>com.yammer.metrics:*</include>
>> > +        <include>com.google.guava:guava</include>
>> > +        <include>org.apache.httpcomponents:*</include>
>> > +        <include>org.apache.lucene:*</include>
>> > +        <include>com.spatial4j:spatial4j</include>
>> > +        <include>commons-cli:commons-cli</include>
>> > +        <include>org.eclipse.jetty:*</include>
>> > +        <include>com.googlecode.concurrentlinkedhashmap:
>> concurrentlinkedhashmap-lru</include>
>> > +        <include>jline:jline</include>
>> > +        <include>com.fasterxml.jackson.core:*</include>
>> > +      </includes>
>> > +    </dependencySet>
>> > +  </dependencySets>
>> > +
>> > +  <fileSets>
>> > +    <fileSet>
>> > +      <directory>${project.build.scriptSourceDirectory}</directory>
>> > +      <outputDirectory>blur-indexer-${project.version}</
>> outputDirectory>
>> > +      <excludes>
>> > +        <exclude>**/.empty</exclude>
>> > +      </excludes>
>> > +    </fileSet>
>> > +  </fileSets>
>> > +</assembly>
>> >
>> > http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/
>> ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/
>> BlurIndexCounter.java
>> > ----------------------------------------------------------------------
>> > diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
>> b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/
>> update/BlurIndexCounter.java
>> > new file mode 100644
>> > index 0000000..a9caabb
>> > --- /dev/null
>> > +++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/
>> update/BlurIndexCounter.java
>> > @@ -0,0 +1,17 @@
>> > +package org.apache.blur.mapreduce.lib.update;
>> > +
>> > +public enum BlurIndexCounter {
>> > +
>> > +  NEW_RECORDS,
>> > +  ROW_IDS_FROM_INDEX,
>> > +  ROW_IDS_TO_UPDATE_FROM_NEW_DATA,
>> > +  ROW_IDS_FROM_NEW_DATA,
>> > +
>> > +  INPUT_FORMAT_MAPPER,
>> > +  INPUT_FORMAT_EXISTING_RECORDS,
>> > +
>> > +  LOOKUP_MAPPER,
>> > +  LOOKUP_MAPPER_EXISTING_RECORDS,
>> > +  LOOKUP_MAPPER_ROW_LOOKUP_ATTEMPT
>> > +
>> > +}
>> >
>> > http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/
>> ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/
>> ClusterDriver.java
>> > ----------------------------------------------------------------------
>> > diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
>> b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/
>> update/ClusterDriver.java
>> > new file mode 100644
>> > index 0000000..d44adf1
>> > --- /dev/null
>> > +++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/
>> update/ClusterDriver.java
>> > @@ -0,0 +1,362 @@
>> > +package org.apache.blur.mapreduce.lib.update;
>> > +
>> > +import java.io.ByteArrayInputStream;
>> > +import java.io.ByteArrayOutputStream;
>> > +import java.io.IOException;
>> > +import java.io.InputStream;
>> > +import java.net.URL;
>> > +import java.util.HashMap;
>> > +import java.util.HashSet;
>> > +import java.util.List;
>> > +import java.util.Map;
>> > +import java.util.Map.Entry;
>> > +import java.util.Set;
>> > +import java.util.UUID;
>> > +import java.util.concurrent.Callable;
>> > +import java.util.concurrent.ExecutionException;
>> > +import java.util.concurrent.ExecutorService;
>> > +import java.util.concurrent.Executors;
>> > +import java.util.concurrent.Future;
>> > +import java.util.concurrent.TimeUnit;
>> > +import java.util.concurrent.atomic.AtomicBoolean;
>> > +
>> > +import org.apache.blur.log.Log;
>> > +import org.apache.blur.log.LogFactory;
>> > +import org.apache.blur.mapreduce.lib.BlurInputFormat;
>> > +import org.apache.blur.thirdparty.thrift_0_9_0.TException;
>> > +import org.apache.blur.thrift.BlurClient;
>> > +import org.apache.blur.thrift.generated.Blur.Iface;
>> > +import org.apache.blur.thrift.generated.BlurException;
>> > +import org.apache.blur.thrift.generated.TableDescriptor;
>> > +import org.apache.blur.thrift.generated.TableStats;
>> > +import org.apache.blur.utils.BlurConstants;
>> > +import org.apache.commons.io.IOUtils;
>> > +import org.apache.hadoop.conf.Configuration;
>> > +import org.apache.hadoop.conf.Configured;
>> > +import org.apache.hadoop.fs.FSDataInputStream;
>> > +import org.apache.hadoop.fs.FileStatus;
>> > +import org.apache.hadoop.fs.FileSystem;
>> > +import org.apache.hadoop.fs.Path;
>> > +import org.apache.hadoop.fs.permission.FsAction;
>> > +import org.apache.hadoop.mapreduce.Cluster;
>> > +import org.apache.hadoop.mapreduce.Job;
>> > +import org.apache.hadoop.mapreduce.JobID;
>> > +import org.apache.hadoop.mapreduce.JobStatus;
>> > +import org.apache.hadoop.util.Tool;
>> > +import org.apache.hadoop.util.ToolRunner;
>> > +import org.apache.hadoop.yarn.exceptions.YarnException;
>> > +import org.apache.log4j.LogManager;
>> > +import org.apache.log4j.xml.DOMConfigurator;
>> > +
>> > +public class ClusterDriver extends Configured implements Tool {
>> > +
>> > +  private static final String BLUR_ENV = "blur.env";
>> > +  private static final Log LOG = LogFactory.getLog(
>> ClusterDriver.class);
>> > +  private static final String _SEP = "_";
>> > +  private static final String IMPORT = "import";
>> > +
>> > +  public static void main(String[] args) throws Exception {
>> > +    String logFilePath = System.getenv("BLUR_INDEXER_LOG_FILE");
>> > +    System.out.println("Log file path [" + logFilePath + "]");
>> > +    System.setProperty("BLUR_INDEXER_LOG_FILE", logFilePath);
>> > +    URL url = ClusterDriver.class.getResource("/program-log4j.xml");
>> > +    if (url != null) {
>> > +      LOG.info("Reseting log4j config from classpath resource [{0}]",
>> url);
>> > +      LogManager.resetConfiguration();
>> > +      DOMConfigurator.configure(url);
>> > +    }
>> > +    int res = ToolRunner.run(new Configuration(), new ClusterDriver(),
>> args);
>>
>> Not sure what this thing does yet but it seems we should validate
>> those args since their accessed blindly in run...
>>
>> --tim
>>

Re: [03/13] git commit: Third round of updates.

Posted by Aaron McCurry <am...@gmail.com>.
I apologize for the big commits without proper messaging.  It was difficult
to remember the changs and the original commit messages were lost due to an
offline git repo (which is no longer is use).  I only had the diff between
the original git repo and everything after the changes.  Plus the diff
didn't apply cleanly so that's why I broke it up in to different sections.

I suppose I should have broke up the changes manually out of the diff and
applied them separately and recreated all the commit messages but I didn't
have the time to work through all of them.  Sorry.

Aaron


On Tuesday, August 30, 2016, Tim Williams <wi...@gmail.com> wrote:

> NoNot sure what this is yet but itPlease be more considerate with your
> commit messages... it's a lot of code to look through without having
> any context besides "N round of updates."
>
>
> On Mon, Aug 29, 2016 at 9:57 PM,  <amccurry@apache.org <javascript:;>>
> wrote:
> > Third round of updates.
> >
> >
> > Project: http://git-wip-us.apache.org/repos/asf/incubator-blur/repo
> > Commit: http://git-wip-us.apache.org/repos/asf/incubator-blur/
> commit/ea50630a
> > Tree: http://git-wip-us.apache.org/repos/asf/incubator-blur/tree/
> ea50630a
> > Diff: http://git-wip-us.apache.org/repos/asf/incubator-blur/diff/
> ea50630a
> >
> > Branch: refs/heads/master
> > Commit: ea50630a38d67675a61a916b144f3c0ce85d7f7a
> > Parents: 0141656
> > Author: Aaron McCurry <amccurry@gmail.com <javascript:;>>
> > Authored: Sat May 7 13:11:54 2016 -0400
> > Committer: Aaron McCurry <amccurry@gmail.com <javascript:;>>
> > Committed: Sat May 7 13:11:54 2016 -0400
> >
> > ----------------------------------------------------------------------
> >  blur-indexer/pom.xml                            |  58 +++
> >  blur-indexer/src/main/assemble/bin.xml          |  45 ++
> >  .../mapreduce/lib/update/BlurIndexCounter.java  |  17 +
> >  .../mapreduce/lib/update/ClusterDriver.java     | 362 ++++++++++++++
> >  .../blur/mapreduce/lib/update/FasterDriver.java | 486
> +++++++++++++++++++
> >  .../update/HdfsConfigurationNamespaceMerge.java | 115 +++++
> >  .../lib/update/InputSplitPruneUtil.java         | 133 +++++
> >  .../lib/update/LookupBuilderMapper.java         |  18 +
> >  .../lib/update/LookupBuilderReducer.java        | 165 +++++++
> >  .../lib/update/MapperForExistingDataMod.java    |  46 ++
> >  .../MapperForExistingDataWithIndexLookup.java   | 228 +++++++++
> >  .../lib/update/MapperForNewDataMod.java         |  82 ++++
> >  .../lib/update/MergeSortRowIdMatcher.java       | 372 ++++++++++++++
> >  .../lib/update/PrunedBlurInputFormat.java       |  57 +++
> >  .../update/PrunedSequenceFileInputFormat.java   |  59 +++
> >  .../src/main/resources/blur-site.properties     |   1 +
> >  .../src/main/resources/program-log4j.xml        |  29 ++
> >  blur-indexer/src/main/resources/test-log4j.xml  |  46 ++
> >  18 files changed, 2319 insertions(+)
> > ----------------------------------------------------------------------
> >
> >
> > http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/
> ea50630a/blur-indexer/pom.xml
> > ----------------------------------------------------------------------
> > diff --git a/blur-indexer/pom.xml b/blur-indexer/pom.xml
> > new file mode 100644
> > index 0000000..c7c1753
> > --- /dev/null
> > +++ b/blur-indexer/pom.xml
> > @@ -0,0 +1,58 @@
> > +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="
> http://www.w3.org/2001/XMLSchema-instance"
> > +       xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
> http://maven.apache.org/xsd/maven-4.0.0.xsd">
> > +       <modelVersion>4.0.0</modelVersion>
> > +       <groupId>org.apache.blur</groupId>
> > +       <artifactId>blur-indexer</artifactId>
> > +       <version>0.2.8</version>
> > +       <name>blur-indexer</name>
> > +       <packaging>jar</packaging>
> > +
> > +       <properties>
> > +               <blur.version>0.3.0.incubating.2.5.0.cdh5.3.3-
> SNAPSHOT</blur.version>
> > +       </properties>
> > +       <dependencies>
> > +               <dependency>
> > +                       <groupId>org.apache.blur</groupId>
> > +                       <artifactId>blur-mapred</artifactId>
> > +                       <version>${blur.version}</version>
> > +               </dependency>
> > +               <dependency>
> > +                       <groupId>junit</groupId>
> > +                       <artifactId>junit</artifactId>
> > +                       <version>4.9</version>
> > +                       <scope>test</scope>
> > +               </dependency>
> > +       </dependencies>
> > +
> > +       <build>
> > +               <pluginManagement>
> > +                       <plugins>
> > +                               <plugin>
> > +                                       <groupId>org.apache.maven.
> plugins</groupId>
> > +                                       <artifactId>maven-compiler-
> plugin</artifactId>
> > +                                       <configuration>
> > +                                               <source>1.8</source>
> > +                                               <target>1.8</target>
> > +                                       </configuration>
> > +                               </plugin>
> > +                       </plugins>
> > +               </pluginManagement>
> > +               <plugins>
> > +                       <plugin>
> > +                               <artifactId>maven-assembly-
> plugin</artifactId>
> > +                               <configuration>
> > +                                       <descriptor>src/main/assemble/
> bin.xml</descriptor>
> > +                                       <finalName>blur-indexer-${
> project.version}</finalName>
> > +                               </configuration>
> > +                               <executions>
> > +                                       <execution>
> > +                                               <phase>package</phase>
> > +                                               <goals>
> > +
>  <goal>single</goal>
> > +                                               </goals>
> > +                                       </execution>
> > +                               </executions>
> > +                       </plugin>
> > +               </plugins>
> > +       </build>
> > +</project>
> >
> > http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/
> ea50630a/blur-indexer/src/main/assemble/bin.xml
> > ----------------------------------------------------------------------
> > diff --git a/blur-indexer/src/main/assemble/bin.xml
> b/blur-indexer/src/main/assemble/bin.xml
> > new file mode 100644
> > index 0000000..5fddd56
> > --- /dev/null
> > +++ b/blur-indexer/src/main/assemble/bin.xml
> > @@ -0,0 +1,45 @@
> > +<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-
> plugin/assembly/1.1.2"
> > +    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
> > +           xsi:schemaLocation="http://maven.apache.org/plugins/
> maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/
> assembly-1.1.2.xsd">
> > +  <formats>
> > +    <format>tar.gz</format>
> > +  </formats>
> > +  <includeBaseDirectory>false</includeBaseDirectory>
> > +
> > +  <dependencySets>
> > +    <dependencySet>
> > +      <useProjectArtifact>true</useProjectArtifact>
> > +      <outputDirectory>blur-indexer-${project.version}/lib</
> outputDirectory>
> > +      <unpack>false</unpack>
> > +      <includes>
> > +        <include>org.apache.blur:blur-indexer</include>
> > +        <include>org.apache.blur:*</include>
> > +        <include>org.apache.zookeeper:zookeeper</include>
> > +        <include>org.slf4j:slf4j-api</include>
> > +        <include>org.slf4j:slf4j-log4j12</include>
> > +        <include>org.json:json</include>
> > +        <include>log4j:log4j</include>
> > +        <include>com.yammer.metrics:*</include>
> > +        <include>com.google.guava:guava</include>
> > +        <include>org.apache.httpcomponents:*</include>
> > +        <include>org.apache.lucene:*</include>
> > +        <include>com.spatial4j:spatial4j</include>
> > +        <include>commons-cli:commons-cli</include>
> > +        <include>org.eclipse.jetty:*</include>
> > +        <include>com.googlecode.concurrentlinkedhashmap:
> concurrentlinkedhashmap-lru</include>
> > +        <include>jline:jline</include>
> > +        <include>com.fasterxml.jackson.core:*</include>
> > +      </includes>
> > +    </dependencySet>
> > +  </dependencySets>
> > +
> > +  <fileSets>
> > +    <fileSet>
> > +      <directory>${project.build.scriptSourceDirectory}</directory>
> > +      <outputDirectory>blur-indexer-${project.version}</
> outputDirectory>
> > +      <excludes>
> > +        <exclude>**/.empty</exclude>
> > +      </excludes>
> > +    </fileSet>
> > +  </fileSets>
> > +</assembly>
> >
> > http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/
> ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/
> BlurIndexCounter.java
> > ----------------------------------------------------------------------
> > diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
> b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/
> update/BlurIndexCounter.java
> > new file mode 100644
> > index 0000000..a9caabb
> > --- /dev/null
> > +++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/
> update/BlurIndexCounter.java
> > @@ -0,0 +1,17 @@
> > +package org.apache.blur.mapreduce.lib.update;
> > +
> > +public enum BlurIndexCounter {
> > +
> > +  NEW_RECORDS,
> > +  ROW_IDS_FROM_INDEX,
> > +  ROW_IDS_TO_UPDATE_FROM_NEW_DATA,
> > +  ROW_IDS_FROM_NEW_DATA,
> > +
> > +  INPUT_FORMAT_MAPPER,
> > +  INPUT_FORMAT_EXISTING_RECORDS,
> > +
> > +  LOOKUP_MAPPER,
> > +  LOOKUP_MAPPER_EXISTING_RECORDS,
> > +  LOOKUP_MAPPER_ROW_LOOKUP_ATTEMPT
> > +
> > +}
> >
> > http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/
> ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/
> ClusterDriver.java
> > ----------------------------------------------------------------------
> > diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
> b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/
> update/ClusterDriver.java
> > new file mode 100644
> > index 0000000..d44adf1
> > --- /dev/null
> > +++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/
> update/ClusterDriver.java
> > @@ -0,0 +1,362 @@
> > +package org.apache.blur.mapreduce.lib.update;
> > +
> > +import java.io.ByteArrayInputStream;
> > +import java.io.ByteArrayOutputStream;
> > +import java.io.IOException;
> > +import java.io.InputStream;
> > +import java.net.URL;
> > +import java.util.HashMap;
> > +import java.util.HashSet;
> > +import java.util.List;
> > +import java.util.Map;
> > +import java.util.Map.Entry;
> > +import java.util.Set;
> > +import java.util.UUID;
> > +import java.util.concurrent.Callable;
> > +import java.util.concurrent.ExecutionException;
> > +import java.util.concurrent.ExecutorService;
> > +import java.util.concurrent.Executors;
> > +import java.util.concurrent.Future;
> > +import java.util.concurrent.TimeUnit;
> > +import java.util.concurrent.atomic.AtomicBoolean;
> > +
> > +import org.apache.blur.log.Log;
> > +import org.apache.blur.log.LogFactory;
> > +import org.apache.blur.mapreduce.lib.BlurInputFormat;
> > +import org.apache.blur.thirdparty.thrift_0_9_0.TException;
> > +import org.apache.blur.thrift.BlurClient;
> > +import org.apache.blur.thrift.generated.Blur.Iface;
> > +import org.apache.blur.thrift.generated.BlurException;
> > +import org.apache.blur.thrift.generated.TableDescriptor;
> > +import org.apache.blur.thrift.generated.TableStats;
> > +import org.apache.blur.utils.BlurConstants;
> > +import org.apache.commons.io.IOUtils;
> > +import org.apache.hadoop.conf.Configuration;
> > +import org.apache.hadoop.conf.Configured;
> > +import org.apache.hadoop.fs.FSDataInputStream;
> > +import org.apache.hadoop.fs.FileStatus;
> > +import org.apache.hadoop.fs.FileSystem;
> > +import org.apache.hadoop.fs.Path;
> > +import org.apache.hadoop.fs.permission.FsAction;
> > +import org.apache.hadoop.mapreduce.Cluster;
> > +import org.apache.hadoop.mapreduce.Job;
> > +import org.apache.hadoop.mapreduce.JobID;
> > +import org.apache.hadoop.mapreduce.JobStatus;
> > +import org.apache.hadoop.util.Tool;
> > +import org.apache.hadoop.util.ToolRunner;
> > +import org.apache.hadoop.yarn.exceptions.YarnException;
> > +import org.apache.log4j.LogManager;
> > +import org.apache.log4j.xml.DOMConfigurator;
> > +
> > +public class ClusterDriver extends Configured implements Tool {
> > +
> > +  private static final String BLUR_ENV = "blur.env";
> > +  private static final Log LOG = LogFactory.getLog(
> ClusterDriver.class);
> > +  private static final String _SEP = "_";
> > +  private static final String IMPORT = "import";
> > +
> > +  public static void main(String[] args) throws Exception {
> > +    String logFilePath = System.getenv("BLUR_INDEXER_LOG_FILE");
> > +    System.out.println("Log file path [" + logFilePath + "]");
> > +    System.setProperty("BLUR_INDEXER_LOG_FILE", logFilePath);
> > +    URL url = ClusterDriver.class.getResource("/program-log4j.xml");
> > +    if (url != null) {
> > +      LOG.info("Reseting log4j config from classpath resource [{0}]",
> url);
> > +      LogManager.resetConfiguration();
> > +      DOMConfigurator.configure(url);
> > +    }
> > +    int res = ToolRunner.run(new Configuration(), new ClusterDriver(),
> args);
>
> Not sure what this thing does yet but it seems we should validate
> those args since their accessed blindly in run...
>
> --tim
>

Re: [04/13] git commit: Third round of updates.

Posted by Tim Williams <wi...@gmail.com>.
NoNot sure what this is yet but itPlease be more considerate with your
commit messages... it's a lot of code to look through without having
any context besides "N round of updates."


On Mon, Aug 29, 2016 at 9:57 PM,  <am...@apache.org> wrote:
> Third round of updates.
>
>
> Project: http://git-wip-us.apache.org/repos/asf/incubator-blur/repo
> Commit: http://git-wip-us.apache.org/repos/asf/incubator-blur/commit/ea50630a
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-blur/tree/ea50630a
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-blur/diff/ea50630a
>
> Branch: refs/heads/master
> Commit: ea50630a38d67675a61a916b144f3c0ce85d7f7a
> Parents: 0141656
> Author: Aaron McCurry <am...@gmail.com>
> Authored: Sat May 7 13:11:54 2016 -0400
> Committer: Aaron McCurry <am...@gmail.com>
> Committed: Sat May 7 13:11:54 2016 -0400
>
> ----------------------------------------------------------------------
>  blur-indexer/pom.xml                            |  58 +++
>  blur-indexer/src/main/assemble/bin.xml          |  45 ++
>  .../mapreduce/lib/update/BlurIndexCounter.java  |  17 +
>  .../mapreduce/lib/update/ClusterDriver.java     | 362 ++++++++++++++
>  .../blur/mapreduce/lib/update/FasterDriver.java | 486 +++++++++++++++++++
>  .../update/HdfsConfigurationNamespaceMerge.java | 115 +++++
>  .../lib/update/InputSplitPruneUtil.java         | 133 +++++
>  .../lib/update/LookupBuilderMapper.java         |  18 +
>  .../lib/update/LookupBuilderReducer.java        | 165 +++++++
>  .../lib/update/MapperForExistingDataMod.java    |  46 ++
>  .../MapperForExistingDataWithIndexLookup.java   | 228 +++++++++
>  .../lib/update/MapperForNewDataMod.java         |  82 ++++
>  .../lib/update/MergeSortRowIdMatcher.java       | 372 ++++++++++++++
>  .../lib/update/PrunedBlurInputFormat.java       |  57 +++
>  .../update/PrunedSequenceFileInputFormat.java   |  59 +++
>  .../src/main/resources/blur-site.properties     |   1 +
>  .../src/main/resources/program-log4j.xml        |  29 ++
>  blur-indexer/src/main/resources/test-log4j.xml  |  46 ++
>  18 files changed, 2319 insertions(+)
> ----------------------------------------------------------------------
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/pom.xml
> ----------------------------------------------------------------------
> diff --git a/blur-indexer/pom.xml b/blur-indexer/pom.xml
> new file mode 100644
> index 0000000..c7c1753
> --- /dev/null
> +++ b/blur-indexer/pom.xml
> @@ -0,0 +1,58 @@
> +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
> +       xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
> +       <modelVersion>4.0.0</modelVersion>
> +       <groupId>org.apache.blur</groupId>
> +       <artifactId>blur-indexer</artifactId>
> +       <version>0.2.8</version>
> +       <name>blur-indexer</name>
> +       <packaging>jar</packaging>
> +
> +       <properties>
> +               <blur.version>0.3.0.incubating.2.5.0.cdh5.3.3-SNAPSHOT</blur.version>
> +       </properties>
> +       <dependencies>
> +               <dependency>
> +                       <groupId>org.apache.blur</groupId>
> +                       <artifactId>blur-mapred</artifactId>
> +                       <version>${blur.version}</version>
> +               </dependency>
> +               <dependency>
> +                       <groupId>junit</groupId>
> +                       <artifactId>junit</artifactId>
> +                       <version>4.9</version>
> +                       <scope>test</scope>
> +               </dependency>
> +       </dependencies>
> +
> +       <build>
> +               <pluginManagement>
> +                       <plugins>
> +                               <plugin>
> +                                       <groupId>org.apache.maven.plugins</groupId>
> +                                       <artifactId>maven-compiler-plugin</artifactId>
> +                                       <configuration>
> +                                               <source>1.8</source>
> +                                               <target>1.8</target>
> +                                       </configuration>
> +                               </plugin>
> +                       </plugins>
> +               </pluginManagement>
> +               <plugins>
> +                       <plugin>
> +                               <artifactId>maven-assembly-plugin</artifactId>
> +                               <configuration>
> +                                       <descriptor>src/main/assemble/bin.xml</descriptor>
> +                                       <finalName>blur-indexer-${project.version}</finalName>
> +                               </configuration>
> +                               <executions>
> +                                       <execution>
> +                                               <phase>package</phase>
> +                                               <goals>
> +                                                       <goal>single</goal>
> +                                               </goals>
> +                                       </execution>
> +                               </executions>
> +                       </plugin>
> +               </plugins>
> +       </build>
> +</project>
>
> http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/assemble/bin.xml
> ----------------------------------------------------------------------
> diff --git a/blur-indexer/src/main/assemble/bin.xml b/blur-indexer/src/main/assemble/bin.xml
> new file mode 100644
> index 0000000..5fddd56
> --- /dev/null
> +++ b/blur-indexer/src/main/assemble/bin.xml
> @@ -0,0 +1,45 @@
> +<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
> +    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
> +           xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
> +  <formats>
> +    <format>tar.gz</format>
> +  </formats>
> +  <includeBaseDirectory>false</includeBaseDirectory>
> +
> +  <dependencySets>
> +    <dependencySet>
> +      <useProjectArtifact>true</useProjectArtifact>
> +      <outputDirectory>blur-indexer-${project.version}/lib</outputDirectory>
> +      <unpack>false</unpack>
> +      <includes>
> +        <include>org.apache.blur:blur-indexer</include>
> +        <include>org.apache.blur:*</include>
> +        <include>org.apache.zookeeper:zookeeper</include>
> +        <include>org.slf4j:slf4j-api</include>
> +        <include>org.slf4j:slf4j-log4j12</include>
> +        <include>org.json:json</include>
> +        <include>log4j:log4j</include>
> +        <include>com.yammer.metrics:*</include>
> +        <include>com.google.guava:guava</include>
> +        <include>org.apache.httpcomponents:*</include>
> +        <include>org.apache.lucene:*</include>
> +        <include>com.spatial4j:spatial4j</include>
> +        <include>commons-cli:commons-cli</include>
> +        <include>org.eclipse.jetty:*</include>
> +        <include>com.googlecode.concurrentlinkedhashmap:concurrentlinkedhashmap-lru</include>
> +        <include>jline:jline</include>
> +        <include>com.fasterxml.jackson.core:*</include>
> +      </includes>
> +    </dependencySet>
> +  </dependencySets>
> +
> +  <fileSets>
> +    <fileSet>
> +      <directory>${project.build.scriptSourceDirectory}</directory>
> +      <outputDirectory>blur-indexer-${project.version}</outputDirectory>
> +      <excludes>
> +        <exclude>**/.empty</exclude>
> +      </excludes>
> +    </fileSet>
> +  </fileSets>
> +</assembly>
>
> http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
> ----------------------------------------------------------------------
> diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
> new file mode 100644
> index 0000000..a9caabb
> --- /dev/null
> +++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
> @@ -0,0 +1,17 @@
> +package org.apache.blur.mapreduce.lib.update;
> +
> +public enum BlurIndexCounter {
> +
> +  NEW_RECORDS,
> +  ROW_IDS_FROM_INDEX,
> +  ROW_IDS_TO_UPDATE_FROM_NEW_DATA,
> +  ROW_IDS_FROM_NEW_DATA,
> +
> +  INPUT_FORMAT_MAPPER,
> +  INPUT_FORMAT_EXISTING_RECORDS,
> +
> +  LOOKUP_MAPPER,
> +  LOOKUP_MAPPER_EXISTING_RECORDS,
> +  LOOKUP_MAPPER_ROW_LOOKUP_ATTEMPT
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
> ----------------------------------------------------------------------
> diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
> new file mode 100644
> index 0000000..d44adf1
> --- /dev/null
> +++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
> @@ -0,0 +1,362 @@
> +package org.apache.blur.mapreduce.lib.update;
> +
> +import java.io.ByteArrayInputStream;
> +import java.io.ByteArrayOutputStream;
> +import java.io.IOException;
> +import java.io.InputStream;
> +import java.net.URL;
> +import java.util.HashMap;
> +import java.util.HashSet;
> +import java.util.List;
> +import java.util.Map;
> +import java.util.Map.Entry;
> +import java.util.Set;
> +import java.util.UUID;
> +import java.util.concurrent.Callable;
> +import java.util.concurrent.ExecutionException;
> +import java.util.concurrent.ExecutorService;
> +import java.util.concurrent.Executors;
> +import java.util.concurrent.Future;
> +import java.util.concurrent.TimeUnit;
> +import java.util.concurrent.atomic.AtomicBoolean;
> +
> +import org.apache.blur.log.Log;
> +import org.apache.blur.log.LogFactory;
> +import org.apache.blur.mapreduce.lib.BlurInputFormat;
> +import org.apache.blur.thirdparty.thrift_0_9_0.TException;
> +import org.apache.blur.thrift.BlurClient;
> +import org.apache.blur.thrift.generated.Blur.Iface;
> +import org.apache.blur.thrift.generated.BlurException;
> +import org.apache.blur.thrift.generated.TableDescriptor;
> +import org.apache.blur.thrift.generated.TableStats;
> +import org.apache.blur.utils.BlurConstants;
> +import org.apache.commons.io.IOUtils;
> +import org.apache.hadoop.conf.Configuration;
> +import org.apache.hadoop.conf.Configured;
> +import org.apache.hadoop.fs.FSDataInputStream;
> +import org.apache.hadoop.fs.FileStatus;
> +import org.apache.hadoop.fs.FileSystem;
> +import org.apache.hadoop.fs.Path;
> +import org.apache.hadoop.fs.permission.FsAction;
> +import org.apache.hadoop.mapreduce.Cluster;
> +import org.apache.hadoop.mapreduce.Job;
> +import org.apache.hadoop.mapreduce.JobID;
> +import org.apache.hadoop.mapreduce.JobStatus;
> +import org.apache.hadoop.util.Tool;
> +import org.apache.hadoop.util.ToolRunner;
> +import org.apache.hadoop.yarn.exceptions.YarnException;
> +import org.apache.log4j.LogManager;
> +import org.apache.log4j.xml.DOMConfigurator;
> +
> +public class ClusterDriver extends Configured implements Tool {
> +
> +  private static final String BLUR_ENV = "blur.env";
> +  private static final Log LOG = LogFactory.getLog(ClusterDriver.class);
> +  private static final String _SEP = "_";
> +  private static final String IMPORT = "import";
> +
> +  public static void main(String[] args) throws Exception {
> +    String logFilePath = System.getenv("BLUR_INDEXER_LOG_FILE");
> +    System.out.println("Log file path [" + logFilePath + "]");
> +    System.setProperty("BLUR_INDEXER_LOG_FILE", logFilePath);
> +    URL url = ClusterDriver.class.getResource("/program-log4j.xml");
> +    if (url != null) {
> +      LOG.info("Reseting log4j config from classpath resource [{0}]", url);
> +      LogManager.resetConfiguration();
> +      DOMConfigurator.configure(url);
> +    }
> +    int res = ToolRunner.run(new Configuration(), new ClusterDriver(), args);

Not sure what this thing does yet but it seems we should validate
those args since their accessed blindly in run...

--tim

[04/13] git commit: Third round of updates.

Posted by am...@apache.org.
Third round of updates.


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

Branch: refs/heads/master
Commit: ea50630a38d67675a61a916b144f3c0ce85d7f7a
Parents: 0141656
Author: Aaron McCurry <am...@gmail.com>
Authored: Sat May 7 13:11:54 2016 -0400
Committer: Aaron McCurry <am...@gmail.com>
Committed: Sat May 7 13:11:54 2016 -0400

----------------------------------------------------------------------
 blur-indexer/pom.xml                            |  58 +++
 blur-indexer/src/main/assemble/bin.xml          |  45 ++
 .../mapreduce/lib/update/BlurIndexCounter.java  |  17 +
 .../mapreduce/lib/update/ClusterDriver.java     | 362 ++++++++++++++
 .../blur/mapreduce/lib/update/FasterDriver.java | 486 +++++++++++++++++++
 .../update/HdfsConfigurationNamespaceMerge.java | 115 +++++
 .../lib/update/InputSplitPruneUtil.java         | 133 +++++
 .../lib/update/LookupBuilderMapper.java         |  18 +
 .../lib/update/LookupBuilderReducer.java        | 165 +++++++
 .../lib/update/MapperForExistingDataMod.java    |  46 ++
 .../MapperForExistingDataWithIndexLookup.java   | 228 +++++++++
 .../lib/update/MapperForNewDataMod.java         |  82 ++++
 .../lib/update/MergeSortRowIdMatcher.java       | 372 ++++++++++++++
 .../lib/update/PrunedBlurInputFormat.java       |  57 +++
 .../update/PrunedSequenceFileInputFormat.java   |  59 +++
 .../src/main/resources/blur-site.properties     |   1 +
 .../src/main/resources/program-log4j.xml        |  29 ++
 blur-indexer/src/main/resources/test-log4j.xml  |  46 ++
 18 files changed, 2319 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/pom.xml
----------------------------------------------------------------------
diff --git a/blur-indexer/pom.xml b/blur-indexer/pom.xml
new file mode 100644
index 0000000..c7c1753
--- /dev/null
+++ b/blur-indexer/pom.xml
@@ -0,0 +1,58 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<modelVersion>4.0.0</modelVersion>
+	<groupId>org.apache.blur</groupId>
+	<artifactId>blur-indexer</artifactId>
+	<version>0.2.8</version>
+	<name>blur-indexer</name>
+	<packaging>jar</packaging>
+
+	<properties>
+		<blur.version>0.3.0.incubating.2.5.0.cdh5.3.3-SNAPSHOT</blur.version>
+	</properties>
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.blur</groupId>
+			<artifactId>blur-mapred</artifactId>
+			<version>${blur.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<version>4.9</version>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<pluginManagement>
+			<plugins>
+				<plugin>
+					<groupId>org.apache.maven.plugins</groupId>
+					<artifactId>maven-compiler-plugin</artifactId>
+					<configuration>
+						<source>1.8</source>
+						<target>1.8</target>
+					</configuration>
+				</plugin>
+			</plugins>
+		</pluginManagement>
+		<plugins>
+			<plugin>
+				<artifactId>maven-assembly-plugin</artifactId>
+				<configuration>
+					<descriptor>src/main/assemble/bin.xml</descriptor>
+					<finalName>blur-indexer-${project.version}</finalName>
+				</configuration>
+				<executions>
+					<execution>
+						<phase>package</phase>
+						<goals>
+							<goal>single</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/assemble/bin.xml
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/assemble/bin.xml b/blur-indexer/src/main/assemble/bin.xml
new file mode 100644
index 0000000..5fddd56
--- /dev/null
+++ b/blur-indexer/src/main/assemble/bin.xml
@@ -0,0 +1,45 @@
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	    xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
+  <formats>
+    <format>tar.gz</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+
+  <dependencySets>
+    <dependencySet>
+      <useProjectArtifact>true</useProjectArtifact>
+      <outputDirectory>blur-indexer-${project.version}/lib</outputDirectory>
+      <unpack>false</unpack>
+      <includes>
+        <include>org.apache.blur:blur-indexer</include>
+        <include>org.apache.blur:*</include>
+        <include>org.apache.zookeeper:zookeeper</include>
+        <include>org.slf4j:slf4j-api</include>
+        <include>org.slf4j:slf4j-log4j12</include>
+        <include>org.json:json</include>
+        <include>log4j:log4j</include>
+        <include>com.yammer.metrics:*</include>
+        <include>com.google.guava:guava</include>
+        <include>org.apache.httpcomponents:*</include>
+        <include>org.apache.lucene:*</include>
+        <include>com.spatial4j:spatial4j</include>
+        <include>commons-cli:commons-cli</include>
+        <include>org.eclipse.jetty:*</include>
+        <include>com.googlecode.concurrentlinkedhashmap:concurrentlinkedhashmap-lru</include>
+        <include>jline:jline</include>
+        <include>com.fasterxml.jackson.core:*</include>
+      </includes>
+    </dependencySet>
+  </dependencySets>
+
+  <fileSets>
+    <fileSet>
+      <directory>${project.build.scriptSourceDirectory}</directory>
+      <outputDirectory>blur-indexer-${project.version}</outputDirectory>
+      <excludes>
+        <exclude>**/.empty</exclude>
+      </excludes>
+    </fileSet>
+  </fileSets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
new file mode 100644
index 0000000..a9caabb
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
@@ -0,0 +1,17 @@
+package org.apache.blur.mapreduce.lib.update;
+
+public enum BlurIndexCounter {
+
+  NEW_RECORDS,
+  ROW_IDS_FROM_INDEX,
+  ROW_IDS_TO_UPDATE_FROM_NEW_DATA,
+  ROW_IDS_FROM_NEW_DATA,
+  
+  INPUT_FORMAT_MAPPER, 
+  INPUT_FORMAT_EXISTING_RECORDS,
+  
+  LOOKUP_MAPPER, 
+  LOOKUP_MAPPER_EXISTING_RECORDS, 
+  LOOKUP_MAPPER_ROW_LOOKUP_ATTEMPT
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
new file mode 100644
index 0000000..d44adf1
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
@@ -0,0 +1,362 @@
+package org.apache.blur.mapreduce.lib.update;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.blur.log.Log;
+import org.apache.blur.log.LogFactory;
+import org.apache.blur.mapreduce.lib.BlurInputFormat;
+import org.apache.blur.thirdparty.thrift_0_9_0.TException;
+import org.apache.blur.thrift.BlurClient;
+import org.apache.blur.thrift.generated.Blur.Iface;
+import org.apache.blur.thrift.generated.BlurException;
+import org.apache.blur.thrift.generated.TableDescriptor;
+import org.apache.blur.thrift.generated.TableStats;
+import org.apache.blur.utils.BlurConstants;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.xml.DOMConfigurator;
+
+public class ClusterDriver extends Configured implements Tool {
+
+  private static final String BLUR_ENV = "blur.env";
+  private static final Log LOG = LogFactory.getLog(ClusterDriver.class);
+  private static final String _SEP = "_";
+  private static final String IMPORT = "import";
+
+  public static void main(String[] args) throws Exception {
+    String logFilePath = System.getenv("BLUR_INDEXER_LOG_FILE");
+    System.out.println("Log file path [" + logFilePath + "]");
+    System.setProperty("BLUR_INDEXER_LOG_FILE", logFilePath);
+    URL url = ClusterDriver.class.getResource("/program-log4j.xml");
+    if (url != null) {
+      LOG.info("Reseting log4j config from classpath resource [{0}]", url);
+      LogManager.resetConfiguration();
+      DOMConfigurator.configure(url);
+    }
+    int res = ToolRunner.run(new Configuration(), new ClusterDriver(), args);
+    System.exit(res);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    int c = 0;
+    final String blurEnv = args[c++];
+    final String blurZkConnection = args[c++];
+    final String extraConfig = args[c++];
+    final int reducerMultiplier = Integer.parseInt(args[c++]);
+    final Configuration conf = getConf();
+
+    final ExecutorService service = Executors.newCachedThreadPool();
+    final AtomicBoolean running = new AtomicBoolean();
+    running.set(true);
+
+    // Load configs for all filesystems.
+    Path path = new Path(extraConfig);
+    Configuration mergeHdfsConfigs = HdfsConfigurationNamespaceMerge.mergeHdfsConfigs(path.getFileSystem(conf), path);
+    conf.addResource(mergeHdfsConfigs);
+    conf.set(BlurConstants.BLUR_ZOOKEEPER_CONNECTION, blurZkConnection);
+    conf.set(BLUR_ENV, blurEnv);
+
+    final Iface client = BlurClient.getClientFromZooKeeperConnectionStr(blurZkConnection);
+
+    stopAllExistingMRJobs(blurEnv, conf);
+    cleanUpOldImportDirs(client, conf);
+    moveInprogressDirsBackToNew(client, conf);
+    unlockLockedTables(client);
+
+    Map<String, Future<Void>> futures = new HashMap<String, Future<Void>>();
+    while (running.get()) {
+      LOG.debug("Starting index update check for blur cluster [" + blurZkConnection + "].");
+      try {
+        List<String> tableList = client.tableList();
+        startMissingIndexerThreads(tableList, service, futures, blurZkConnection, conf, client, reducerMultiplier);
+      } catch (TException t) {
+        LOG.error("Unknown Blur Thrift Error, Retrying...", t);
+      }
+      Thread.sleep(TimeUnit.SECONDS.toMillis(10));
+    }
+    return 0;
+  }
+
+  private void unlockLockedTables(Iface client) throws BlurException, TException {
+    List<String> tableList = client.tableList();
+    for (String table : tableList) {
+      TableDescriptor tableDescriptor = client.describe(table);
+      if (tableDescriptor.isEnabled()) {
+        unlockLockedTables(client, table);
+      }
+    }
+  }
+
+  private void unlockLockedTables(Iface client, String table) throws BlurException, TException {
+    Map<String, List<String>> listSnapshots = client.listSnapshots(table);
+    for (Entry<String, List<String>> e : listSnapshots.entrySet()) {
+      List<String> value = e.getValue();
+      if (value.contains(FasterDriver.MRUPDATE_SNAPSHOT)) {
+        LOG.info("Unlocking table [{0}]", table);
+        client.removeSnapshot(table, FasterDriver.MRUPDATE_SNAPSHOT);
+        return;
+      }
+    }
+  }
+
+  private void moveInprogressDirsBackToNew(Iface client, Configuration conf) throws BlurException, TException,
+      IOException {
+    List<String> tableList = client.tableList();
+    for (String table : tableList) {
+      String mrIncWorkingPathStr = getMRIncWorkingPathStr(client, table);
+      Path mrIncWorkingPath = new Path(mrIncWorkingPathStr);
+      Path newData = new Path(mrIncWorkingPath, FasterDriver.NEW);
+      Path inprogressData = new Path(mrIncWorkingPath, FasterDriver.INPROGRESS);
+      FileSystem fileSystem = inprogressData.getFileSystem(conf);
+      FileStatus[] listStatus = fileSystem.listStatus(inprogressData);
+      for (FileStatus fileStatus : listStatus) {
+        Path src = fileStatus.getPath();
+        Path dst = new Path(newData, src.getName());
+        if (fileSystem.rename(src, dst)) {
+          LOG.info("Moved [{0}] to [{1}] to be reprocessed.", src, dst);
+        } else {
+          LOG.error("Could not move [{0}] to [{1}] to be reprocessed.", src, dst);
+        }
+      }
+    }
+  }
+
+  private void cleanUpOldImportDirs(Iface client, Configuration conf) throws BlurException, TException, IOException {
+    List<String> tableList = client.tableList();
+    for (String table : tableList) {
+      cleanUpOldImportDirs(client, conf, table);
+    }
+  }
+
+  private void cleanUpOldImportDirs(Iface client, Configuration conf, String table) throws BlurException, TException,
+      IOException {
+    TableDescriptor descriptor = client.describe(table);
+    String tableUri = descriptor.getTableUri();
+    Path tablePath = new Path(tableUri);
+    FileSystem fileSystem = tablePath.getFileSystem(getConf());
+    Path importPath = new Path(tablePath, IMPORT);
+    if (fileSystem.exists(importPath)) {
+      for (FileStatus fileStatus : fileSystem.listStatus(importPath)) {
+        Path path = fileStatus.getPath();
+        LOG.info("Removing failed import [{0}]", path);
+        fileSystem.delete(path, true);
+      }
+    }
+  }
+
+  private void stopAllExistingMRJobs(String blurEnv, Configuration conf) throws YarnException, IOException,
+      InterruptedException {
+    Cluster cluster = new Cluster(conf);
+    JobStatus[] allJobStatuses = cluster.getAllJobStatuses();
+    for (JobStatus jobStatus : allJobStatuses) {
+      if (jobStatus.isJobComplete()) {
+        continue;
+      }
+      String jobFile = jobStatus.getJobFile();
+      JobID jobID = jobStatus.getJobID();
+      Job job = cluster.getJob(jobID);
+      FileSystem fileSystem = FileSystem.get(job.getConfiguration());
+      Configuration configuration = new Configuration(false);
+      Path path = new Path(jobFile);
+      Path makeQualified = path.makeQualified(fileSystem.getUri(), fileSystem.getWorkingDirectory());
+      if (hasReadAccess(fileSystem, makeQualified)) {
+        try (FSDataInputStream in = fileSystem.open(makeQualified)) {
+          configuration.addResource(copy(in));
+        }
+        String jobBlurEnv = configuration.get(BLUR_ENV);
+        LOG.info("Checking job [{0}] has env [{1}] current env set to [{2}]", jobID, jobBlurEnv, blurEnv);
+        if (blurEnv.equals(jobBlurEnv)) {
+          LOG.info("Killing running job [{0}]", jobID);
+          job.killJob();
+        }
+      }
+    }
+  }
+
+  private static InputStream copy(FSDataInputStream input) throws IOException {
+    try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) {
+      IOUtils.copy(input, outputStream);
+      return new ByteArrayInputStream(outputStream.toByteArray());
+    }
+  }
+
+  private static boolean hasReadAccess(FileSystem fileSystem, Path p) {
+    try {
+      fileSystem.access(p, FsAction.READ);
+      return true;
+    } catch (IOException e) {
+      return false;
+    }
+  }
+
+  private Callable<Void> getCallable(final String blurZkConnection, final Configuration conf, final Iface client,
+      final String table, final int reducerMultiplier) {
+    return new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        String originalThreadName = Thread.currentThread().getName();
+        try {
+          Thread.currentThread().setName(table);
+          if (!isEnabled(client, table)) {
+            LOG.info("Table [" + table + "] is not enabled.");
+            return null;
+          }
+          waitForDataToLoad(client, table);
+          LOG.debug("Starting index update for table [" + table + "].");
+          final String mrIncWorkingPathStr = getMRIncWorkingPathStr(client, table);
+          final String outputPathStr = getOutputPathStr(client, table);
+          Path path = new Path(outputPathStr);
+          FileSystem fileSystem = path.getFileSystem(getConf());
+
+          Configuration configuration = new Configuration(conf);
+          BlurInputFormat.setMaxNumberOfMaps(configuration, 10000);
+
+          FasterDriver driver = new FasterDriver();
+          driver.setConf(configuration);
+          try {
+            driver.run(new String[] { table, mrIncWorkingPathStr, outputPathStr, blurZkConnection,
+                Integer.toString(reducerMultiplier) });
+          } finally {
+            if (fileSystem.exists(path)) {
+              fileSystem.delete(path, true);
+            }
+          }
+          return null;
+        } finally {
+          Thread.currentThread().setName(originalThreadName);
+        }
+      }
+    };
+  }
+
+  private void startMissingIndexerThreads(List<String> tableList, ExecutorService service,
+      Map<String, Future<Void>> futures, final String blurZkConnection, final Configuration conf, final Iface client,
+      int reducerMultiplier) throws BlurException, TException {
+    Set<String> tables = new HashSet<String>(tableList);
+
+    // remove futures that are complete
+    for (String table : tables) {
+      Future<Void> future = futures.get(table);
+      if (future != null) {
+        if (future.isDone()) {
+          try {
+            future.get();
+          } catch (InterruptedException e) {
+            LOG.error("Unknown error while processing table [" + table + "].", e);
+          } catch (ExecutionException e) {
+            LOG.error("Unknown error while processing table [" + table + "].", e.getCause());
+          }
+          futures.remove(table);
+        } else {
+          LOG.info("Update for table [" + table + "] still running.");
+        }
+      }
+    }
+
+    // start missing tables
+    for (String table : tables) {
+      if (!futures.containsKey(table)) {
+        if (isEnabled(client, table)) {
+          Future<Void> future = service.submit(getCallable(blurZkConnection, conf, client, table, reducerMultiplier));
+          futures.put(table, future);
+        }
+      }
+    }
+  }
+
+  public static void waitForDataToLoad(Iface client, String table) throws BlurException, TException,
+      InterruptedException {
+    if (isFullyLoaded(client.tableStats(table))) {
+      return;
+    }
+    while (true) {
+      TableStats tableStats = client.tableStats(table);
+      if (isFullyLoaded(tableStats)) {
+        LOG.info("Data load complete in table [" + table + "] [" + tableStats + "]");
+        return;
+      }
+      LOG.info("Waiting for data to load in table [" + table + "] [" + tableStats + "]");
+      Thread.sleep(5000);
+    }
+  }
+
+  private static boolean isFullyLoaded(TableStats tableStats) {
+    if (tableStats.getSegmentImportInProgressCount() == 0 && tableStats.getSegmentImportPendingCount() == 0) {
+      return true;
+    }
+    return false;
+  }
+
+  private boolean isEnabled(Iface client, String table) throws BlurException, TException {
+    TableDescriptor tableDescriptor = client.describe(table);
+    return tableDescriptor.isEnabled();
+  }
+
+  private void mkdirs(FileSystem fileSystem, Path path) throws IOException {
+    if (fileSystem.exists(path)) {
+      return;
+    }
+    LOG.info("Creating path [" + path + "].");
+    if (!fileSystem.mkdirs(path)) {
+      LOG.error("Path [" + path + "] could not be created.");
+    }
+  }
+
+  public static String getMRIncWorkingPathStr(Iface client, String table) throws BlurException, TException, IOException {
+    TableDescriptor descriptor = client.describe(table);
+    Map<String, String> tableProperties = descriptor.getTableProperties();
+    if (tableProperties != null) {
+      String workingPath = tableProperties.get(BlurConstants.BLUR_BULK_UPDATE_WORKING_PATH);
+      if (workingPath != null) {
+        return workingPath;
+      }
+    }
+    throw new IOException("Table [" + table + "] does not have the property ["
+        + BlurConstants.BLUR_BULK_UPDATE_WORKING_PATH + "] setup correctly.");
+  }
+
+  private String getOutputPathStr(Iface client, String table) throws BlurException, TException, IOException {
+    TableDescriptor descriptor = client.describe(table);
+    String tableUri = descriptor.getTableUri();
+    Path tablePath = new Path(tableUri);
+    FileSystem fileSystem = tablePath.getFileSystem(getConf());
+    Path importPath = new Path(tablePath, IMPORT);
+    mkdirs(fileSystem, importPath);
+    return new Path(importPath, IMPORT + _SEP + System.currentTimeMillis() + _SEP + UUID.randomUUID().toString())
+        .toString();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/FasterDriver.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/FasterDriver.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/FasterDriver.java
new file mode 100644
index 0000000..f43cba5
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/FasterDriver.java
@@ -0,0 +1,486 @@
+/**
+ * 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.blur.mapreduce.lib.update;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.blur.log.Log;
+import org.apache.blur.log.LogFactory;
+import org.apache.blur.mapreduce.lib.BlurInputFormat;
+import org.apache.blur.mapreduce.lib.BlurOutputFormat;
+import org.apache.blur.thirdparty.thrift_0_9_0.TException;
+import org.apache.blur.thrift.BlurClient;
+import org.apache.blur.thrift.generated.Blur.Iface;
+import org.apache.blur.thrift.generated.BlurException;
+import org.apache.blur.thrift.generated.TableDescriptor;
+import org.apache.blur.thrift.generated.TableStats;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.TaskReport;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+public class FasterDriver extends Configured implements Tool {
+
+  public static final String BLUR_UPDATE_ID = "blur.update.id";
+  private static final String BLUR_EXEC_TYPE = "blur.exec.type";
+  public static final String TMP = "tmp";
+
+  public enum EXEC {
+    MR_ONLY, MR_WITH_LOOKUP, AUTOMATIC
+  }
+
+  public static final String MRUPDATE_SNAPSHOT = "mrupdate-snapshot";
+  public static final String CACHE = "cache";
+  public static final String COMPLETE = "complete";
+  public static final String INPROGRESS = "inprogress";
+  public static final String NEW = "new";
+  private static final Log LOG = LogFactory.getLog(FasterDriver.class);
+
+  public static void main(String[] args) throws Exception {
+    int res = ToolRunner.run(new Configuration(), new FasterDriver(), args);
+    System.exit(res);
+  }
+
+  static class PartitionedInputResult {
+    final Path _partitionedInputData;
+    final Counters _counters;
+    final long[] _rowIdsFromNewData;
+    final long[] _rowIdsToUpdateFromNewData;
+    final long[] _rowIdsFromIndex;
+
+    PartitionedInputResult(Path partitionedInputData, Counters counters, int shards, TaskReport[] taskReports) {
+      _partitionedInputData = partitionedInputData;
+      _counters = counters;
+      _rowIdsFromNewData = new long[shards];
+      _rowIdsToUpdateFromNewData = new long[shards];
+      _rowIdsFromIndex = new long[shards];
+      for (TaskReport tr : taskReports) {
+        int id = tr.getTaskID().getId();
+        Counters taskCounters = tr.getTaskCounters();
+        Counter total = taskCounters.findCounter(BlurIndexCounter.ROW_IDS_FROM_NEW_DATA);
+        _rowIdsFromNewData[id] = total.getValue();
+        Counter update = taskCounters.findCounter(BlurIndexCounter.ROW_IDS_TO_UPDATE_FROM_NEW_DATA);
+        _rowIdsToUpdateFromNewData[id] = update.getValue();
+        Counter index = taskCounters.findCounter(BlurIndexCounter.ROW_IDS_FROM_INDEX);
+        _rowIdsFromIndex[id] = index.getValue();
+      }
+    }
+
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    int c = 0;
+    if (args.length < 5) {
+      System.err
+          .println("Usage Driver <table> <mr inc working path> <output path> <zk connection> <reducer multipler> <extra config files...>");
+      return 1;
+    }
+    String table = args[c++];
+    String mrIncWorkingPathStr = args[c++];
+    String outputPathStr = args[c++];
+    String blurZkConnection = args[c++];
+    int reducerMultipler = Integer.parseInt(args[c++]);
+    for (; c < args.length; c++) {
+      String externalConfigFileToAdd = args[c];
+      getConf().addResource(new Path(externalConfigFileToAdd));
+    }
+
+    Path outputPath = new Path(outputPathStr);
+    Path mrIncWorkingPath = new Path(mrIncWorkingPathStr);
+    FileSystem fileSystem = mrIncWorkingPath.getFileSystem(getConf());
+
+    Path newData = new Path(mrIncWorkingPath, NEW);
+    Path inprogressData = new Path(mrIncWorkingPath, INPROGRESS);
+    Path completeData = new Path(mrIncWorkingPath, COMPLETE);
+    Path fileCache = new Path(mrIncWorkingPath, CACHE);
+    Path tmpPathDontDelete = new Path(mrIncWorkingPath, TMP);
+
+    Path tmpPath = new Path(tmpPathDontDelete, UUID.randomUUID().toString());
+
+    fileSystem.mkdirs(newData);
+    fileSystem.mkdirs(inprogressData);
+    fileSystem.mkdirs(completeData);
+    fileSystem.mkdirs(fileCache);
+
+    List<Path> srcPathList = new ArrayList<Path>();
+    for (FileStatus fileStatus : fileSystem.listStatus(newData)) {
+      srcPathList.add(fileStatus.getPath());
+    }
+    if (srcPathList.isEmpty()) {
+      return 0;
+    }
+
+    List<Path> inprogressPathList = new ArrayList<Path>();
+    boolean success = false;
+    Iface client = null;
+
+    EXEC exec = EXEC.valueOf(getConf().get(BLUR_EXEC_TYPE, EXEC.AUTOMATIC.name()).toUpperCase());
+
+    String uuid = UUID.randomUUID().toString();
+
+    try {
+      client = BlurClient.getClientFromZooKeeperConnectionStr(blurZkConnection);
+      TableDescriptor descriptor = client.describe(table);
+      Map<String, String> tableProperties = descriptor.getTableProperties();
+      String fastDir = tableProperties.get("blur.table.disable.fast.dir");
+      if (fastDir == null || !fastDir.equals("true")) {
+        LOG.error("Table [{0}] has blur.table.disable.fast.dir enabled, not supported in fast MR update.", table);
+        return 1;
+      }
+
+      waitForOtherSnapshotsToBeRemoved(client, table, MRUPDATE_SNAPSHOT);
+      client.createSnapshot(table, MRUPDATE_SNAPSHOT);
+      TableStats tableStats = client.tableStats(table);
+
+      inprogressPathList = movePathList(fileSystem, inprogressData, srcPathList);
+
+      switch (exec) {
+      case MR_ONLY:
+        success = runMrOnly(descriptor, inprogressPathList, table, fileCache, outputPath, reducerMultipler);
+        break;
+      case MR_WITH_LOOKUP:
+        success = runMrWithLookup(uuid, descriptor, inprogressPathList, table, fileCache, outputPath, reducerMultipler,
+            tmpPath, tableStats, MRUPDATE_SNAPSHOT);
+        break;
+      case AUTOMATIC:
+        success = runAutomatic(uuid, descriptor, inprogressPathList, table, fileCache, outputPath, reducerMultipler,
+            tmpPath, tableStats, MRUPDATE_SNAPSHOT);
+        break;
+      default:
+        throw new RuntimeException("Exec type [" + exec + "] not supported.");
+      }
+    } finally {
+      if (success) {
+        LOG.info("Associate lookup cache with new data!");
+        associateLookupCache(uuid, fileCache, outputPath);
+        LOG.info("Indexing job succeeded!");
+        client.loadData(table, outputPathStr);
+        LOG.info("Load data called");
+        movePathList(fileSystem, completeData, inprogressPathList);
+        LOG.info("Input data moved to complete");
+        ClusterDriver.waitForDataToLoad(client, table);
+        LOG.info("Data loaded");
+      } else {
+        LOG.error("Indexing job failed!");
+        movePathList(fileSystem, newData, inprogressPathList);
+      }
+      fileSystem.delete(tmpPath, true);
+      if (client != null) {
+        client.removeSnapshot(table, MRUPDATE_SNAPSHOT);
+      }
+    }
+
+    if (success) {
+      return 0;
+    } else {
+      return 1;
+    }
+  }
+
+  private void associateLookupCache(String uuid, Path fileCache, Path outputPath) throws IOException {
+    FileSystem fileSystem = fileCache.getFileSystem(getConf());
+    cleanupExtraFileFromSpecX(fileSystem, uuid, fileCache);
+    associateLookupCache(fileSystem, uuid, fileSystem.getFileStatus(fileCache), outputPath);
+  }
+
+  private void cleanupExtraFileFromSpecX(FileSystem fileSystem, String uuid, Path fileCache) throws IOException {
+    FileStatus[] listStatus = fileSystem.listStatus(fileCache);
+    List<FileStatus> uuidPaths = new ArrayList<FileStatus>();
+    for (FileStatus fs : listStatus) {
+      Path path = fs.getPath();
+      if (fs.isDirectory()) {
+        cleanupExtraFileFromSpecX(fileSystem, uuid, path);
+      } else if (path.getName().startsWith(uuid)) {
+        uuidPaths.add(fs);
+      }
+    }
+    if (uuidPaths.size() > 1) {
+      deleteIncomplete(fileSystem, uuidPaths);
+    }
+  }
+
+  private void deleteIncomplete(FileSystem fileSystem, List<FileStatus> uuidPaths) throws IOException {
+    long max = 0;
+    FileStatus keeper = null;
+    for (FileStatus fs : uuidPaths) {
+      long len = fs.getLen();
+      if (len > max) {
+        keeper = fs;
+        max = len;
+      }
+    }
+    for (FileStatus fs : uuidPaths) {
+      if (fs != keeper) {
+        LOG.info("Deleteing incomplete cache file [{0}]", fs.getPath());
+        fileSystem.delete(fs.getPath(), false);
+      }
+    }
+  }
+
+  private void associateLookupCache(FileSystem fileSystem, String uuid, FileStatus fileCache, Path outputPath)
+      throws IOException {
+    Path path = fileCache.getPath();
+    if (fileCache.isDirectory()) {
+      FileStatus[] listStatus = fileSystem.listStatus(path);
+      for (FileStatus fs : listStatus) {
+        associateLookupCache(fileSystem, uuid, fs, outputPath);
+      }
+    } else if (path.getName().startsWith(uuid)) {
+      Path parent = path.getParent();
+      String shardName = parent.getName();
+      Path indexPath = findOutputDirPath(outputPath, shardName);
+      LOG.info("Path found for shard [{0}] outputPath [{1}]", shardName, outputPath);
+      String id = MergeSortRowIdMatcher.getIdForSingleSegmentIndex(getConf(), indexPath);
+      Path file = new Path(path.getParent(), id + ".seq");
+      MergeSortRowIdMatcher.commitWriter(getConf(), file, path);
+    }
+  }
+
+  private Path findOutputDirPath(Path outputPath, String shardName) throws IOException {
+    FileSystem fileSystem = outputPath.getFileSystem(getConf());
+    Path shardPath = new Path(outputPath, shardName);
+    if (!fileSystem.exists(shardPath)) {
+      throw new IOException("Shard path [" + shardPath + "]");
+    }
+    FileStatus[] listStatus = fileSystem.listStatus(shardPath, new PathFilter() {
+      @Override
+      public boolean accept(Path path) {
+        return path.getName().endsWith(".commit");          
+      }
+    });
+    if (listStatus.length == 1) {
+      FileStatus fs = listStatus[0];
+      return fs.getPath();
+    } else {
+      throw new IOException("More than one sub dir [" + shardPath + "]");
+    }
+  }
+
+  private boolean runAutomatic(String uuid, TableDescriptor descriptor, List<Path> inprogressPathList, String table,
+      Path fileCache, Path outputPath, int reducerMultipler, Path tmpPath, TableStats tableStats, String snapshot)
+      throws ClassNotFoundException, IOException, InterruptedException {
+    PartitionedInputResult result = buildPartitionedInputData(uuid, tmpPath, descriptor, inprogressPathList, snapshot,
+        fileCache);
+
+    Job job = Job.getInstance(getConf(), "Blur Row Updater for table [" + table + "]");
+
+    InputSplitPruneUtil.setBlurLookupRowIdFromNewDataCounts(job, table, result._rowIdsFromNewData);
+    InputSplitPruneUtil.setBlurLookupRowIdUpdateFromNewDataCounts(job, table, result._rowIdsToUpdateFromNewData);
+    InputSplitPruneUtil.setBlurLookupRowIdFromIndexCounts(job, table, result._rowIdsFromIndex);
+    InputSplitPruneUtil.setTable(job, table);
+
+    BlurInputFormat.setLocalCachePath(job, fileCache);
+
+    // Existing data - This adds the copy data files first open and stream
+    // through all documents.
+    {
+      Path tablePath = new Path(descriptor.getTableUri());
+      BlurInputFormat.addTable(job, descriptor, MRUPDATE_SNAPSHOT);
+      MultipleInputs.addInputPath(job, tablePath, PrunedBlurInputFormat.class, MapperForExistingDataMod.class);
+    }
+
+    // Existing data - This adds the row id lookup
+    {
+      MapperForExistingDataWithIndexLookup.setSnapshot(job, MRUPDATE_SNAPSHOT);
+      FileInputFormat.addInputPath(job, result._partitionedInputData);
+      MultipleInputs.addInputPath(job, result._partitionedInputData, PrunedSequenceFileInputFormat.class,
+          MapperForExistingDataWithIndexLookup.class);
+    }
+
+    // New Data
+    for (Path p : inprogressPathList) {
+      FileInputFormat.addInputPath(job, p);
+      MultipleInputs.addInputPath(job, p, SequenceFileInputFormat.class, MapperForNewDataMod.class);
+    }
+
+    BlurOutputFormat.setOutputPath(job, outputPath);
+    BlurOutputFormat.setupJob(job, descriptor);
+
+    job.setReducerClass(UpdateReducer.class);
+    job.setMapOutputKeyClass(IndexKey.class);
+    job.setMapOutputValueClass(IndexValue.class);
+    job.setPartitionerClass(IndexKeyPartitioner.class);
+    job.setGroupingComparatorClass(IndexKeyWritableComparator.class);
+
+    BlurOutputFormat.setReducerMultiplier(job, reducerMultipler);
+
+    boolean success = job.waitForCompletion(true);
+    Counters counters = job.getCounters();
+    LOG.info("Counters [" + counters + "]");
+    return success;
+  }
+
+  private boolean runMrWithLookup(String uuid, TableDescriptor descriptor, List<Path> inprogressPathList, String table,
+      Path fileCache, Path outputPath, int reducerMultipler, Path tmpPath, TableStats tableStats, String snapshot)
+      throws ClassNotFoundException, IOException, InterruptedException {
+    PartitionedInputResult result = buildPartitionedInputData(uuid, tmpPath, descriptor, inprogressPathList, snapshot,
+        fileCache);
+
+    Job job = Job.getInstance(getConf(), "Blur Row Updater for table [" + table + "]");
+
+    MapperForExistingDataWithIndexLookup.setSnapshot(job, MRUPDATE_SNAPSHOT);
+    FileInputFormat.addInputPath(job, result._partitionedInputData);
+    MultipleInputs.addInputPath(job, result._partitionedInputData, SequenceFileInputFormat.class,
+        MapperForExistingDataWithIndexLookup.class);
+
+    for (Path p : inprogressPathList) {
+      FileInputFormat.addInputPath(job, p);
+      MultipleInputs.addInputPath(job, p, SequenceFileInputFormat.class, MapperForNewDataMod.class);
+    }
+
+    BlurOutputFormat.setOutputPath(job, outputPath);
+    BlurOutputFormat.setupJob(job, descriptor);
+
+    job.setReducerClass(UpdateReducer.class);
+    job.setMapOutputKeyClass(IndexKey.class);
+    job.setMapOutputValueClass(IndexValue.class);
+    job.setPartitionerClass(IndexKeyPartitioner.class);
+    job.setGroupingComparatorClass(IndexKeyWritableComparator.class);
+
+    BlurOutputFormat.setReducerMultiplier(job, reducerMultipler);
+
+    boolean success = job.waitForCompletion(true);
+    Counters counters = job.getCounters();
+    LOG.info("Counters [" + counters + "]");
+    return success;
+  }
+
+  private boolean runMrOnly(TableDescriptor descriptor, List<Path> inprogressPathList, String table, Path fileCache,
+      Path outputPath, int reducerMultipler) throws IOException, ClassNotFoundException, InterruptedException {
+    Job job = Job.getInstance(getConf(), "Blur Row Updater for table [" + table + "]");
+    Path tablePath = new Path(descriptor.getTableUri());
+    BlurInputFormat.setLocalCachePath(job, fileCache);
+    BlurInputFormat.addTable(job, descriptor, MRUPDATE_SNAPSHOT);
+    MultipleInputs.addInputPath(job, tablePath, BlurInputFormat.class, MapperForExistingDataMod.class);
+
+    for (Path p : inprogressPathList) {
+      FileInputFormat.addInputPath(job, p);
+      MultipleInputs.addInputPath(job, p, SequenceFileInputFormat.class, MapperForNewDataMod.class);
+    }
+
+    BlurOutputFormat.setOutputPath(job, outputPath);
+    BlurOutputFormat.setupJob(job, descriptor);
+
+    job.setReducerClass(UpdateReducer.class);
+    job.setMapOutputKeyClass(IndexKey.class);
+    job.setMapOutputValueClass(IndexValue.class);
+    job.setPartitionerClass(IndexKeyPartitioner.class);
+    job.setGroupingComparatorClass(IndexKeyWritableComparator.class);
+
+    BlurOutputFormat.setReducerMultiplier(job, reducerMultipler);
+
+    boolean success = job.waitForCompletion(true);
+    Counters counters = job.getCounters();
+    LOG.info("Counters [" + counters + "]");
+    return success;
+  }
+
+  private PartitionedInputResult buildPartitionedInputData(String uuid, Path tmpPath, TableDescriptor descriptor,
+      List<Path> inprogressPathList, String snapshot, Path fileCachePath) throws IOException, ClassNotFoundException,
+      InterruptedException {
+    Job job = Job.getInstance(getConf(), "Partitioning data for table [" + descriptor.getName() + "]");
+    job.getConfiguration().set(BLUR_UPDATE_ID, uuid);
+
+    // Needed for the bloom filter path information.
+    BlurOutputFormat.setTableDescriptor(job, descriptor);
+    BlurInputFormat.setLocalCachePath(job, fileCachePath);
+    MapperForExistingDataWithIndexLookup.setSnapshot(job, snapshot);
+
+    for (Path p : inprogressPathList) {
+      FileInputFormat.addInputPath(job, p);
+    }
+    Path outputPath = new Path(tmpPath, UUID.randomUUID().toString());
+    job.setJarByClass(getClass());
+    job.setMapperClass(LookupBuilderMapper.class);
+    job.setReducerClass(LookupBuilderReducer.class);
+
+    int shardCount = descriptor.getShardCount();
+    job.setNumReduceTasks(shardCount);
+    job.setInputFormatClass(SequenceFileInputFormat.class);
+    job.setMapOutputKeyClass(Text.class);
+    job.setMapOutputValueClass(NullWritable.class);
+    job.setOutputFormatClass(SequenceFileOutputFormat.class);
+    job.setOutputKeyClass(Text.class);
+    job.setOutputValueClass(BooleanWritable.class);
+    FileOutputFormat.setOutputPath(job, outputPath);
+    if (job.waitForCompletion(true)) {
+      return new PartitionedInputResult(outputPath, job.getCounters(), shardCount, job.getTaskReports(TaskType.REDUCE));
+    } else {
+      throw new IOException("Partitioning failed!");
+    }
+  }
+
+  private void waitForOtherSnapshotsToBeRemoved(Iface client, String table, String snapshot) throws BlurException,
+      TException, InterruptedException {
+    while (true) {
+      Map<String, List<String>> listSnapshots = client.listSnapshots(table);
+      boolean mrupdateSnapshots = false;
+      for (Entry<String, List<String>> e : listSnapshots.entrySet()) {
+        List<String> value = e.getValue();
+        if (value.contains(snapshot)) {
+          mrupdateSnapshots = true;
+        }
+      }
+      if (!mrupdateSnapshots) {
+        return;
+      } else {
+        LOG.info(snapshot + " Snapshot for table [{0}] already exists", table);
+        Thread.sleep(TimeUnit.SECONDS.toMillis(10));
+        LOG.info("Retrying");
+      }
+    }
+  }
+
+  private List<Path> movePathList(FileSystem fileSystem, Path dstDir, List<Path> lst) throws IOException {
+    List<Path> result = new ArrayList<Path>();
+    for (Path src : lst) {
+      Path dst = new Path(dstDir, src.getName());
+      if (fileSystem.rename(src, dst)) {
+        LOG.info("Moving [{0}] to [{1}]", src, dst);
+        result.add(dst);
+      } else {
+        LOG.error("Could not move [{0}] to [{1}]", src, dst);
+      }
+    }
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/HdfsConfigurationNamespaceMerge.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/HdfsConfigurationNamespaceMerge.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/HdfsConfigurationNamespaceMerge.java
new file mode 100644
index 0000000..34d3e99
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/HdfsConfigurationNamespaceMerge.java
@@ -0,0 +1,115 @@
+package org.apache.blur.mapreduce.lib.update;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class HdfsConfigurationNamespaceMerge {
+
+  private static final String DFS_NAMESERVICES = "dfs.nameservices";
+  private static final Log LOG = LogFactory.getLog(HdfsConfigurationNamespaceMerge.class);
+
+  public static void main(String[] args) throws IOException {
+    Path p = new Path("./src/main/scripts/conf/hdfs");
+
+    Configuration configuration = mergeHdfsConfigs(p.getFileSystem(new Configuration()), p);
+
+    // configuration.writeXml(System.out);
+
+    Collection<String> nameServices = configuration.getStringCollection(DFS_NAMESERVICES);
+    for (String name : nameServices) {
+      Path path = new Path("hdfs://" + name + "/");
+      FileSystem fileSystem = path.getFileSystem(configuration);
+      FileStatus[] listStatus = fileSystem.listStatus(path);
+      for (FileStatus fileStatus : listStatus) {
+        System.out.println(fileStatus.getPath());
+      }
+    }
+  }
+
+  private static boolean checkHostName(String host) {
+    try {
+      InetAddress.getAllByName(host);
+      return true;
+    } catch (UnknownHostException e) {
+      LOG.warn("Host not found [" + host + "]");
+      return false;
+    }
+  }
+
+  public static Configuration mergeHdfsConfigs(FileSystem fs, Path p) throws IOException {
+    List<Configuration> configList = new ArrayList<Configuration>();
+    gatherConfigs(fs, p, configList);
+    return merge(configList);
+  }
+
+  public static Configuration merge(List<Configuration> configList) throws IOException {
+    Configuration merge = new Configuration(false);
+    Set<String> nameServices = new HashSet<String>();
+    for (Configuration configuration : configList) {
+      String nameService = configuration.get(DFS_NAMESERVICES);
+      if (nameServices.contains(nameService)) {
+        throw new IOException("Multiple confs define namespace [" + nameService + "]");
+      }
+      nameServices.add(nameService);
+      if (shouldAdd(configuration, nameService)) {
+        for (Entry<String, String> e : configuration) {
+          String key = e.getKey();
+          if (key.contains(nameService)) {
+            String value = e.getValue();
+            merge.set(key, value);
+          }
+        }
+      }
+    }
+    merge.set(DFS_NAMESERVICES, StringUtils.join(nameServices, ","));
+    return merge;
+  }
+
+  private static boolean shouldAdd(Configuration configuration, String nameService) {
+    for (Entry<String, String> e : configuration) {
+      String key = e.getKey();
+      if (key.contains(nameService) && key.startsWith("dfs.namenode.rpc-address.")) {
+        return checkHostName(getHost(e.getValue()));
+      }
+    }
+    return false;
+  }
+
+  private static String getHost(String host) {
+    return host.substring(0, host.indexOf(":"));
+  }
+
+  public static void gatherConfigs(FileSystem fs, Path p, List<Configuration> configList) throws IOException {
+    if (fs.isFile(p)) {
+      if (p.getName().endsWith(".xml")) {
+        LOG.info("Loading file [" + p + "]");
+        Configuration configuration = new Configuration(false);
+        configuration.addResource(p);
+        configList.add(configuration);
+      } else {
+        LOG.info("Skipping file [" + p + "]");
+      }
+    } else {
+      FileStatus[] listStatus = fs.listStatus(p);
+      for (FileStatus fileStatus : listStatus) {
+        gatherConfigs(fs, fileStatus.getPath(), configList);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/InputSplitPruneUtil.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/InputSplitPruneUtil.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/InputSplitPruneUtil.java
new file mode 100644
index 0000000..e295073
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/InputSplitPruneUtil.java
@@ -0,0 +1,133 @@
+package org.apache.blur.mapreduce.lib.update;
+
+import org.apache.blur.utils.ShardUtil;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Job;
+
+public class InputSplitPruneUtil {
+
+  private static final String BLUR_LOOKUP_ROWID_UPDATE_FROM_NEW_DATA_COUNT_PREFIX = "blur.lookup.rowid.update.from.new.data.count";
+  private static final String BLUR_LOOKUP_ROWID_FROM_NEW_DATA_COUNT_PREFIX = "blur.lookup.rowid.from.new.data.count.";
+  private static final String BLUR_LOOKUP_ROWID_FROM_INDEX_COUNT_PREFIX = "blur.lookup.rowid.from.index.count.";
+
+  private static final String BLUR_LOOKUP_TABLE = "blur.lookup.table";
+  private static final String BLUR_LOOKUP_RATIO_PER_SHARD = "blur.lookup.ratio.per.shard";
+  private static final String BLUR_LOOKUP_MAX_TOTAL_PER_SHARD = "blur.lookup.max.total.per.shard";
+
+  private static final double DEFAULT_LOOKUP_RATIO = 0.5;
+  private static final long DEFAULT_LOOKUP_MAX_TOTAL = Long.MAX_VALUE;
+
+  public static boolean shouldLookupExecuteOnShard(Configuration configuration, String table, int shard) {
+    double lookupRatio = getLookupRatio(configuration);
+    long maxLookupCount = getMaxLookupCount(configuration);
+    long rowIdFromNewDataCount = getBlurLookupRowIdFromNewDataCount(configuration, table, shard);
+    long rowIdUpdateFromNewDataCount = getBlurLookupRowIdUpdateFromNewDataCount(configuration, table, shard);
+    long rowIdFromIndexCount = getBlurLookupRowIdFromIndexCount(configuration, table, shard);
+    return shouldLookupRun(rowIdFromIndexCount, rowIdFromNewDataCount, rowIdUpdateFromNewDataCount, lookupRatio,
+        maxLookupCount);
+  }
+
+  private static boolean shouldLookupRun(long rowIdFromIndexCount, long rowIdFromNewDataCount,
+      long rowIdUpdateFromNewDataCount, double lookupRatio, long maxLookupCount) {
+    if (rowIdUpdateFromNewDataCount > maxLookupCount) {
+      return false;
+    }
+    double d = (double) rowIdUpdateFromNewDataCount / (double) rowIdFromIndexCount;
+    if (d <= lookupRatio) {
+      return true;
+    }
+    return false;
+  }
+
+  public static double getLookupRatio(Configuration configuration) {
+    return configuration.getDouble(BLUR_LOOKUP_RATIO_PER_SHARD, DEFAULT_LOOKUP_RATIO);
+  }
+
+  private static long getMaxLookupCount(Configuration configuration) {
+    return configuration.getLong(BLUR_LOOKUP_MAX_TOTAL_PER_SHARD, DEFAULT_LOOKUP_MAX_TOTAL);
+  }
+
+  public static void setTable(Job job, String table) {
+    setTable(job.getConfiguration(), table);
+  }
+
+  public static void setTable(Configuration configuration, String table) {
+    configuration.set(BLUR_LOOKUP_TABLE, table);
+  }
+
+  public static String getTable(Configuration configuration) {
+    return configuration.get(BLUR_LOOKUP_TABLE);
+  }
+
+  public static String getBlurLookupRowIdFromIndexCountName(String table) {
+    return BLUR_LOOKUP_ROWID_FROM_INDEX_COUNT_PREFIX + table;
+  }
+
+  public static String getBlurLookupRowIdFromNewDataCountName(String table) {
+    return BLUR_LOOKUP_ROWID_FROM_NEW_DATA_COUNT_PREFIX + table;
+  }
+
+  public static String getBlurLookupRowIdUpdateFromNewDataCountName(String table) {
+    return BLUR_LOOKUP_ROWID_UPDATE_FROM_NEW_DATA_COUNT_PREFIX + table;
+  }
+
+  public static long getBlurLookupRowIdUpdateFromNewDataCount(Configuration configuration, String table, int shard) {
+    String[] strings = configuration.getStrings(getBlurLookupRowIdUpdateFromNewDataCountName(table));
+    return getCount(strings, shard);
+  }
+
+  public static long getBlurLookupRowIdFromNewDataCount(Configuration configuration, String table, int shard) {
+    String[] strings = configuration.getStrings(getBlurLookupRowIdFromNewDataCountName(table));
+    return getCount(strings, shard);
+  }
+
+  public static long getBlurLookupRowIdFromIndexCount(Configuration configuration, String table, int shard) {
+    String[] strings = configuration.getStrings(getBlurLookupRowIdFromIndexCountName(table));
+    return getCount(strings, shard);
+  }
+
+  public static void setBlurLookupRowIdFromNewDataCounts(Job job, String table, long[] counts) {
+    setBlurLookupRowIdFromNewDataCounts(job.getConfiguration(), table, counts);
+  }
+
+  public static void setBlurLookupRowIdFromNewDataCounts(Configuration configuration, String table, long[] counts) {
+    configuration.setStrings(getBlurLookupRowIdFromNewDataCountName(table), toStrings(counts));
+  }
+
+  public static void setBlurLookupRowIdUpdateFromNewDataCounts(Job job, String table, long[] counts) {
+    setBlurLookupRowIdUpdateFromNewDataCounts(job.getConfiguration(), table, counts);
+  }
+
+  public static void setBlurLookupRowIdUpdateFromNewDataCounts(Configuration configuration, String table, long[] counts) {
+    configuration.setStrings(getBlurLookupRowIdUpdateFromNewDataCountName(table), toStrings(counts));
+  }
+
+  public static void setBlurLookupRowIdFromIndexCounts(Job job, String table, long[] counts) {
+    setBlurLookupRowIdFromIndexCounts(job.getConfiguration(), table, counts);
+  }
+
+  public static void setBlurLookupRowIdFromIndexCounts(Configuration configuration, String table, long[] counts) {
+    configuration.setStrings(getBlurLookupRowIdFromIndexCountName(table), toStrings(counts));
+  }
+
+  public static long getCount(String[] strings, int shard) {
+    return Long.parseLong(strings[shard]);
+  }
+
+  public static int getShardFromDirectoryPath(Path path) {
+    return ShardUtil.getShardIndex(path.getName());
+  }
+
+  public static String[] toStrings(long[] counts) {
+    if (counts == null) {
+      return null;
+    }
+    String[] strs = new String[counts.length];
+    for (int i = 0; i < counts.length; i++) {
+      strs[i] = Long.toString(counts[i]);
+    }
+    return strs;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderMapper.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderMapper.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderMapper.java
new file mode 100644
index 0000000..ac0d91f
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderMapper.java
@@ -0,0 +1,18 @@
+package org.apache.blur.mapreduce.lib.update;
+
+import java.io.IOException;
+
+import org.apache.blur.mapreduce.lib.BlurRecord;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+
+public class LookupBuilderMapper extends Mapper<Text, BlurRecord, Text, NullWritable> {
+
+  @Override
+  protected void map(Text key, BlurRecord value, Mapper<Text, BlurRecord, Text, NullWritable>.Context context)
+      throws IOException, InterruptedException {
+    context.write(new Text(value.getRowId()), NullWritable.get());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderReducer.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderReducer.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderReducer.java
new file mode 100644
index 0000000..1983cae
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderReducer.java
@@ -0,0 +1,165 @@
+package org.apache.blur.mapreduce.lib.update;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.blur.BlurConfiguration;
+import org.apache.blur.manager.BlurPartitioner;
+import org.apache.blur.manager.writer.SnapshotIndexDeletionPolicy;
+import org.apache.blur.mapreduce.lib.BlurInputFormat;
+import org.apache.blur.mapreduce.lib.BlurOutputFormat;
+import org.apache.blur.mapreduce.lib.update.MergeSortRowIdMatcher.Action;
+import org.apache.blur.store.BlockCacheDirectoryFactoryV2;
+import org.apache.blur.store.hdfs.HdfsDirectory;
+import org.apache.blur.thrift.generated.TableDescriptor;
+import org.apache.blur.utils.BlurConstants;
+import org.apache.blur.utils.ShardUtil;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexCommit;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.store.Directory;
+
+import com.google.common.io.Closer;
+
+public class LookupBuilderReducer extends Reducer<Text, NullWritable, Text, BooleanWritable> {
+
+  public static final String BLUR_CACHE_DIR_TOTAL_BYTES = "blur.cache.dir.total.bytes";
+  private Counter _rowIds;
+  private Counter _rowIdsToUpdate;
+
+  private MergeSortRowIdMatcher _matcher;
+  private int _numberOfShardsInTable;
+  private Configuration _configuration;
+  private String _snapshot;
+  private Path _tablePath;
+  private Counter _rowIdsFromIndex;
+  private long _totalNumberOfBytes;
+  private Action _action;
+  private Closer _closer;
+  private Path _cachePath;
+  private String _table;
+  private Writer _writer;
+
+  @Override
+  protected void setup(Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) throws IOException,
+      InterruptedException {
+    _configuration = context.getConfiguration();
+    _rowIds = context.getCounter(BlurIndexCounter.ROW_IDS_FROM_NEW_DATA);
+    _rowIdsToUpdate = context.getCounter(BlurIndexCounter.ROW_IDS_TO_UPDATE_FROM_NEW_DATA);
+    _rowIdsFromIndex = context.getCounter(BlurIndexCounter.ROW_IDS_FROM_INDEX);
+    TableDescriptor tableDescriptor = BlurOutputFormat.getTableDescriptor(_configuration);
+    _numberOfShardsInTable = tableDescriptor.getShardCount();
+    _tablePath = new Path(tableDescriptor.getTableUri());
+    _snapshot = MapperForExistingDataWithIndexLookup.getSnapshot(_configuration);
+    _totalNumberOfBytes = _configuration.getLong(BLUR_CACHE_DIR_TOTAL_BYTES, 128 * 1024 * 1024);
+    _cachePath = BlurInputFormat.getLocalCachePath(_configuration);
+    _table = tableDescriptor.getName();
+    _closer = Closer.create();
+  }
+
+  @Override
+  protected void reduce(Text rowId, Iterable<NullWritable> nothing,
+      Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) throws IOException, InterruptedException {
+    if (_matcher == null) {
+      _matcher = getMergeSortRowIdMatcher(rowId, context);
+    }
+    if (_writer == null) {
+      _writer = getRowIdWriter(rowId, context);
+    }
+    _writer.append(rowId, NullWritable.get());
+    _rowIds.increment(1);
+    if (_action == null) {
+      _action = new Action() {
+        @Override
+        public void found(Text rowId) throws IOException {
+          _rowIdsToUpdate.increment(1);
+          try {
+            context.write(rowId, new BooleanWritable(true));
+          } catch (InterruptedException e) {
+            throw new IOException(e);
+          }
+        }
+      };
+    }
+    _matcher.lookup(rowId, _action);
+  }
+
+  private Writer getRowIdWriter(Text rowId, Reducer<Text, NullWritable, Text, BooleanWritable>.Context context)
+      throws IOException {
+    BlurPartitioner blurPartitioner = new BlurPartitioner();
+    int shard = blurPartitioner.getShard(rowId, _numberOfShardsInTable);
+    String shardName = ShardUtil.getShardName(shard);
+    Path cachePath = MergeSortRowIdMatcher.getCachePath(_cachePath, _table, shardName);
+    Configuration configuration = context.getConfiguration();
+    String uuid = configuration.get(FasterDriver.BLUR_UPDATE_ID);
+    Path tmpPath = new Path(cachePath, uuid + "_" + getAttemptString(context));
+    return _closer.register(MergeSortRowIdMatcher.createWriter(_configuration, tmpPath));
+  }
+
+  private String getAttemptString(Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) {
+    TaskAttemptID taskAttemptID = context.getTaskAttemptID();
+    return taskAttemptID.toString();
+  }
+
+  @Override
+  protected void cleanup(Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) throws IOException,
+      InterruptedException {
+    _closer.close();
+  }
+
+  private MergeSortRowIdMatcher getMergeSortRowIdMatcher(Text rowId,
+      Reducer<Text, NullWritable, Text, BooleanWritable>.Context context) throws IOException {
+    BlurPartitioner blurPartitioner = new BlurPartitioner();
+    int shard = blurPartitioner.getShard(rowId, _numberOfShardsInTable);
+    String shardName = ShardUtil.getShardName(shard);
+
+    Path shardPath = new Path(_tablePath, shardName);
+    HdfsDirectory hdfsDirectory = new HdfsDirectory(_configuration, shardPath);
+    SnapshotIndexDeletionPolicy policy = new SnapshotIndexDeletionPolicy(_configuration,
+        SnapshotIndexDeletionPolicy.getGenerationsPath(shardPath));
+    Long generation = policy.getGeneration(_snapshot);
+    if (generation == null) {
+      hdfsDirectory.close();
+      throw new IOException("Snapshot [" + _snapshot + "] not found in shard [" + shardPath + "]");
+    }
+
+    BlurConfiguration bc = new BlurConfiguration();
+    BlockCacheDirectoryFactoryV2 blockCacheDirectoryFactoryV2 = new BlockCacheDirectoryFactoryV2(bc,
+        _totalNumberOfBytes);
+    _closer.register(blockCacheDirectoryFactoryV2);
+    Directory dir = blockCacheDirectoryFactoryV2.newDirectory("table", "shard", hdfsDirectory, null);
+    List<IndexCommit> listCommits = DirectoryReader.listCommits(dir);
+    IndexCommit indexCommit = MapperForExistingDataWithIndexLookup.findIndexCommit(listCommits, generation, shardPath);
+    DirectoryReader reader = DirectoryReader.open(indexCommit);
+    _rowIdsFromIndex.setValue(getTotalNumberOfRowIds(reader));
+
+    Path cachePath = MergeSortRowIdMatcher.getCachePath(_cachePath, _table, shardName);
+    return new MergeSortRowIdMatcher(dir, generation, _configuration, cachePath, context);
+  }
+
+  private long getTotalNumberOfRowIds(DirectoryReader reader) throws IOException {
+    long total = 0;
+    List<AtomicReaderContext> leaves = reader.leaves();
+    for (AtomicReaderContext context : leaves) {
+      AtomicReader atomicReader = context.reader();
+      Terms terms = atomicReader.terms(BlurConstants.ROW_ID);
+      long expectedInsertions = terms.size();
+      if (expectedInsertions < 0) {
+        return -1;
+      }
+      total += expectedInsertions;
+    }
+    return total;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataMod.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataMod.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataMod.java
new file mode 100644
index 0000000..bf86e19
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataMod.java
@@ -0,0 +1,46 @@
+/**
+ * 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.blur.mapreduce.lib.update;
+
+import java.io.IOException;
+
+import org.apache.blur.mapreduce.lib.BlurRecord;
+import org.apache.blur.mapreduce.lib.TableBlurRecord;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.Mapper;
+
+public class MapperForExistingDataMod extends Mapper<Text, TableBlurRecord, IndexKey, IndexValue> {
+
+  private Counter _existingRecords;
+
+  @Override
+  protected void setup(Context context) throws IOException, InterruptedException {
+    Counter counter = context.getCounter(BlurIndexCounter.INPUT_FORMAT_MAPPER);
+    counter.increment(1);
+    _existingRecords = context.getCounter(BlurIndexCounter.INPUT_FORMAT_EXISTING_RECORDS);
+  }
+
+  @Override
+  protected void map(Text key, TableBlurRecord value, Context context) throws IOException, InterruptedException {
+    BlurRecord blurRecord = value.getBlurRecord();
+    IndexKey oldDataKey = IndexKey.oldData(blurRecord.getRowId(), blurRecord.getRecordId());
+    context.write(oldDataKey, new IndexValue(blurRecord));
+    _existingRecords.increment(1L);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataWithIndexLookup.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataWithIndexLookup.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataWithIndexLookup.java
new file mode 100644
index 0000000..0e2fe66
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForExistingDataWithIndexLookup.java
@@ -0,0 +1,228 @@
+/**
+ * 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.blur.mapreduce.lib.update;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.blur.BlurConfiguration;
+import org.apache.blur.manager.BlurPartitioner;
+import org.apache.blur.manager.writer.SnapshotIndexDeletionPolicy;
+import org.apache.blur.mapreduce.lib.BlurOutputFormat;
+import org.apache.blur.mapreduce.lib.BlurRecord;
+import org.apache.blur.store.BlockCacheDirectoryFactoryV2;
+import org.apache.blur.store.hdfs.HdfsDirectory;
+import org.apache.blur.thrift.generated.Column;
+import org.apache.blur.thrift.generated.FetchRecordResult;
+import org.apache.blur.thrift.generated.Record;
+import org.apache.blur.thrift.generated.TableDescriptor;
+import org.apache.blur.utils.BlurConstants;
+import org.apache.blur.utils.RowDocumentUtil;
+import org.apache.blur.utils.ShardUtil;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexCommit;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.Directory;
+
+import com.google.common.io.Closer;
+
+public class MapperForExistingDataWithIndexLookup extends Mapper<Text, BooleanWritable, IndexKey, IndexValue> {
+
+  private static final Log LOG = LogFactory.getLog(MapperForExistingDataWithIndexLookup.class);
+
+  private static final String BLUR_SNAPSHOT = "blur.snapshot";
+  private Counter _existingRecords;
+  private Counter _rowLookup;
+  private BlurPartitioner _blurPartitioner;
+  private Path _tablePath;
+  private int _numberOfShardsInTable;
+  private Configuration _configuration;
+  private String _snapshot;
+
+  private int _indexShard = -1;
+  private DirectoryReader _reader;
+  private IndexSearcher _indexSearcher;
+  private long _totalNumberOfBytes;
+  private Closer _closer;
+
+  @Override
+  protected void setup(Context context) throws IOException, InterruptedException {
+    Counter counter = context.getCounter(BlurIndexCounter.LOOKUP_MAPPER);
+    counter.increment(1);
+
+    _configuration = context.getConfiguration();
+    _existingRecords = context.getCounter(BlurIndexCounter.LOOKUP_MAPPER_EXISTING_RECORDS);
+    _rowLookup = context.getCounter(BlurIndexCounter.LOOKUP_MAPPER_ROW_LOOKUP_ATTEMPT);
+    _blurPartitioner = new BlurPartitioner();
+    TableDescriptor tableDescriptor = BlurOutputFormat.getTableDescriptor(_configuration);
+    _numberOfShardsInTable = tableDescriptor.getShardCount();
+    _tablePath = new Path(tableDescriptor.getTableUri());
+    _snapshot = getSnapshot(_configuration);
+    _totalNumberOfBytes = _configuration.getLong(LookupBuilderReducer.BLUR_CACHE_DIR_TOTAL_BYTES, 128 * 1024 * 1024);
+    _closer = Closer.create();
+  }
+
+  @Override
+  protected void map(Text key, BooleanWritable value, Context context) throws IOException, InterruptedException {
+    if (value.get()) {
+      String rowId = key.toString();
+      LOG.debug("Looking up rowid [" + rowId + "]");
+      _rowLookup.increment(1);
+      IndexSearcher indexSearcher = getIndexSearcher(rowId);
+      Term term = new Term(BlurConstants.ROW_ID, rowId);
+      RowCollector collector = getCollector(context);
+      indexSearcher.search(new TermQuery(term), collector);
+      LOG.debug("Looking for rowid [" + rowId + "] has [" + collector.records + "] records");
+    }
+  }
+
+  @Override
+  protected void cleanup(Mapper<Text, BooleanWritable, IndexKey, IndexValue>.Context context) throws IOException,
+      InterruptedException {
+    _closer.close();
+  }
+
+  static class RowCollector extends Collector {
+
+    private AtomicReader reader;
+    private Mapper<Text, BooleanWritable, IndexKey, IndexValue>.Context _context;
+    private Counter _existingRecords;
+    int records;
+
+    RowCollector(Mapper<Text, BooleanWritable, IndexKey, IndexValue>.Context context, Counter existingRecords) {
+      _context = context;
+      _existingRecords = existingRecords;
+    }
+
+    @Override
+    public void setScorer(Scorer scorer) throws IOException {
+
+    }
+
+    @Override
+    public void setNextReader(AtomicReaderContext context) throws IOException {
+      reader = context.reader();
+    }
+
+    @Override
+    public void collect(int doc) throws IOException {
+      Document document = reader.document(doc);
+      FetchRecordResult result = RowDocumentUtil.getRecord(document);
+      String rowid = result.getRowid();
+      Record record = result.getRecord();
+      String recordId = record.getRecordId();
+      IndexKey oldDataKey = IndexKey.oldData(rowid, recordId);
+      try {
+        _context.write(oldDataKey, new IndexValue(toBlurRecord(rowid, record)));
+      } catch (InterruptedException e) {
+        throw new IOException(e);
+      }
+      _existingRecords.increment(1L);
+    }
+
+    private BlurRecord toBlurRecord(String rowId, Record record) {
+      BlurRecord blurRecord = new BlurRecord();
+      blurRecord.setRowId(rowId);
+      blurRecord.setRecordId(record.getRecordId());
+      blurRecord.setFamily(record.getFamily());
+      List<Column> columns = record.getColumns();
+      for (Column column : columns) {
+        blurRecord.addColumn(column.getName(), column.getValue());
+      }
+      return blurRecord;
+    }
+
+    @Override
+    public boolean acceptsDocsOutOfOrder() {
+      return false;
+    }
+  }
+
+  private RowCollector getCollector(Mapper<Text, BooleanWritable, IndexKey, IndexValue>.Context context) {
+    return new RowCollector(context, _existingRecords);
+  }
+
+  private IndexSearcher getIndexSearcher(String rowId) throws IOException {
+    int shard = _blurPartitioner.getShard(rowId, _numberOfShardsInTable);
+    if (_indexSearcher != null) {
+      if (shard != _indexShard) {
+        throw new IOException("Input data is not partitioned correctly.");
+      }
+      return _indexSearcher;
+    } else {
+      _indexShard = shard;
+      Path shardPath = new Path(_tablePath, ShardUtil.getShardName(_indexShard));
+      HdfsDirectory hdfsDirectory = new HdfsDirectory(_configuration, shardPath);
+      SnapshotIndexDeletionPolicy policy = new SnapshotIndexDeletionPolicy(_configuration,
+          SnapshotIndexDeletionPolicy.getGenerationsPath(shardPath));
+      Long generation = policy.getGeneration(_snapshot);
+      if (generation == null) {
+        hdfsDirectory.close();
+        throw new IOException("Snapshot [" + _snapshot + "] not found in shard [" + shardPath + "]");
+      }
+
+      BlurConfiguration bc = new BlurConfiguration();
+      BlockCacheDirectoryFactoryV2 blockCacheDirectoryFactoryV2 = new BlockCacheDirectoryFactoryV2(bc,
+          _totalNumberOfBytes);
+      _closer.register(blockCacheDirectoryFactoryV2);
+      Directory dir = blockCacheDirectoryFactoryV2.newDirectory("table", "shard", hdfsDirectory, null);
+
+      List<IndexCommit> listCommits = DirectoryReader.listCommits(dir);
+      IndexCommit indexCommit = findIndexCommit(listCommits, generation, shardPath);
+      _reader = DirectoryReader.open(indexCommit);
+      return _indexSearcher = new IndexSearcher(_reader);
+    }
+  }
+
+  public static IndexCommit findIndexCommit(List<IndexCommit> listCommits, long generation, Path shardDir)
+      throws IOException {
+    for (IndexCommit commit : listCommits) {
+      if (commit.getGeneration() == generation) {
+        return commit;
+      }
+    }
+    throw new IOException("Generation [" + generation + "] not found in shard [" + shardDir + "]");
+  }
+
+  public static void setSnapshot(Job job, String snapshot) {
+    setSnapshot(job.getConfiguration(), snapshot);
+  }
+
+  public static void setSnapshot(Configuration configuration, String snapshot) {
+    configuration.set(BLUR_SNAPSHOT, snapshot);
+  }
+
+  public static String getSnapshot(Configuration configuration) {
+    return configuration.get(BLUR_SNAPSHOT);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForNewDataMod.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForNewDataMod.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForNewDataMod.java
new file mode 100644
index 0000000..d91d1f5
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MapperForNewDataMod.java
@@ -0,0 +1,82 @@
+/**
+ * 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.blur.mapreduce.lib.update;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+
+import org.apache.blur.mapreduce.lib.BlurRecord;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+public class MapperForNewDataMod extends Mapper<Text, BlurRecord, IndexKey, IndexValue> {
+
+  private static final IndexValue EMPTY_RECORD = new IndexValue();
+  private long _timestamp;
+  private Counter _newRecords;
+
+  @Override
+  protected void setup(Context context) throws IOException, InterruptedException {
+    InputSplit inputSplit = context.getInputSplit();
+    FileSplit fileSplit = getFileSplit(inputSplit);
+    Path path = fileSplit.getPath();
+    FileSystem fileSystem = path.getFileSystem(context.getConfiguration());
+    FileStatus fileStatus = fileSystem.getFileStatus(path);
+    _timestamp = fileStatus.getModificationTime();
+    _newRecords = context.getCounter(BlurIndexCounter.NEW_RECORDS);
+  }
+
+  private FileSplit getFileSplit(InputSplit inputSplit) throws IOException {
+    if (inputSplit instanceof FileSplit) {
+      return (FileSplit) inputSplit;
+    }
+    if (inputSplit.getClass().getName().equals("org.apache.hadoop.mapreduce.lib.input.TaggedInputSplit")) {
+      try {
+        Field declaredField = inputSplit.getClass().getDeclaredField("inputSplit");
+        declaredField.setAccessible(true);
+        return getFileSplit((InputSplit) declaredField.get(inputSplit));
+      } catch (NoSuchFieldException e) {
+        throw new IOException(e);
+      } catch (SecurityException e) {
+        throw new IOException(e);
+      } catch (IllegalArgumentException e) {
+        throw new IOException(e);
+      } catch (IllegalAccessException e) {
+        throw new IOException(e);
+      }
+    } else {
+      throw new IOException("Unknown input split type [" + inputSplit + "] [" + inputSplit.getClass() + "]");
+    }
+  }
+
+  @Override
+  protected void map(Text key, BlurRecord blurRecord, Context context) throws IOException, InterruptedException {
+    IndexKey newDataKey = IndexKey.newData(blurRecord.getRowId(), blurRecord.getRecordId(), _timestamp);
+    context.write(newDataKey, new IndexValue(blurRecord));
+    _newRecords.increment(1L);
+
+    IndexKey newDataMarker = IndexKey.newDataMarker(blurRecord.getRowId());
+    context.write(newDataMarker, EMPTY_RECORD);
+  }
+
+}


[05/13] git commit: Fourth round of udpates.

Posted by am...@apache.org.
Fourth round of udpates.


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

Branch: refs/heads/master
Commit: 96a1821a0a723fafefd79a8a6769cf1286e95eaa
Parents: ea50630
Author: Aaron McCurry <am...@gmail.com>
Authored: Sat May 7 13:12:48 2016 -0400
Committer: Aaron McCurry <am...@gmail.com>
Committed: Sat May 7 13:12:48 2016 -0400

----------------------------------------------------------------------
 .../mapreduce/lib/GenericBlurRecordWriter.java  |  57 +++++-
 .../mapreduce/lib/PrimeDocOverFlowHelper.java   |  31 ++-
 .../mapreduce/lib/update/UpdateReducer.java     |  23 ++-
 .../blur/lucene/search/FacetExecutor.java       |  89 ++++++---
 .../apache/blur/lucene/search/FacetQuery.java   |  14 +-
 .../BaseReadMaskFieldTypeDefinitionTest.java    |  13 +-
 blur-shell/pom.xml                              |   7 +
 .../ListRunningPlatformCommandsCommand.java     |  43 ++--
 .../main/java/org/apache/blur/shell/Main.java   |   3 +-
 .../org/apache/blur/shell/TableDisplay.java     |   3 +-
 .../org/apache/blur/shell/WatchCommands.java    | 149 ++++++++++++++
 .../lucene/codec/DiskDocValuesProducer.java     | 195 ++++++++++++++-----
 .../store/blockcache_v2/CacheIndexInput.java    |   4 +-
 .../blur/store/blockcache_v2/MeterWrapper.java  |  13 +-
 .../cachevalue/DetachableCacheValue.java        |  48 +++--
 .../blur/store/hdfs/SequentialReadControl.java  |   3 +-
 .../packed/DirectPacked64SingleBlockReader.java |  65 +++++++
 .../lucene/util/packed/DirectPackedReader.java  |  80 ++++++++
 .../org/apache/blur/utils/BlurConstants.java    |   5 +-
 deploy.sh                                       |  62 ++++++
 20 files changed, 754 insertions(+), 153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/GenericBlurRecordWriter.java
----------------------------------------------------------------------
diff --git a/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/GenericBlurRecordWriter.java b/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/GenericBlurRecordWriter.java
index a947980..8828f85 100644
--- a/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/GenericBlurRecordWriter.java
+++ b/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/GenericBlurRecordWriter.java
@@ -18,9 +18,12 @@ package org.apache.blur.mapreduce.lib;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.blur.analysis.FieldManager;
 import org.apache.blur.log.Log;
@@ -47,11 +50,11 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.SlowCompositeReaderWrapper;
 import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -95,6 +98,7 @@ public class GenericBlurRecordWriter {
   private ProgressableDirectory _localTmpDir;
   private String _deletedRowId;
   private Configuration _configuration;
+  private String _currentRowId;
 
   public GenericBlurRecordWriter(Configuration configuration, int attemptId, String tmpDirName) throws IOException {
     _configuration = configuration;
@@ -200,6 +204,7 @@ public class GenericBlurRecordWriter {
 
   private void add(BlurMutate value) throws IOException {
     BlurRecord blurRecord = value.getRecord();
+    _currentRowId = blurRecord.getRowId();
     Record record = getRecord(blurRecord);
     String recordId = record.getRecordId();
     if (value.getMutateType() == MUTATE_TYPE.DELETE) {
@@ -224,7 +229,7 @@ public class GenericBlurRecordWriter {
 
   private void flushToTmpIndexIfNeeded() throws IOException {
     if (_documentBufferStrategy.isFull()) {
-      LOG.info("Document Buffer is full overflow to disk.");
+      LOG.info("RowId [" + _currentRowId + "] - Document Buffer is full overflow to disk.");
       flushToTmpIndex();
     }
   }
@@ -273,15 +278,35 @@ public class GenericBlurRecordWriter {
     return record;
   }
 
+  private static ThreadLocal<AtomicBoolean> _existingRow = new ThreadLocal<AtomicBoolean>() {
+    @Override
+    protected AtomicBoolean initialValue() {
+      return new AtomicBoolean();
+    }
+  };
+
+  public static boolean isCurrentRowExisting() {
+    return _existingRow.get().get();
+  }
+
+  public static void setCurrentRowExistingRowId(boolean existing) {
+    _existingRow.get().set(existing);
+  }
+
   private void flush() throws CorruptIndexException, IOException {
+    boolean newRow = !isCurrentRowExisting();
     if (_usingLocalTmpindex) {
       // since we have flushed to disk then we do not need to index the
       // delete.
       flushToTmpIndex();
-      _localTmpWriter.close(false);
+      LOG.info("RowId [" + _currentRowId + "] - forceMerge");
+      _localTmpWriter.forceMerge(1, true);
+      _localTmpWriter.close(true);
+
       DirectoryReader reader = DirectoryReader.open(_localTmpDir);
-      AtomicReader atomicReader = SlowCompositeReaderWrapper.wrap(reader);
-      AtomicReader primeDocAtomicReader = PrimeDocOverFlowHelper.addPrimeDoc(atomicReader);
+      AtomicReader atomicReader = getAtomicReader(reader);
+      LOG.info("RowId [" + _currentRowId + "] - total documents [" + atomicReader.maxDoc() + "]");
+      AtomicReader primeDocAtomicReader = PrimeDocOverFlowHelper.addPrimeDoc(atomicReader, newRow, _currentRowId);
       if (_countersSetup) {
         _recordRateCounter.mark(reader.numDocs());
       }
@@ -289,6 +314,7 @@ public class GenericBlurRecordWriter {
       primeDocAtomicReader.close();
       resetLocalTmp();
       _writer.maybeMerge();
+      LOG.info("RowId [" + _currentRowId + "] - add complete");
       if (_countersSetup) {
         _rowOverFlowCount.increment(1);
       }
@@ -303,6 +329,11 @@ public class GenericBlurRecordWriter {
       } else {
         List<List<Field>> docs = _documentBufferStrategy.getAndClearBuffer();
         docs.get(0).add(new StringField(BlurConstants.PRIME_DOC, BlurConstants.PRIME_DOC_VALUE, Store.NO));
+        if (newRow) {
+          docs.get(0).add(new StringField(BlurConstants.NEW_ROW, BlurConstants.PRIME_DOC_VALUE, Store.NO));
+        } else {
+          docs.get(0).add(new StringField(BlurConstants.UPDATE_ROW, _currentRowId, Store.NO));
+        }
         _writer.addDocuments(docs);
         if (_countersSetup) {
           _recordRateCounter.mark(docs.size());
@@ -316,10 +347,19 @@ public class GenericBlurRecordWriter {
     }
   }
 
+  private AtomicReader getAtomicReader(DirectoryReader reader) throws IOException {
+    List<AtomicReaderContext> leaves = reader.leaves();
+    if (leaves.size() == 1) {
+      return leaves.get(0).reader();
+    }
+    throw new IOException("Reader [" + reader + "] has more than one segment after optimize.");
+  }
+
   private Document getDeleteDoc() {
     Document document = new Document();
     document.add(new StringField(BlurConstants.ROW_ID, _deletedRowId, Store.NO));
     document.add(new StringField(BlurConstants.DELETE_MARKER, BlurConstants.DELETE_MARKER_VALUE, Store.NO));
+    document.add(new StringField(BlurConstants.UPDATE_ROW, _deletedRowId, Store.NO));
     return document;
   }
 
@@ -348,10 +388,17 @@ public class GenericBlurRecordWriter {
     DirectoryReader reader = DirectoryReader.open(_localDir);
     IndexWriter writer = new IndexWriter(copyRateDirectory, _conf.clone());
     writer.addIndexes(reader);
+    writer.setCommitData(getInternalMarker());
     writer.close();
     rm(_localPath);
   }
 
+  private Map<String, String> getInternalMarker() {
+    Map<String, String> map = new HashMap<String, String>();
+    map.put(BlurConstants.INTERNAL, BlurConstants.INTERNAL);
+    return map;
+  }
+
   private void copyDir() throws IOException {
     CopyRateDirectory copyRateDirectory = new CopyRateDirectory(_finalDir, _copyRateCounter);
     String[] fileNames = _localDir.listAll();

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/PrimeDocOverFlowHelper.java
----------------------------------------------------------------------
diff --git a/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/PrimeDocOverFlowHelper.java b/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/PrimeDocOverFlowHelper.java
index 672a1c1..73d9c78 100644
--- a/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/PrimeDocOverFlowHelper.java
+++ b/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/PrimeDocOverFlowHelper.java
@@ -38,26 +38,47 @@ import org.apache.lucene.util.Version;
 
 public class PrimeDocOverFlowHelper {
 
-  private static Directory _directory;
+  private static Directory _directoryNewRow;
 
   static {
     try {
-      _directory = new RAMDirectory();
-      IndexWriter writer = new IndexWriter(_directory, new IndexWriterConfig(Version.LUCENE_43, new KeywordAnalyzer()));
+      _directoryNewRow = new RAMDirectory();
+      IndexWriter writer = new IndexWriter(_directoryNewRow, new IndexWriterConfig(Version.LUCENE_43,
+          new KeywordAnalyzer()));
       Document document = new Document();
       document.add(new StringField(BlurConstants.PRIME_DOC, BlurConstants.PRIME_DOC_VALUE, Store.NO));
+      document.add(new StringField(BlurConstants.NEW_ROW, BlurConstants.PRIME_DOC_VALUE, Store.NO));
       writer.addDocument(document);
       writer.close();
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
+
   }
 
-  public static AtomicReader addPrimeDoc(AtomicReader atomicReader) throws IOException {
-    AtomicReaderContext context = DirectoryReader.open(_directory).leaves().get(0);
+  public static AtomicReader addPrimeDoc(AtomicReader atomicReader, boolean newRow, String currentRowId)
+      throws IOException {
+    AtomicReaderContext context = DirectoryReader.open(newRow ? _directoryNewRow : getDirectoryUpdateRow(currentRowId))
+        .leaves().get(0);
     return new ParallelAtomicReader(true, setDocSize(context.reader(), atomicReader.maxDoc()), atomicReader);
   }
 
+  private static Directory getDirectoryUpdateRow(String currentRowId) {
+    try {
+      RAMDirectory directoryUpdateRow = new RAMDirectory();
+      IndexWriter writer = new IndexWriter(directoryUpdateRow, new IndexWriterConfig(Version.LUCENE_43,
+          new KeywordAnalyzer()));
+      Document document = new Document();
+      document.add(new StringField(BlurConstants.PRIME_DOC, BlurConstants.PRIME_DOC_VALUE, Store.NO));
+      document.add(new StringField(BlurConstants.UPDATE_ROW, currentRowId, Store.NO));
+      writer.addDocument(document);
+      writer.close();
+      return directoryUpdateRow;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
   private static AtomicReader setDocSize(AtomicReader reader, final int count) {
     return new FilterAtomicReader(reader) {
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/update/UpdateReducer.java
----------------------------------------------------------------------
diff --git a/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/update/UpdateReducer.java b/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/update/UpdateReducer.java
index f8705aa..d62617b 100644
--- a/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/update/UpdateReducer.java
+++ b/blur-mapred/src/main/java/org/apache/blur/mapreduce/lib/update/UpdateReducer.java
@@ -20,10 +20,11 @@ import java.io.IOException;
 
 import org.apache.blur.mapreduce.lib.BlurMutate;
 import org.apache.blur.mapreduce.lib.BlurMutate.MUTATE_TYPE;
-import org.apache.blur.mapreduce.lib.update.IndexKey.TYPE;
 import org.apache.blur.mapreduce.lib.BlurOutputFormat;
 import org.apache.blur.mapreduce.lib.BlurRecord;
+import org.apache.blur.mapreduce.lib.GenericBlurRecordWriter;
 import org.apache.blur.mapreduce.lib.GetCounter;
+import org.apache.blur.mapreduce.lib.update.IndexKey.TYPE;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.Reducer;
@@ -37,9 +38,9 @@ public class UpdateReducer extends Reducer<IndexKey, IndexValue, Text, BlurMutat
   private static final String MARKER_RECORDS = "Marker Records";
   private static final String SEP = " - ";
   private static final String BLUR_UPDATE = "Blur Update";
-  private static final String EXISTING_RCORDS = "Existing Rcords";
-  private static final String NEW_RCORDS = "New Rcords";
-  private static final String NO_UPDATE = "NoUpdate";
+  private static final String EXISTING_RECORDS = "Existing Records";
+  private static final String NEW_RECORDS = "New Records";
+  private static final String NO_UPDATE = "No Update";
   private static final String UPDATE = "Update";
   private static final String BLUR_UPDATE_DEBUG = BLUR_UPDATE + SEP + "DEBUG";
 
@@ -64,10 +65,10 @@ public class UpdateReducer extends Reducer<IndexKey, IndexValue, Text, BlurMutat
       }
     });
 
-    _newRecordsUpdate = context.getCounter(BLUR_UPDATE, NEW_RCORDS + SEP + UPDATE);
-    _newRecordsNoUpdate = context.getCounter(BLUR_UPDATE, NEW_RCORDS + SEP + NO_UPDATE);
-    _existingRecordsUpdate = context.getCounter(BLUR_UPDATE, EXISTING_RCORDS + SEP + UPDATE);
-    _existingRecordsNoUpdate = context.getCounter(BLUR_UPDATE, EXISTING_RCORDS + SEP + NO_UPDATE);
+    _newRecordsUpdate = context.getCounter(BLUR_UPDATE, NEW_RECORDS + SEP + UPDATE);
+    _newRecordsNoUpdate = context.getCounter(BLUR_UPDATE, NEW_RECORDS + SEP + NO_UPDATE);
+    _existingRecordsUpdate = context.getCounter(BLUR_UPDATE, EXISTING_RECORDS + SEP + UPDATE);
+    _existingRecordsNoUpdate = context.getCounter(BLUR_UPDATE, EXISTING_RECORDS + SEP + NO_UPDATE);
     _ignoredExistingRows = context.getCounter(BLUR_UPDATE, IGNORED_EXISTING_ROWS);
 
     _debugRecordsWithSameRecordId = context.getCounter(BLUR_UPDATE_DEBUG, MULTIPLE_RECORD_W_SAME_RECORD_ID);
@@ -76,7 +77,6 @@ public class UpdateReducer extends Reducer<IndexKey, IndexValue, Text, BlurMutat
     _debugMarkerRecordsUpdate = context.getCounter(BLUR_UPDATE_DEBUG, MARKER_RECORDS + SEP + UPDATE);
     _debugIndexValues = context.getCounter(BLUR_UPDATE_DEBUG, INDEX_VALUES);
     _debugNullBlurRecords = context.getCounter(BLUR_UPDATE_DEBUG, NULL_BLUR_RECORDS);
-
   }
 
   @Override
@@ -93,6 +93,7 @@ public class UpdateReducer extends Reducer<IndexKey, IndexValue, Text, BlurMutat
       InterruptedException {
     BlurRecord prevBlurRecord = null;
     String prevRecordId = null;
+    boolean existing = false;
     for (IndexValue value : values) {
       updateCounters(true, key);
       BlurRecord br = value.getBlurRecord();
@@ -103,6 +104,9 @@ public class UpdateReducer extends Reducer<IndexKey, IndexValue, Text, BlurMutat
       } else {
         // Safe Copy
         BlurRecord currentBlurRecord = new BlurRecord(br);
+        if (key.getType() == IndexKey.TYPE.OLD_DATA) {
+          existing = true;
+        }
         String currentRecordId = currentBlurRecord.getRecordId();
         if (prevRecordId != null) {
           if (prevRecordId.equals(currentRecordId)) {
@@ -120,6 +124,7 @@ public class UpdateReducer extends Reducer<IndexKey, IndexValue, Text, BlurMutat
     if (prevBlurRecord != null) {
       context.write(new Text(prevBlurRecord.getRowId()), toMutate(prevBlurRecord));
     }
+    GenericBlurRecordWriter.setCurrentRowExistingRowId(existing);
   }
 
   private void updateCounters(boolean update, IndexKey key) {

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-query/src/main/java/org/apache/blur/lucene/search/FacetExecutor.java
----------------------------------------------------------------------
diff --git a/blur-query/src/main/java/org/apache/blur/lucene/search/FacetExecutor.java b/blur-query/src/main/java/org/apache/blur/lucene/search/FacetExecutor.java
index 683ba98..00383f7 100644
--- a/blur-query/src/main/java/org/apache/blur/lucene/search/FacetExecutor.java
+++ b/blur-query/src/main/java/org/apache/blur/lucene/search/FacetExecutor.java
@@ -18,11 +18,13 @@ package org.apache.blur.lucene.search;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.UUID;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
@@ -37,10 +39,11 @@ import org.apache.blur.log.Log;
 import org.apache.blur.log.LogFactory;
 import org.apache.blur.trace.Trace;
 import org.apache.blur.trace.Tracer;
+import org.apache.blur.user.User;
+import org.apache.blur.user.UserContext;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.search.Collector;
-import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.util.OpenBitSet;
 
@@ -75,19 +78,21 @@ public class FacetExecutor {
     public void collect(int doc) throws IOException {
       if (_bitSet.fastGet(doc)) {
         _hits++;
-      } else {
-        int nextSetBit = _bitSet.nextSetBit(doc);
-        if (nextSetBit < 0) {
-          throw new Finished();
-        } else {
-          int advance = _scorer.advance(nextSetBit);
-          if (advance == DocIdSetIterator.NO_MORE_DOCS) {
-            throw new Finished();
-          }
-          if (_bitSet.fastGet(advance)) {
-            _hits++;
-          }
-        }
+        // } else {
+        // int nextSetBit = _bitSet.nextSetBit(doc);
+        // if (nextSetBit < 0) {
+        // LOG.debug("finished early, no more hits in query.");
+        // throw new Finished();
+        // } else {
+        // int advance = _scorer.advance(nextSetBit);
+        // if (advance == DocIdSetIterator.NO_MORE_DOCS) {
+        // LOG.debug("finished early, no more hits in facet.");
+        // throw new Finished();
+        // }
+        // if (_bitSet.fastGet(advance)) {
+        // _hits++;
+        // }
+        // }
       }
     }
 
@@ -137,14 +142,17 @@ public class FacetExecutor {
     final String _readerStr;
     final int _maxDoc;
     final Lock[] _locks;
+    final String _instance;
 
     @Override
     public String toString() {
-      return "Info scorers length [" + _scorers.length + "] reader [" + _reader + "]";
+      return "Info scorers length [" + _scorers.length + "] reader [" + _reader + "] scorers ["
+          + Arrays.asList(_scorers) + "]";
     }
 
-    Info(AtomicReaderContext context, Scorer[] scorers, Lock[] locks) {
+    Info(AtomicReaderContext context, Scorer[] scorers, Lock[] locks, String instance) {
       AtomicReader reader = context.reader();
+      _instance = instance;
       _bitSet = new OpenBitSet(reader.maxDoc());
       _scorers = scorers;
       _reader = reader;
@@ -155,13 +163,18 @@ public class FacetExecutor {
 
     void process(AtomicLongArray counts, long[] minimumsBeforeReturning, AtomicBoolean running) throws IOException {
       if (minimumsBeforeReturning == null) {
+        LOG.debug(getPrefix("no minimums before returning."));
         Tracer trace = Trace.trace("processing facet - segment", Trace.param("reader", _readerStr),
             Trace.param("maxDoc", _maxDoc), Trace.param("minimums", "NONE"), Trace.param("scorers", _scorers.length));
         try {
           for (int i = 0; i < _scorers.length && running.get(); i++) {
+            LOG.debug(getPrefix("running facet for scorer [{0}] [{1}]."), i, _scorers[i]);
             SimpleCollector col = new SimpleCollector(_bitSet);
             runFacet(counts, col, i);
           }
+          if (!running.get()) {
+            LOG.debug(getPrefix("running was stopped."));
+          }
         } finally {
           trace.done();
         }
@@ -177,7 +190,7 @@ public class FacetExecutor {
                 long min = minimumsBeforeReturning[id];
                 long currentCount = counts.get(id);
                 if (currentCount < min) {
-                  LOG.debug("Running facet, current count [{0}] min [{1}]", currentCount, min);
+                  LOG.debug(getPrefix("Running facet, current count [{0}] min [{1}]"), currentCount, min);
                   SimpleCollectorExitEarly col = new SimpleCollectorExitEarly(_bitSet, currentCount, min);
                   runFacet(counts, col, id);
                 }
@@ -188,6 +201,9 @@ public class FacetExecutor {
               ids.put(id);
             }
           }
+          if (!running.get()) {
+            LOG.debug(getPrefix("running was stopped."));
+          }
         } catch (Exception e) {
           throw new IOException(e);
         }
@@ -206,19 +222,26 @@ public class FacetExecutor {
         Tracer traceInner = Trace.trace("processing facet - segment - scorer", Trace.param("scorer", scorer),
             Trace.param("scorer.cost", scorer.cost()));
         try {
-          // new ExitScorer(scorer).score(col);
+          LOG.debug(getPrefix("starting scorer [" + i + "]."));
           scorer.score(col);
         } catch (Finished e) {
           // Do nothing, exiting early.
+          LOG.debug(getPrefix("finished early."));
         } finally {
           traceInner.done();
         }
         int hits = col._hits;
-        LOG.debug("Facet [{0}] result [{1}]", i, hits);
+        LOG.debug(getPrefix("Facet [{0}] result [{1}]"), i, hits);
         counts.addAndGet(i, hits);
+      } else {
+        LOG.debug(getPrefix("scorer [" + i + "] is null."));
       }
       col._hits = 0;
     }
+
+    private String getPrefix(String s) {
+      return _instance + " " + s;
+    }
   }
 
   private final Map<Object, Info> _infoMap = new ConcurrentHashMap<Object, FacetExecutor.Info>();
@@ -228,6 +251,7 @@ public class FacetExecutor {
   private final Lock[] _locks;
   private final AtomicBoolean _running;
   private boolean _processed;
+  private final String _instance = UUID.randomUUID().toString();
 
   public FacetExecutor(int length) {
     this(length, null, new AtomicLongArray(length));
@@ -250,26 +274,34 @@ public class FacetExecutor {
       _locks[i] = new ReentrantReadWriteLock().writeLock();
     }
     _running = running;
+    User user = UserContext.getUser();
+    LOG.debug(getPrefix("User [{0}]"), user);
   }
 
   public void addScorers(AtomicReaderContext context, Scorer[] scorers) throws IOException {
+    LOG.debug(getPrefix("adding scorers context [{0}] [{1}]"), context, Arrays.asList(scorers));
     if (scorers.length != _length) {
       throw new IOException("Scorer length is not correct expecting [" + _length + "] actual [" + scorers.length + "]");
     }
     Object key = getKey(context);
     Info info = _infoMap.get(key);
     if (info == null) {
-      info = new Info(context, scorers, _locks);
+      info = new Info(context, scorers, _locks, _instance);
       _infoMap.put(key, info);
     } else {
       AtomicReader reader = context.reader();
-      LOG.warn("Info about reader context [{0}] already created, existing Info [{1}] current reader [{2}].", context,
-          info, reader);
+      LOG.warn(getPrefix("Info about reader context [{0}] already created, existing Info [{1}] current reader [{2}]."),
+          context, info, reader);
     }
   }
 
+  public String getPrefix(String s) {
+    return _instance + " " + s;
+  }
+
   public boolean scorersAlreadyAdded(AtomicReaderContext context) {
     Object key = getKey(context);
+    LOG.debug(getPrefix("scorersAlreadyAdded key [{0}]"), context);
     return _infoMap.containsKey(key);
   }
 
@@ -297,7 +329,9 @@ public class FacetExecutor {
   }
 
   public void processFacets(ExecutorService executor) throws IOException {
+    LOG.debug(getPrefix("processFacets called"));
     if (!_processed) {
+      LOG.debug(getPrefix("processing Facets"));
       Tracer trace = Trace.trace("processing facets");
       try {
         processInternal(executor);
@@ -310,11 +344,16 @@ public class FacetExecutor {
 
   private void processInternal(ExecutorService executor) throws IOException {
     List<Entry<Object, Info>> entries = new ArrayList<Entry<Object, Info>>(_infoMap.entrySet());
+    LOG.debug(getPrefix("entries count [{0}]"), entries.size());
     Collections.sort(entries, COMPARATOR);
     if (executor == null) {
+      LOG.debug(getPrefix("no executor"), entries.size());
       for (Entry<Object, Info> e : entries) {
         if (_running.get()) {
+          LOG.debug(getPrefix("processing [{0}] [{1}]"), e.getKey(), e.getValue());
           e.getValue().process(_counts, _minimumsBeforeReturning, _running);
+        } else {
+          LOG.debug(getPrefix("No longer running."));
         }
       }
     } else {
@@ -326,14 +365,18 @@ public class FacetExecutor {
             @Override
             public void run() {
               try {
+                LOG.debug(getPrefix("processing [{0}] [{1}]"), entry.getKey(), entry.getValue());
                 entry.getValue().process(_counts, _minimumsBeforeReturning, _running);
               } catch (Throwable e) {
-                LOG.error("Unknown error", e);
+                LOG.error(getPrefix("Unknown error"), e);
               } finally {
                 finished.incrementAndGet();
               }
             }
           });
+        } else {
+          LOG.debug(getPrefix("No longer running."));
+          return;
         }
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-query/src/main/java/org/apache/blur/lucene/search/FacetQuery.java
----------------------------------------------------------------------
diff --git a/blur-query/src/main/java/org/apache/blur/lucene/search/FacetQuery.java b/blur-query/src/main/java/org/apache/blur/lucene/search/FacetQuery.java
index fbb342c..74ee6e0 100644
--- a/blur-query/src/main/java/org/apache/blur/lucene/search/FacetQuery.java
+++ b/blur-query/src/main/java/org/apache/blur/lucene/search/FacetQuery.java
@@ -17,6 +17,7 @@ package org.apache.blur.lucene.search;
  * limitations under the License.
  */
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.blur.log.Log;
 import org.apache.blur.log.LogFactory;
@@ -32,6 +33,8 @@ import org.apache.lucene.util.OpenBitSet;
 
 public class FacetQuery extends AbstractWrapperQuery {
 
+  private static final Log LOG = LogFactory.getLog(FacetQuery.class);
+
   private final Query[] _facets;
   private final FacetExecutor _executor;
 
@@ -69,10 +72,11 @@ public class FacetQuery extends AbstractWrapperQuery {
     if (_rewritten) {
       return this;
     }
+    Query[] facets = new Query[_facets.length];
     for (int i = 0; i < _facets.length; i++) {
-      _facets[i] = _facets[i].rewrite(reader);
+      facets[i] = _facets[i].rewrite(reader);
     }
-    return new FacetQuery(_query.rewrite(reader), _facets, _executor, true);
+    return new FacetQuery(_query.rewrite(reader), facets, _executor, true);
   }
 
   @Override
@@ -125,7 +129,11 @@ public class FacetQuery extends AbstractWrapperQuery {
       }
       if (!_executor.scorersAlreadyAdded(context)) {
         Scorer[] scorers = getScorers(context, true, topScorer, acceptDocs);
-        _executor.addScorers(context, scorers);  
+        LOG.debug(_executor.getPrefix("Adding scorers for context [{0}] scorers [{1}]"), context,
+            Arrays.asList(scorers));
+        _executor.addScorers(context, scorers);
+      } else {
+        LOG.debug(_executor.getPrefix("Scorers already added for context [{0}]"), context);
       }
       return new FacetScorer(scorer, _executor, context);
     }

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-query/src/test/java/org/apache/blur/analysis/type/BaseReadMaskFieldTypeDefinitionTest.java
----------------------------------------------------------------------
diff --git a/blur-query/src/test/java/org/apache/blur/analysis/type/BaseReadMaskFieldTypeDefinitionTest.java b/blur-query/src/test/java/org/apache/blur/analysis/type/BaseReadMaskFieldTypeDefinitionTest.java
index 8be969b..4f4f29c 100644
--- a/blur-query/src/test/java/org/apache/blur/analysis/type/BaseReadMaskFieldTypeDefinitionTest.java
+++ b/blur-query/src/test/java/org/apache/blur/analysis/type/BaseReadMaskFieldTypeDefinitionTest.java
@@ -49,7 +49,6 @@ import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -160,7 +159,7 @@ public abstract class BaseReadMaskFieldTypeDefinitionTest {
   }
 
   private void setupFieldManager(BaseFieldManager fieldManager) throws IOException {
-    fieldManager.addColumnDefinition(FAM, "string", null, false, "string", true, false, null);
+    fieldManager.addColumnDefinition(FAM, "string", null, false, "string", false, false, null);
     fieldManager.addColumnDefinition(FAM, "string2", null, false, "string", false, false, null);
     fieldManager.addColumnDefinition(FAM, "read", null, false, "acl-read", false, false, null);
     fieldManager.addColumnDefinition(FAM, "mask", null, false, "read-mask", false, false, null);
@@ -222,16 +221,6 @@ public abstract class BaseReadMaskFieldTypeDefinitionTest {
           assertEquals(defaultReadMask, s);
         }
       }
-
-      String s = document.get("fam.string");
-      if (s == null || s.equals(getDefaultReadMask())) {
-        AtomicReader atomicReader = searcher.getIndexReader().leaves().get(0).reader();
-        SortedDocValues sortedDocValues = atomicReader.getSortedDocValues("fam.string");
-        BytesRef result = new BytesRef();
-        sortedDocValues.get(doc, result);
-        assertEquals(0, result.length);
-      }
-
     }
 
     reader.close();

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-shell/pom.xml
----------------------------------------------------------------------
diff --git a/blur-shell/pom.xml b/blur-shell/pom.xml
index fd7e92d..8834a3c 100644
--- a/blur-shell/pom.xml
+++ b/blur-shell/pom.xml
@@ -61,6 +61,13 @@
 	<build>
 		<finalName>blur-shell-${project.version}</finalName>
 		<plugins>
+
+			<plugin>
+				<groupId>org.codehaus.mojo</groupId>
+				<artifactId>exec-maven-plugin</artifactId>
+				<version>1.2.1</version>
+			</plugin>
+
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-source-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-shell/src/main/java/org/apache/blur/shell/ListRunningPlatformCommandsCommand.java
----------------------------------------------------------------------
diff --git a/blur-shell/src/main/java/org/apache/blur/shell/ListRunningPlatformCommandsCommand.java b/blur-shell/src/main/java/org/apache/blur/shell/ListRunningPlatformCommandsCommand.java
index 2b48374..d92ad80 100644
--- a/blur-shell/src/main/java/org/apache/blur/shell/ListRunningPlatformCommandsCommand.java
+++ b/blur-shell/src/main/java/org/apache/blur/shell/ListRunningPlatformCommandsCommand.java
@@ -40,13 +40,24 @@ public class ListRunningPlatformCommandsCommand extends Command {
     if (args.length != 1) {
       throw new CommandException("Invalid args: " + help());
     }
-
     List<String> commandStatusList = client.commandStatusList(0, Short.MAX_VALUE);
     RunningSummary runningSummary = new RunningSummary();
     for (String id : commandStatusList) {
-      CommandStatus commandStatus = client.commandStatus(id);
+      CommandStatus commandStatus;
+      try {
+        commandStatus = client.commandStatus(id);
+      } catch (BlurException e) {
+        String message = e.getMessage();
+        if (message != null && message.startsWith("NOT_FOUND")) {
+          commandStatus = null;
+        } else {
+          throw e;
+        }
+      }
+      if (commandStatus == null) {
+        continue;
+      }
       Map<String, Map<CommandStatusState, Long>> serverStateMap = commandStatus.getServerStateMap();
-      out.println(serverStateMap);
       Map<CommandStatusState, Long> summary = getSummary(serverStateMap);
       if (summary.containsKey(CommandStatusState.RUNNING)) {
         runningSummary.add(commandStatus, summary);
@@ -56,7 +67,7 @@ public class ListRunningPlatformCommandsCommand extends Command {
     runningSummary.print(out);
   }
 
-  private Map<CommandStatusState, Long> getSummary(Map<String, Map<CommandStatusState, Long>> serverStateMap) {
+  public static Map<CommandStatusState, Long> getSummary(Map<String, Map<CommandStatusState, Long>> serverStateMap) {
     Map<CommandStatusState, Long> map = new HashMap<CommandStatusState, Long>();
     for (Map<CommandStatusState, Long> m : serverStateMap.values()) {
       for (Entry<CommandStatusState, Long> e : m.entrySet()) {
@@ -95,18 +106,7 @@ public class ListRunningPlatformCommandsCommand extends Command {
       String executionId = commandStatus.getExecutionId();
       String commandName = commandStatus.getCommandName();
       User user = commandStatus.getUser();
-      _summary.add(Arrays.asList(executionId, commandName, user.getUsername(), toString(summary)));
-    }
-
-    private String toString(Map<CommandStatusState, Long> summary) {
-      StringBuilder builder = new StringBuilder();
-      for (Entry<CommandStatusState, Long> e : summary.entrySet()) {
-        if (builder.length() != 0) {
-          builder.append(',');
-        }
-        builder.append(e.getKey().name()).append(":").append(e.getValue());
-      }
-      return builder.toString();
+      _summary.add(Arrays.asList(executionId, commandName, user.getUsername(), toStringSummary(summary)));
     }
 
     public void print(PrintWriter out) {
@@ -158,4 +158,15 @@ public class ListRunningPlatformCommandsCommand extends Command {
       return len;
     }
   }
+
+  public static String toStringSummary(Map<CommandStatusState, Long> summary) {
+    StringBuilder builder = new StringBuilder();
+    for (Entry<CommandStatusState, Long> e : summary.entrySet()) {
+      if (builder.length() != 0) {
+        builder.append(',');
+      }
+      builder.append(e.getKey().name()).append(":").append(e.getValue());
+    }
+    return builder.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-shell/src/main/java/org/apache/blur/shell/Main.java
----------------------------------------------------------------------
diff --git a/blur-shell/src/main/java/org/apache/blur/shell/Main.java b/blur-shell/src/main/java/org/apache/blur/shell/Main.java
index bf4b0b5..82d8a42 100644
--- a/blur-shell/src/main/java/org/apache/blur/shell/Main.java
+++ b/blur-shell/src/main/java/org/apache/blur/shell/Main.java
@@ -400,7 +400,7 @@ public class Main {
   public static String[] shellCommands = { "help", "debug", "timed", "quit", "reset", "user", "whoami", "trace",
       "trace-remove", "trace-list" };
   public static String[] platformCommands = { "command-list", "command-exec", "command-desc", "command-running",
-      "command-cancel" };
+      "command-cancel", "command-watch" };
   public static String[] serverCommands = { "logger", "logger-reset", "remove-shard" };
 
   private static class HelpCommand extends Command {
@@ -704,6 +704,7 @@ public class Main {
     register(builder, new ImportDataCommand());
     register(builder, new ListRunningPlatformCommandsCommand());
     register(builder, new CancelPlatformCommandCommand());
+    register(builder, new WatchCommands());
     commands = builder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-shell/src/main/java/org/apache/blur/shell/TableDisplay.java
----------------------------------------------------------------------
diff --git a/blur-shell/src/main/java/org/apache/blur/shell/TableDisplay.java b/blur-shell/src/main/java/org/apache/blur/shell/TableDisplay.java
index 94783e8..31a08e8 100644
--- a/blur-shell/src/main/java/org/apache/blur/shell/TableDisplay.java
+++ b/blur-shell/src/main/java/org/apache/blur/shell/TableDisplay.java
@@ -289,7 +289,7 @@ public class TableDisplay implements Closeable {
       width--;
     }
   }
-  
+
   private void buffer(Canvas canvas, String value, int width) {
     canvas.append(value);
     width -= getVisibleLength(value);
@@ -683,4 +683,5 @@ public class TableDisplay implements Closeable {
   public void setStopReadingInput(boolean b) {
     _stopReadingInput.set(true);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-shell/src/main/java/org/apache/blur/shell/WatchCommands.java
----------------------------------------------------------------------
diff --git a/blur-shell/src/main/java/org/apache/blur/shell/WatchCommands.java b/blur-shell/src/main/java/org/apache/blur/shell/WatchCommands.java
new file mode 100644
index 0000000..4b284e9
--- /dev/null
+++ b/blur-shell/src/main/java/org/apache/blur/shell/WatchCommands.java
@@ -0,0 +1,149 @@
+/**
+ * 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.blur.shell;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import jline.console.ConsoleReader;
+
+import org.apache.blur.thirdparty.thrift_0_9_0.TException;
+import org.apache.blur.thrift.generated.Blur;
+import org.apache.blur.thrift.generated.Blur.Iface;
+import org.apache.blur.thrift.generated.BlurException;
+import org.apache.blur.thrift.generated.CommandStatus;
+import org.apache.blur.thrift.generated.CommandStatusState;
+import org.apache.blur.thrift.generated.User;
+
+public class WatchCommands extends Command {
+
+  @Override
+  public void doit(PrintWriter out, Blur.Iface client, String[] args) throws CommandException, TException,
+      BlurException {
+    ConsoleReader reader = this.getConsoleReader();
+    try {
+      doitInternal(client, reader);
+    } catch (IOException e) {
+      if (Main.debug) {
+        e.printStackTrace();
+      }
+      throw new CommandException(e.getMessage());
+    } finally {
+      if (reader != null) {
+        reader.setPrompt(Main.PROMPT);
+      }
+    }
+  }
+
+  private static void doitInternal(Iface client, ConsoleReader reader) throws IOException, TException, CommandException {
+    TableDisplay tableDisplay = new TableDisplay(reader);
+    tableDisplay.setSeperator("|");
+    tableDisplay.setHeader(0, "id");
+    tableDisplay.setHeader(1, "command");
+    tableDisplay.setHeader(2, "user");
+    tableDisplay.setHeader(3, "summary");
+
+    final AtomicBoolean running = new AtomicBoolean(true);
+    tableDisplay.addKeyHook(new Runnable() {
+      @Override
+      public void run() {
+        synchronized (running) {
+          running.set(false);
+          running.notifyAll();
+        }
+      }
+    }, 'q');
+
+    try {
+      int maxL = 0;
+      while (running.get()) {
+
+        List<String> commandStatusList = client.commandStatusList(0, Short.MAX_VALUE);
+        List<String[]> display = new ArrayList<String[]>();
+        for (String id : commandStatusList) {
+          CommandStatus commandStatus;
+          try {
+            commandStatus = client.commandStatus(id);
+          } catch (BlurException e) {
+            String message = e.getMessage();
+            if (message != null && message.startsWith("NOT_FOUND")) {
+              commandStatus = null;
+            } else {
+              throw e;
+            }
+          }
+          if (commandStatus == null) {
+            continue;
+          }
+          Map<String, Map<CommandStatusState, Long>> serverStateMap = commandStatus.getServerStateMap();
+          Map<CommandStatusState, Long> summary = ListRunningPlatformCommandsCommand.getSummary(serverStateMap);
+          String executionId = commandStatus.getExecutionId();
+          String commandName = commandStatus.getCommandName();
+          User user = commandStatus.getUser();
+          if (summary.containsKey(CommandStatusState.RUNNING)) {
+            String stringSummary = ListRunningPlatformCommandsCommand.toStringSummary(summary);
+            display.add(new String[] { executionId, commandName, user.toString(), stringSummary });
+          } else if (summary.containsKey(CommandStatusState.INTERRUPTED)) {
+            display
+                .add(new String[] { executionId, commandName, user.toString(), CommandStatusState.INTERRUPTED.name() });
+          } else {
+            display.add(new String[] { executionId, commandName, user.toString(), CommandStatusState.COMPLETE.name() });
+          }
+        }
+
+        int l = 0;
+        for (String[] array : display) {
+          tableDisplay.set(0, l, array[0]);
+          tableDisplay.set(1, l, array[1]);
+          tableDisplay.set(2, l, array[2]);
+          tableDisplay.set(3, l, array[3]);
+          l++;
+        }
+        if (l > maxL) {
+          maxL = l;
+        }
+        Thread.sleep(3000);
+      }
+    } catch (InterruptedException e) {
+      if (Main.debug) {
+        e.printStackTrace();
+      }
+      throw new CommandException(e.getMessage());
+    } finally {
+      tableDisplay.close();
+    }
+  }
+
+  @Override
+  public String description() {
+    return "Watch commands execute.";
+  }
+
+  @Override
+  public String usage() {
+    return "";
+  }
+
+  @Override
+  public String name() {
+    return "command-watch";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-store/src/main/java/org/apache/blur/lucene/codec/DiskDocValuesProducer.java
----------------------------------------------------------------------
diff --git a/blur-store/src/main/java/org/apache/blur/lucene/codec/DiskDocValuesProducer.java b/blur-store/src/main/java/org/apache/blur/lucene/codec/DiskDocValuesProducer.java
index 3bc6737..fd617b6 100644
--- a/blur-store/src/main/java/org/apache/blur/lucene/codec/DiskDocValuesProducer.java
+++ b/blur-store/src/main/java/org/apache/blur/lucene/codec/DiskDocValuesProducer.java
@@ -18,8 +18,8 @@ package org.apache.blur.lucene.codec;
  */
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.blur.trace.Trace;
 import org.apache.blur.trace.Tracer;
@@ -41,25 +41,33 @@ import org.apache.lucene.util.packed.BlockPackedReader;
 import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
 
 class DiskDocValuesProducer extends DocValuesProducer {
-  private final Map<Integer,NumericEntry> numerics;
-  private final Map<Integer,BinaryEntry> binaries;
-  private final Map<Integer,NumericEntry> ords;
-  private final Map<Integer,NumericEntry> ordIndexes;
+  private final Map<Integer, NumericEntry> numerics;
+  private final Map<Integer, BinaryEntry> binaries;
+  private final Map<Integer, NumericEntry> ords;
+  private final Map<Integer, NumericEntry> ordIndexes;
+  private final Map<Integer, BinaryDocValues> _binaryDocValuesCache;
+  private final Map<Integer, NumericDocValues> _numericDocValuesCache;
+  private final Map<Integer, SortedDocValues> _sortedDocValuesCache;
+  private final Map<Integer, SortedSetDocValues> _sortedSetDocValuesCache;
   private final IndexInput data;
+  private final boolean _cache = true;
 
-  DiskDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+  DiskDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec,
+      String metaExtension) throws IOException {
     String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
     // read in the entries from the metadata file.
     IndexInput in = state.directory.openInput(metaName, state.context);
     boolean success = false;
     try {
-      CodecUtil.checkHeader(in, metaCodec, 
-                                DiskDocValuesFormat.VERSION_START,
-                                DiskDocValuesFormat.VERSION_START);
-      numerics = new HashMap<Integer,NumericEntry>();
-      ords = new HashMap<Integer,NumericEntry>();
-      ordIndexes = new HashMap<Integer,NumericEntry>();
-      binaries = new HashMap<Integer,BinaryEntry>();
+      CodecUtil.checkHeader(in, metaCodec, DiskDocValuesFormat.VERSION_START, DiskDocValuesFormat.VERSION_START);
+      numerics = new ConcurrentHashMap<Integer, NumericEntry>();
+      ords = new ConcurrentHashMap<Integer, NumericEntry>();
+      ordIndexes = new ConcurrentHashMap<Integer, NumericEntry>();
+      binaries = new ConcurrentHashMap<Integer, BinaryEntry>();
+      _binaryDocValuesCache = new ConcurrentHashMap<Integer, BinaryDocValues>();
+      _numericDocValuesCache = new ConcurrentHashMap<Integer, NumericDocValues>();
+      _sortedDocValuesCache = new ConcurrentHashMap<Integer, SortedDocValues>();
+      _sortedSetDocValuesCache = new ConcurrentHashMap<Integer, SortedSetDocValues>();
       readFields(in, state.fieldInfos);
       success = true;
     } finally {
@@ -69,14 +77,12 @@ class DiskDocValuesProducer extends DocValuesProducer {
         IOUtils.closeWhileHandlingException(in);
       }
     }
-    
+
     String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
     data = state.directory.openInput(dataName, state.context);
-    CodecUtil.checkHeader(data, dataCodec, 
-                                DiskDocValuesFormat.VERSION_START,
-                                DiskDocValuesFormat.VERSION_START);
+    CodecUtil.checkHeader(data, dataCodec, DiskDocValuesFormat.VERSION_START, DiskDocValuesFormat.VERSION_START);
   }
-  
+
   private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
     int fieldNumber = meta.readVInt();
     while (fieldNumber != -1) {
@@ -96,7 +102,7 @@ class DiskDocValuesProducer extends DocValuesProducer {
         }
         BinaryEntry b = readBinaryEntry(meta);
         binaries.put(fieldNumber, b);
-        
+
         if (meta.readVInt() != fieldNumber) {
           throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
         }
@@ -115,7 +121,7 @@ class DiskDocValuesProducer extends DocValuesProducer {
         }
         BinaryEntry b = readBinaryEntry(meta);
         binaries.put(fieldNumber, b);
-        
+
         if (meta.readVInt() != fieldNumber) {
           throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
         }
@@ -124,7 +130,7 @@ class DiskDocValuesProducer extends DocValuesProducer {
         }
         NumericEntry n1 = readNumericEntry(meta);
         ords.put(fieldNumber, n1);
-        
+
         if (meta.readVInt() != fieldNumber) {
           throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
         }
@@ -139,7 +145,7 @@ class DiskDocValuesProducer extends DocValuesProducer {
       fieldNumber = meta.readVInt();
     }
   }
-  
+
   static NumericEntry readNumericEntry(IndexInput meta) throws IOException {
     NumericEntry entry = new NumericEntry();
     entry.packedIntsVersion = meta.readVInt();
@@ -148,7 +154,7 @@ class DiskDocValuesProducer extends DocValuesProducer {
     entry.blockSize = meta.readVInt();
     return entry;
   }
-  
+
   static BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
     BinaryEntry entry = new BinaryEntry();
     entry.minLength = meta.readVInt();
@@ -165,15 +171,30 @@ class DiskDocValuesProducer extends DocValuesProducer {
 
   @Override
   public NumericDocValues getNumeric(FieldInfo field) throws IOException {
-    NumericEntry entry = numerics.get(field.number);
-    return getNumeric(entry);
+    NumericDocValues numericDocValues = _numericDocValuesCache.get(field.number);
+    if (numericDocValues != null) {
+      return numericDocValues;
+    }
+    synchronized (_numericDocValuesCache) {
+      numericDocValues = _numericDocValuesCache.get(field.number);
+      if (numericDocValues != null) {
+        return numericDocValues;
+      }
+      NumericEntry entry = numerics.get(field.number);
+      numericDocValues = newNumeric(entry);
+      if (_cache && numericDocValues != null) {
+        _numericDocValuesCache.put(field.number, numericDocValues);
+      }
+      return numericDocValues;
+    }
   }
-  
-  LongNumericDocValues getNumeric(NumericEntry entry) throws IOException {
+
+  LongNumericDocValues newNumeric(NumericEntry entry) throws IOException {
     final IndexInput data = this.data.clone();
     data.seek(entry.offset);
 
-    final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
+    final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count,
+        true);
     return new LongNumericDocValues() {
       @Override
       public long get(long id) {
@@ -184,6 +205,24 @@ class DiskDocValuesProducer extends DocValuesProducer {
 
   @Override
   public BinaryDocValues getBinary(FieldInfo field) throws IOException {
+    BinaryDocValues binaryDocValues = _binaryDocValuesCache.get(field.number);
+    if (binaryDocValues != null) {
+      return binaryDocValues;
+    }
+    synchronized (_binaryDocValuesCache) {
+      binaryDocValues = _binaryDocValuesCache.get(field.number);
+      if (binaryDocValues != null) {
+        return binaryDocValues;
+      }
+      binaryDocValues = newBinary(field);
+      if (_cache && binaryDocValues != null) {
+        _binaryDocValuesCache.put(field.number, binaryDocValues);
+      }
+      return binaryDocValues;
+    }
+  }
+
+  private BinaryDocValues newBinary(FieldInfo field) throws IOException {
     BinaryEntry bytes = binaries.get(field.number);
     if (bytes.minLength == bytes.maxLength) {
       return getFixedBinary(field, bytes);
@@ -191,20 +230,30 @@ class DiskDocValuesProducer extends DocValuesProducer {
       return getVariableBinary(field, bytes);
     }
   }
-  
+
   private BinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) {
     final IndexInput data = this.data.clone();
 
     return new LongBinaryDocValues() {
+
+      private final ThreadLocal<IndexInput> in = new ThreadLocal<IndexInput>() {
+        @Override
+        protected IndexInput initialValue() {
+          return data.clone();
+        }
+      };
+
       @Override
       public void get(long id, BytesRef result) {
         long address = bytes.offset + id * bytes.maxLength;
         try {
-          data.seek(address);
-          // NOTE: we could have one buffer, but various consumers (e.g. FieldComparatorSource) 
+          IndexInput indexInput = in.get();
+          indexInput.seek(address);
+          // NOTE: we could have one buffer, but various consumers (e.g.
+          // FieldComparatorSource)
           // assume "they" own the bytes after calling this!
           final byte[] buffer = new byte[bytes.maxLength];
-          data.readBytes(buffer, 0, buffer.length);
+          indexInput.readBytes(buffer, 0, buffer.length);
           result.bytes = buffer;
           result.offset = 0;
           result.length = buffer.length;
@@ -214,10 +263,10 @@ class DiskDocValuesProducer extends DocValuesProducer {
       }
     };
   }
-  
+
   private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
     final IndexInput data = this.data.clone();
-    
+
     Tracer trace = Trace.trace("getSorted - BlockPackedReader - create");
     final MonotonicBlockPackedReader addresses;
     try {
@@ -227,17 +276,27 @@ class DiskDocValuesProducer extends DocValuesProducer {
       trace.done();
     }
     return new LongBinaryDocValues() {
+      
+      private final ThreadLocal<IndexInput> _input = new ThreadLocal<IndexInput>() {
+        @Override
+        protected IndexInput initialValue() {
+          return data.clone();
+        }
+      };
+      
       @Override
       public void get(long id, BytesRef result) {
-        long startAddress = bytes.offset + (id == 0 ? 0 : addresses.get(id-1));
+        long startAddress = bytes.offset + (id == 0 ? 0 : addresses.get(id - 1));
         long endAddress = bytes.offset + addresses.get(id);
         int length = (int) (endAddress - startAddress);
         try {
-          data.seek(startAddress);
-          // NOTE: we could have one buffer, but various consumers (e.g. FieldComparatorSource) 
+          IndexInput indexInput = _input.get();
+          indexInput.seek(startAddress);
+          // NOTE: we could have one buffer, but various consumers (e.g.
+          // FieldComparatorSource)
           // assume "they" own the bytes after calling this!
           final byte[] buffer = new byte[length];
-          data.readBytes(buffer, 0, buffer.length);
+          indexInput.readBytes(buffer, 0, buffer.length);
           result.bytes = buffer;
           result.offset = 0;
           result.length = length;
@@ -250,11 +309,29 @@ class DiskDocValuesProducer extends DocValuesProducer {
 
   @Override
   public SortedDocValues getSorted(FieldInfo field) throws IOException {
+    SortedDocValues sortedDocValues = _sortedDocValuesCache.get(field.number);
+    if (sortedDocValues != null) {
+      return sortedDocValues;
+    }
+    synchronized (_sortedDocValuesCache) {
+      sortedDocValues = _sortedDocValuesCache.get(field.number);
+      if (sortedDocValues != null) {
+        return sortedDocValues;
+      }
+      sortedDocValues = newSortedDocValues(field);
+      if (_cache && sortedDocValues != null) {
+        _sortedDocValuesCache.put(field.number, sortedDocValues);
+      }
+      return sortedDocValues;
+    }
+  }
+
+  private SortedDocValues newSortedDocValues(FieldInfo field) throws IOException {
     final int valueCount = (int) binaries.get(field.number).count;
     final BinaryDocValues binary = getBinary(field);
     Tracer trace = Trace.trace("getSorted - BlockPackedReader - create");
     final BlockPackedReader ordinals;
-    try{
+    try {
       NumericEntry entry = ords.get(field.number);
       IndexInput data = this.data.clone();
       data.seek(entry.offset);
@@ -283,14 +360,32 @@ class DiskDocValuesProducer extends DocValuesProducer {
 
   @Override
   public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+    SortedSetDocValues sortedSetDocValues = _sortedSetDocValuesCache.get(field.number);
+    if (sortedSetDocValues != null) {
+      return sortedSetDocValues;
+    }
+    synchronized (_sortedSetDocValuesCache) {
+      sortedSetDocValues = _sortedSetDocValuesCache.get(field.number);
+      if (sortedSetDocValues != null) {
+        return sortedSetDocValues;
+      }
+      sortedSetDocValues = newSortedSetDocValues(field);
+      if (_cache && sortedSetDocValues != null) {
+        _sortedSetDocValuesCache.put(field.number, sortedSetDocValues);
+      }
+      return sortedSetDocValues;
+    }
+  }
+
+  private SortedSetDocValues newSortedSetDocValues(FieldInfo field) throws IOException {
     final long valueCount = binaries.get(field.number).count;
     // we keep the byte[]s and list of ords on disk, these could be large
     final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
-    final LongNumericDocValues ordinals = getNumeric(ords.get(field.number));
+    final LongNumericDocValues ordinals = newNumeric(ords.get(field.number));
 
     Tracer trace = Trace.trace("getSortedSet - MonotonicBlockPackedReader - create");
     final MonotonicBlockPackedReader ordIndex;
-    try{
+    try {
       NumericEntry entry = ordIndexes.get(field.number);
       IndexInput data = this.data.clone();
       data.seek(entry.offset);
@@ -302,7 +397,7 @@ class DiskDocValuesProducer extends DocValuesProducer {
     return new SortedSetDocValues() {
       long offset;
       long endOffset;
-      
+
       @Override
       public long nextOrd() {
         if (offset == endOffset) {
@@ -316,7 +411,7 @@ class DiskDocValuesProducer extends DocValuesProducer {
 
       @Override
       public void setDocument(int docID) {
-        offset = (docID == 0 ? 0 : ordIndex.get(docID-1));
+        offset = (docID == 0 ? 0 : ordIndex.get(docID - 1));
         endOffset = ordIndex.get(docID);
       }
 
@@ -336,7 +431,7 @@ class DiskDocValuesProducer extends DocValuesProducer {
   public void close() throws IOException {
     data.close();
   }
-  
+
   static class NumericEntry {
     long offset;
 
@@ -344,7 +439,7 @@ class DiskDocValuesProducer extends DocValuesProducer {
     long count;
     int blockSize;
   }
-  
+
   static class BinaryEntry {
     long offset;
 
@@ -355,23 +450,23 @@ class DiskDocValuesProducer extends DocValuesProducer {
     int packedIntsVersion;
     int blockSize;
   }
-  
+
   // internally we compose complex dv (sorted/sortedset) from other ones
   static abstract class LongNumericDocValues extends NumericDocValues {
     @Override
     public final long get(int docID) {
       return get((long) docID);
     }
-    
+
     abstract long get(long id);
   }
-  
+
   static abstract class LongBinaryDocValues extends BinaryDocValues {
     @Override
     public final void get(int docID, BytesRef result) {
-      get((long)docID, result);
+      get((long) docID, result);
     }
-    
+
     abstract void get(long id, BytesRef Result);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/CacheIndexInput.java
----------------------------------------------------------------------
diff --git a/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/CacheIndexInput.java b/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/CacheIndexInput.java
index 18b9eda..3803cc5 100644
--- a/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/CacheIndexInput.java
+++ b/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/CacheIndexInput.java
@@ -224,6 +224,7 @@ public class CacheIndexInput extends IndexInput {
   @Override
   public void readBytes(byte[] b, int offset, int len) throws IOException {
     ensureOpen();
+    LOOP:
     while (len > 0) {
       tryToFill();
       int remaining = remaining();
@@ -232,8 +233,7 @@ public class CacheIndexInput extends IndexInput {
         _cacheValue.read(_blockPosition, b, offset, length);
       } catch (EvictionException e) {
         releaseCache();
-        readBytes(b, offset, len);
-        return;
+        continue LOOP;
       }
       offset += length;
       len -= length;

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/MeterWrapper.java
----------------------------------------------------------------------
diff --git a/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/MeterWrapper.java b/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/MeterWrapper.java
index 2e8d245..269245a 100644
--- a/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/MeterWrapper.java
+++ b/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/MeterWrapper.java
@@ -121,11 +121,14 @@ public abstract class MeterWrapper implements Closeable {
   }
 
   private static void register(String id, SimpleMeter meter, AtomicLong counter) {
-    {
-      _counterMap.putIfAbsent(id, new MeterWrapperCounter(meter));
-    }
-    {
-      _counterMap.get(id).add(counter);
+    MeterWrapperCounter meterWrapperCounter = new MeterWrapperCounter(meter);
+    while (true) {
+      _counterMap.putIfAbsent(id, meterWrapperCounter);
+      MeterWrapperCounter wrapperCounter = _counterMap.get(id);
+      if (wrapperCounter != null) {
+        wrapperCounter.add(counter);
+        return;
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/cachevalue/DetachableCacheValue.java
----------------------------------------------------------------------
diff --git a/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/cachevalue/DetachableCacheValue.java b/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/cachevalue/DetachableCacheValue.java
index e5ce99d..24eb7ca 100644
--- a/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/cachevalue/DetachableCacheValue.java
+++ b/blur-store/src/main/java/org/apache/blur/store/blockcache_v2/cachevalue/DetachableCacheValue.java
@@ -38,7 +38,6 @@ public class DetachableCacheValue implements CacheValue {
   }
 
   private volatile CacheValue _baseCacheValue;
-  private volatile boolean _evicted;
 
   public DetachableCacheValue(CacheValue cacheValue) {
     _baseCacheValue = cacheValue;
@@ -46,7 +45,6 @@ public class DetachableCacheValue implements CacheValue {
 
   @Override
   public CacheValue detachFromCache() {
-    _evicted = true;
     if (_baseCacheValue instanceof ByteArrayCacheValue) {
       // already detached
       return null;
@@ -64,8 +62,11 @@ public class DetachableCacheValue implements CacheValue {
 
   @Override
   public int length() throws EvictionException {
-    checkEviction();
-    return _baseCacheValue.length();
+    try {
+      return _baseCacheValue.length();
+    } catch (NullPointerException npe) {
+      throw new EvictionException();
+    }
   }
 
   @Override
@@ -75,20 +76,20 @@ public class DetachableCacheValue implements CacheValue {
 
   @Override
   public void read(int position, byte[] buf, int offset, int length) throws EvictionException {
-    checkEviction();
-    _baseCacheValue.read(position, buf, offset, length);
-  }
-
-  private void checkEviction() throws EvictionException {
-    if (_evicted) {
+    try {
+      _baseCacheValue.read(position, buf, offset, length);
+    } catch (NullPointerException npe) {
       throw new EvictionException();
     }
   }
 
   @Override
   public byte read(int position) throws EvictionException {
-    checkEviction();
-    return _baseCacheValue.read(position);
+    try {
+      return _baseCacheValue.read(position);
+    } catch (NullPointerException npe) {
+      throw new EvictionException();
+    }
   }
 
   @Override
@@ -100,20 +101,29 @@ public class DetachableCacheValue implements CacheValue {
 
   @Override
   public short readShort(int position) throws EvictionException {
-    checkEviction();
-    return _baseCacheValue.readShort(position);
+    try {
+      return _baseCacheValue.readShort(position);
+    } catch (NullPointerException npe) {
+      throw new EvictionException();
+    }
   }
 
   @Override
   public int readInt(int position) throws EvictionException {
-    checkEviction();
-    return _baseCacheValue.readInt(position);
+    try {
+      return _baseCacheValue.readInt(position);
+    } catch (NullPointerException npe) {
+      throw new EvictionException();
+    }
   }
 
   @Override
   public long readLong(int position) throws EvictionException {
-    checkEviction();
-    return _baseCacheValue.readLong(position);
+    try {
+      return _baseCacheValue.readLong(position);
+    } catch (NullPointerException npe) {
+      throw new EvictionException();
+    }
   }
 
   @Override
@@ -123,7 +133,7 @@ public class DetachableCacheValue implements CacheValue {
 
   @Override
   public boolean isEvicted() {
-    return _evicted;
+    return _baseCacheValue == null;
   }
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-store/src/main/java/org/apache/blur/store/hdfs/SequentialReadControl.java
----------------------------------------------------------------------
diff --git a/blur-store/src/main/java/org/apache/blur/store/hdfs/SequentialReadControl.java b/blur-store/src/main/java/org/apache/blur/store/hdfs/SequentialReadControl.java
index ac72eb9..34371e5 100644
--- a/blur-store/src/main/java/org/apache/blur/store/hdfs/SequentialReadControl.java
+++ b/blur-store/src/main/java/org/apache/blur/store/hdfs/SequentialReadControl.java
@@ -44,7 +44,8 @@ public class SequentialReadControl implements Cloneable {
   public SequentialReadControl clone() {
     try {
       SequentialReadControl control = (SequentialReadControl) super.clone();
-      setup(_configuration, control);
+      // Setup too heavy for clones
+      // setup(_configuration, control);
       return control;
     } catch (CloneNotSupportedException e) {
       throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-store/src/main/java/org/apache/lucene/util/packed/DirectPacked64SingleBlockReader.java
----------------------------------------------------------------------
diff --git a/blur-store/src/main/java/org/apache/lucene/util/packed/DirectPacked64SingleBlockReader.java b/blur-store/src/main/java/org/apache/lucene/util/packed/DirectPacked64SingleBlockReader.java
new file mode 100644
index 0000000..e5d495b
--- /dev/null
+++ b/blur-store/src/main/java/org/apache/lucene/util/packed/DirectPacked64SingleBlockReader.java
@@ -0,0 +1,65 @@
+package org.apache.lucene.util.packed;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.store.IndexInput;
+
+final class DirectPacked64SingleBlockReader extends PackedInts.ReaderImpl {
+
+  private final ThreadLocal<IndexInput> in;
+  private final long startPointer;
+  private final int valuesPerBlock;
+  private final long mask;
+
+  DirectPacked64SingleBlockReader(int bitsPerValue, int valueCount,
+      IndexInput input) {
+    super(valueCount, bitsPerValue);
+    this.in = new ThreadLocal<IndexInput>() {
+      @Override
+      protected IndexInput initialValue() {
+        return input.clone();
+      }      
+    };
+    startPointer = input.getFilePointer();
+    valuesPerBlock = 64 / bitsPerValue;
+    mask = ~(~0L << bitsPerValue);
+  }
+
+  @Override
+  public long get(int index) {
+    final int blockOffset = index / valuesPerBlock;
+    final long skip = ((long) blockOffset) << 3;
+    try {
+      IndexInput indexInput = in.get();
+      indexInput.seek(startPointer + skip);
+
+      long block = indexInput.readLong();
+      final int offsetInBlock = index % valuesPerBlock;
+      return (block >>> (offsetInBlock * bitsPerValue)) & mask;
+    } catch (IOException e) {
+      throw new IllegalStateException("failed", e);
+    }
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-store/src/main/java/org/apache/lucene/util/packed/DirectPackedReader.java
----------------------------------------------------------------------
diff --git a/blur-store/src/main/java/org/apache/lucene/util/packed/DirectPackedReader.java b/blur-store/src/main/java/org/apache/lucene/util/packed/DirectPackedReader.java
new file mode 100644
index 0000000..9483a10
--- /dev/null
+++ b/blur-store/src/main/java/org/apache/lucene/util/packed/DirectPackedReader.java
@@ -0,0 +1,80 @@
+package org.apache.lucene.util.packed;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.store.IndexInput;
+
+/* Reads directly from disk on each get */
+class DirectPackedReader extends PackedInts.ReaderImpl {
+  private final long startPointer;
+  private final ThreadLocal<IndexInput> in;
+
+  public DirectPackedReader(int bitsPerValue, int valueCount, IndexInput input) {
+    super(valueCount, bitsPerValue);
+    this.in = new ThreadLocal<IndexInput>() {
+      @Override
+      protected IndexInput initialValue() {
+        return input.clone();
+      }      
+    };
+    startPointer = input.getFilePointer();
+  }
+
+  @Override
+  public long get(int index) {
+    final long majorBitPos = (long)index * bitsPerValue;
+    final long elementPos = majorBitPos >>> 3;
+    try {
+      IndexInput indexInput = in.get();
+      indexInput .seek(startPointer + elementPos);
+
+      final byte b0 = indexInput.readByte();
+      final int bitPos = (int) (majorBitPos & 7);
+      if (bitPos + bitsPerValue <= 8) {
+        // special case: all bits are in the first byte
+        return (b0 & ((1L << (8 - bitPos)) - 1)) >>> (8 - bitPos - bitsPerValue);
+      }
+
+      // take bits from the first byte
+      int remainingBits = bitsPerValue - 8 + bitPos;
+      long result = (b0 & ((1L << (8 - bitPos)) - 1)) << remainingBits;
+
+      // add bits from inner bytes
+      while (remainingBits >= 8) {
+        remainingBits -= 8;
+        result |= (indexInput.readByte() & 0xFFL) << remainingBits;
+      }
+
+      // take bits from the last byte
+      if (remainingBits > 0) {
+        result |= (indexInput.readByte() & 0xFFL) >>> (8 - remainingBits);
+      }
+
+      return result;
+    } catch (IOException ioe) {
+      throw new IllegalStateException("failed", ioe);
+    }
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/blur-util/src/main/java/org/apache/blur/utils/BlurConstants.java
----------------------------------------------------------------------
diff --git a/blur-util/src/main/java/org/apache/blur/utils/BlurConstants.java b/blur-util/src/main/java/org/apache/blur/utils/BlurConstants.java
index 472481b..509c0f0 100644
--- a/blur-util/src/main/java/org/apache/blur/utils/BlurConstants.java
+++ b/blur-util/src/main/java/org/apache/blur/utils/BlurConstants.java
@@ -202,8 +202,11 @@ public class BlurConstants {
   public static final String SHARED_MERGE_SCHEDULER_PREFIX = "shared-merge-scheduler";
 
   public static final String BLUR_FILTER_ALIAS = "blur.filter.alias.";
-  
+
   public static final String HADOOP_CONF = "hadoop_conf.";
+  public static final String UPDATE_ROW = "_update_row_";
+  public static final String NEW_ROW = "_new_row_";
+  public static final String INTERNAL = "blur.internal";
 
   static {
     try {

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/96a1821a/deploy.sh
----------------------------------------------------------------------
diff --git a/deploy.sh b/deploy.sh
new file mode 100644
index 0000000..2731a85
--- /dev/null
+++ b/deploy.sh
@@ -0,0 +1,62 @@
+#!/usr/bin/env bash
+
+# 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.
+
+URL="<URL HERE>"
+REPO_ID="snapshots"
+
+#mvn install -D${PROFILE} -DskipTests
+#[ $? -eq 0 ] || exit $?;
+
+CUR_DIR=`pwd`
+for FILE in *; do
+  if [ -d $FILE ]
+  then
+    if [ -f $FILE/pom.xml ]
+    then
+      echo "#######################################"
+      echo "# Deploying $FILE"
+      echo "#######################################"
+      
+      cd $FILE
+
+      VERSION=`mvn help:evaluate -Dexpression=project.version | grep -v "\[INFO\]" | grep -v "\[WARNING\]"`
+      ARTIFACT=`mvn help:evaluate -Dexpression=project.artifactId | grep -v "\[INFO\]" | grep -v "\[WARNING\]"`
+
+      JAR="target/${ARTIFACT}-${VERSION}.jar"
+      JAR_SOURCES="target/${ARTIFACT}-${VERSION}-sources.jar"
+      TESTS_JAR="target/${ARTIFACT}-${VERSION}-tests.jar"
+      if [ -f $JAR ]
+      then
+        if [ -f target/effective-pom.xml ]
+        then
+          echo "Args PWD=$PWD REPO_ID=${REPO_ID} URL=${URL} ARTIFACT=${ARTIFACT} VERSION=${VERSION}"
+          if [ -f $TESTS_JAR ]
+          then
+            mvn deploy:deploy-file -DrepositoryId=${REPO_ID} -Durl=${URL} -Dfile=$JAR -DpomFile=target/effective-pom.xml -Dtypes=jar -Dclassifiers=tests -Dfiles=$TESTS_JAR -Dsources=$JAR_SOURCES 
+          else
+            mvn deploy:deploy-file -DrepositoryId=${REPO_ID} -Durl=${URL} -Dfile=$JAR -DpomFile=target/effective-pom.xml 
+          fi
+          [ $? -eq 0 ] || exit $?;
+        else 
+          echo "No effective-pom.xml to deploy, SKIPPING."
+        fi
+      fi
+      cd $CUR_DIR
+    fi
+  fi
+done
+


[10/13] git commit: Adding blur indexer project.

Posted by am...@apache.org.
Adding blur indexer project.


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

Branch: refs/heads/master
Commit: 98359a409be47ceb9f06468e3bf701f299c35fe2
Parents: 1d103e4
Author: Aaron McCurry <am...@gmail.com>
Authored: Mon Aug 29 19:16:45 2016 -0400
Committer: Aaron McCurry <am...@gmail.com>
Committed: Mon Aug 29 19:16:45 2016 -0400

----------------------------------------------------------------------
 .../apache/blur/indexer/BlurIndexCounter.java   |  27 +
 .../org/apache/blur/indexer/ClusterDriver.java  | 365 ++++++++++++++
 .../HdfsConfigurationNamespaceMerge.java        | 131 +++++
 .../apache/blur/indexer/IndexerJobDriver.java   | 498 +++++++++++++++++++
 .../blur/indexer/InputSplitPruneUtil.java       | 149 ++++++
 .../blur/indexer/MergeSortRowIdMatcher.java     | 386 ++++++++++++++
 .../ExistingDataIndexLookupMapper.java          | 230 +++++++++
 .../indexer/mapreduce/ExistingDataMapper.java   |  49 ++
 .../indexer/mapreduce/LookupBuilderMapper.java  |  34 ++
 .../indexer/mapreduce/LookupBuilderReducer.java | 184 +++++++
 .../blur/indexer/mapreduce/NewDataMapper.java   |  85 ++++
 .../mapreduce/PrunedBlurInputFormat.java        |  74 +++
 .../PrunedSequenceFileInputFormat.java          |  76 +++
 .../mapreduce/lib/update/BlurIndexCounter.java  |  27 -
 .../mapreduce/lib/update/ClusterDriver.java     | 378 --------------
 .../blur/mapreduce/lib/update/FasterDriver.java | 486 ------------------
 .../update/HdfsConfigurationNamespaceMerge.java | 131 -----
 .../lib/update/InputSplitPruneUtil.java         | 149 ------
 .../lib/update/LookupBuilderMapper.java         |  34 --
 .../lib/update/LookupBuilderReducer.java        | 181 -------
 .../lib/update/MapperForExistingDataMod.java    |  46 --
 .../MapperForExistingDataWithIndexLookup.java   | 228 ---------
 .../lib/update/MapperForNewDataMod.java         |  82 ---
 .../lib/update/MergeSortRowIdMatcher.java       | 388 ---------------
 .../lib/update/PrunedBlurInputFormat.java       |  73 ---
 .../update/PrunedSequenceFileInputFormat.java   |  75 ---
 distribution-bin/pom.xml                        |   5 +
 27 files changed, 2293 insertions(+), 2278 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/indexer/BlurIndexCounter.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/indexer/BlurIndexCounter.java b/blur-indexer/src/main/java/org/apache/blur/indexer/BlurIndexCounter.java
new file mode 100644
index 0000000..d143d12
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/indexer/BlurIndexCounter.java
@@ -0,0 +1,27 @@
+/**
+ * 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.blur.indexer;
+
+public enum BlurIndexCounter {
+
+  NEW_RECORDS, ROW_IDS_FROM_INDEX, ROW_IDS_TO_UPDATE_FROM_NEW_DATA, ROW_IDS_FROM_NEW_DATA,
+
+  INPUT_FORMAT_MAPPER, INPUT_FORMAT_EXISTING_RECORDS,
+
+  LOOKUP_MAPPER, LOOKUP_MAPPER_EXISTING_RECORDS, LOOKUP_MAPPER_ROW_LOOKUP_ATTEMPT
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/indexer/ClusterDriver.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/indexer/ClusterDriver.java b/blur-indexer/src/main/java/org/apache/blur/indexer/ClusterDriver.java
new file mode 100644
index 0000000..b53add2
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/indexer/ClusterDriver.java
@@ -0,0 +1,365 @@
+/**
+ * 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.blur.indexer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.blur.log.Log;
+import org.apache.blur.log.LogFactory;
+import org.apache.blur.mapreduce.lib.BlurInputFormat;
+import org.apache.blur.thirdparty.thrift_0_9_0.TException;
+import org.apache.blur.thrift.BlurClient;
+import org.apache.blur.thrift.generated.Blur.Iface;
+import org.apache.blur.thrift.generated.BlurException;
+import org.apache.blur.thrift.generated.TableDescriptor;
+import org.apache.blur.thrift.generated.TableStats;
+import org.apache.blur.utils.BlurConstants;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+public class ClusterDriver extends Configured implements Tool {
+
+  private static final Log LOG = LogFactory.getLog(ClusterDriver.class);
+  private static final String BLUR_ENV = "blur.env";
+  private static final String SEP = "_";
+  private static final String IMPORT = "import";
+
+  public static void main(String[] args) throws Exception {
+    System.exit(ToolRunner.run(new Configuration(), new ClusterDriver(), args));
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    int c = 0;
+    final String blurEnv = args[c++];
+    final String blurZkConnection = args[c++];
+    final String extraConfig = args[c++];
+    final int reducerMultiplier = 1;
+    final Configuration conf = getConf();
+
+    final ExecutorService service = Executors.newCachedThreadPool();
+    final AtomicBoolean running = new AtomicBoolean();
+    running.set(true);
+
+    // Load configs for all filesystems.
+    Path path = new Path(extraConfig);
+    Configuration mergeHdfsConfigs = HdfsConfigurationNamespaceMerge.mergeHdfsConfigs(path.getFileSystem(conf), path);
+    conf.addResource(mergeHdfsConfigs);
+    conf.set(BlurConstants.BLUR_ZOOKEEPER_CONNECTION, blurZkConnection);
+    conf.set(BLUR_ENV, blurEnv);
+
+    final Iface client = BlurClient.getClientFromZooKeeperConnectionStr(blurZkConnection);
+
+    stopAllExistingMRJobs(blurEnv, conf);
+    cleanUpOldImportDirs(client, conf);
+    moveInprogressDirsBackToNew(client, conf);
+    unlockLockedTables(client);
+
+    Map<String, Future<Void>> futures = new HashMap<String, Future<Void>>();
+    while (running.get()) {
+      LOG.debug("Starting index update check for blur cluster [" + blurZkConnection + "].");
+      try {
+        List<String> tableList = client.tableList();
+        startMissingIndexerThreads(tableList, service, futures, blurZkConnection, conf, client, reducerMultiplier);
+      } catch (TException t) {
+        LOG.error("Unknown Blur Thrift Error, Retrying...", t);
+      }
+      Thread.sleep(TimeUnit.SECONDS.toMillis(10));
+    }
+    return 0;
+  }
+
+  private void unlockLockedTables(Iface client) throws BlurException, TException {
+    List<String> tableList = client.tableList();
+    for (String table : tableList) {
+      TableDescriptor tableDescriptor = client.describe(table);
+      if (tableDescriptor.isEnabled()) {
+        unlockLockedTables(client, table);
+      }
+    }
+  }
+
+  private void unlockLockedTables(Iface client, String table) throws BlurException, TException {
+    Map<String, List<String>> listSnapshots = client.listSnapshots(table);
+    for (Entry<String, List<String>> e : listSnapshots.entrySet()) {
+      List<String> value = e.getValue();
+      if (value.contains(IndexerJobDriver.MRUPDATE_SNAPSHOT)) {
+        LOG.info("Unlocking table [{0}]", table);
+        client.removeSnapshot(table, IndexerJobDriver.MRUPDATE_SNAPSHOT);
+        return;
+      }
+    }
+  }
+
+  private void moveInprogressDirsBackToNew(Iface client, Configuration conf) throws BlurException, TException,
+      IOException {
+    List<String> tableList = client.tableList();
+    for (String table : tableList) {
+      String mrIncWorkingPathStr = getMRIncWorkingPathStr(client, table);
+      Path mrIncWorkingPath = new Path(mrIncWorkingPathStr);
+      Path newData = new Path(mrIncWorkingPath, IndexerJobDriver.NEW);
+      Path inprogressData = new Path(mrIncWorkingPath, IndexerJobDriver.INPROGRESS);
+      FileSystem fileSystem = inprogressData.getFileSystem(conf);
+      FileStatus[] listStatus = fileSystem.listStatus(inprogressData);
+      for (FileStatus fileStatus : listStatus) {
+        Path src = fileStatus.getPath();
+        Path dst = new Path(newData, src.getName());
+        if (fileSystem.rename(src, dst)) {
+          LOG.info("Moved [{0}] to [{1}] to be reprocessed.", src, dst);
+        } else {
+          LOG.error("Could not move [{0}] to [{1}] to be reprocessed.", src, dst);
+        }
+      }
+    }
+  }
+
+  private void cleanUpOldImportDirs(Iface client, Configuration conf) throws BlurException, TException, IOException {
+    List<String> tableList = client.tableList();
+    for (String table : tableList) {
+      cleanUpOldImportDirs(client, conf, table);
+    }
+  }
+
+  private void cleanUpOldImportDirs(Iface client, Configuration conf, String table) throws BlurException, TException,
+      IOException {
+    TableDescriptor descriptor = client.describe(table);
+    String tableUri = descriptor.getTableUri();
+    Path tablePath = new Path(tableUri);
+    FileSystem fileSystem = tablePath.getFileSystem(getConf());
+    Path importPath = new Path(tablePath, IMPORT);
+    if (fileSystem.exists(importPath)) {
+      for (FileStatus fileStatus : fileSystem.listStatus(importPath)) {
+        Path path = fileStatus.getPath();
+        LOG.info("Removing failed import [{0}]", path);
+        fileSystem.delete(path, true);
+      }
+    }
+  }
+
+  private void stopAllExistingMRJobs(String blurEnv, Configuration conf) throws YarnException, IOException,
+      InterruptedException {
+    Cluster cluster = new Cluster(conf);
+    JobStatus[] allJobStatuses = cluster.getAllJobStatuses();
+    for (JobStatus jobStatus : allJobStatuses) {
+      if (jobStatus.isJobComplete()) {
+        continue;
+      }
+      String jobFile = jobStatus.getJobFile();
+      JobID jobID = jobStatus.getJobID();
+      Job job = cluster.getJob(jobID);
+      FileSystem fileSystem = FileSystem.get(job.getConfiguration());
+      Configuration configuration = new Configuration(false);
+      Path path = new Path(jobFile);
+      Path makeQualified = path.makeQualified(fileSystem.getUri(), fileSystem.getWorkingDirectory());
+      if (hasReadAccess(fileSystem, makeQualified)) {
+        try (FSDataInputStream in = fileSystem.open(makeQualified)) {
+          configuration.addResource(copy(in));
+        }
+        String jobBlurEnv = configuration.get(BLUR_ENV);
+        LOG.info("Checking job [{0}] has env [{1}] current env set to [{2}]", jobID, jobBlurEnv, blurEnv);
+        if (blurEnv.equals(jobBlurEnv)) {
+          LOG.info("Killing running job [{0}]", jobID);
+          job.killJob();
+        }
+      }
+    }
+  }
+
+  private static InputStream copy(FSDataInputStream input) throws IOException {
+    try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) {
+      IOUtils.copy(input, outputStream);
+      return new ByteArrayInputStream(outputStream.toByteArray());
+    }
+  }
+
+  private static boolean hasReadAccess(FileSystem fileSystem, Path p) {
+    try {
+      fileSystem.access(p, FsAction.READ);
+      return true;
+    } catch (IOException e) {
+      return false;
+    }
+  }
+
+  private Callable<Void> getCallable(final String blurZkConnection, final Configuration conf, final Iface client,
+      final String table, final int reducerMultiplier) {
+    return new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        String originalThreadName = Thread.currentThread().getName();
+        try {
+          Thread.currentThread().setName(table);
+          if (!isEnabled(client, table)) {
+            LOG.info("Table [" + table + "] is not enabled.");
+            return null;
+          }
+          waitForDataToLoad(client, table);
+          LOG.debug("Starting index update for table [" + table + "].");
+          final String mrIncWorkingPathStr = getMRIncWorkingPathStr(client, table);
+          final String outputPathStr = getOutputPathStr(client, table);
+          Path path = new Path(outputPathStr);
+          FileSystem fileSystem = path.getFileSystem(getConf());
+
+          Configuration configuration = new Configuration(conf);
+          BlurInputFormat.setMaxNumberOfMaps(configuration, 10000);
+
+          IndexerJobDriver driver = new IndexerJobDriver();
+          driver.setConf(configuration);
+          try {
+            driver.run(new String[] { table, mrIncWorkingPathStr, outputPathStr, blurZkConnection,
+                Integer.toString(reducerMultiplier) });
+          } finally {
+            if (fileSystem.exists(path)) {
+              fileSystem.delete(path, true);
+            }
+          }
+          return null;
+        } finally {
+          Thread.currentThread().setName(originalThreadName);
+        }
+      }
+    };
+  }
+
+  private void startMissingIndexerThreads(List<String> tableList, ExecutorService service,
+      Map<String, Future<Void>> futures, final String blurZkConnection, final Configuration conf, final Iface client,
+      int reducerMultiplier) throws BlurException, TException {
+    Set<String> tables = new HashSet<String>(tableList);
+
+    // remove futures that are complete
+    for (String table : tables) {
+      Future<Void> future = futures.get(table);
+      if (future != null) {
+        if (future.isDone()) {
+          try {
+            future.get();
+          } catch (InterruptedException e) {
+            LOG.error("Unknown error while processing table [" + table + "].", e);
+          } catch (ExecutionException e) {
+            LOG.error("Unknown error while processing table [" + table + "].", e.getCause());
+          }
+          futures.remove(table);
+        } else {
+          LOG.info("Update for table [" + table + "] still running.");
+        }
+      }
+    }
+
+    // start missing tables
+    for (String table : tables) {
+      if (!futures.containsKey(table)) {
+        if (isEnabled(client, table)) {
+          Future<Void> future = service.submit(getCallable(blurZkConnection, conf, client, table, reducerMultiplier));
+          futures.put(table, future);
+        }
+      }
+    }
+  }
+
+  public static void waitForDataToLoad(Iface client, String table) throws BlurException, TException,
+      InterruptedException {
+    if (isFullyLoaded(client.tableStats(table))) {
+      return;
+    }
+    while (true) {
+      TableStats tableStats = client.tableStats(table);
+      if (isFullyLoaded(tableStats)) {
+        LOG.info("Data load complete in table [" + table + "] [" + tableStats + "]");
+        return;
+      }
+      LOG.info("Waiting for data to load in table [" + table + "] [" + tableStats + "]");
+      Thread.sleep(5000);
+    }
+  }
+
+  private static boolean isFullyLoaded(TableStats tableStats) {
+    if (tableStats.getSegmentImportInProgressCount() == 0 && tableStats.getSegmentImportPendingCount() == 0) {
+      return true;
+    }
+    return false;
+  }
+
+  private boolean isEnabled(Iface client, String table) throws BlurException, TException {
+    TableDescriptor tableDescriptor = client.describe(table);
+    return tableDescriptor.isEnabled();
+  }
+
+  private void mkdirs(FileSystem fileSystem, Path path) throws IOException {
+    if (fileSystem.exists(path)) {
+      return;
+    }
+    LOG.info("Creating path [" + path + "].");
+    if (!fileSystem.mkdirs(path)) {
+      LOG.error("Path [" + path + "] could not be created.");
+    }
+  }
+
+  public static String getMRIncWorkingPathStr(Iface client, String table) throws BlurException, TException, IOException {
+    TableDescriptor descriptor = client.describe(table);
+    Map<String, String> tableProperties = descriptor.getTableProperties();
+    if (tableProperties != null) {
+      String workingPath = tableProperties.get(BlurConstants.BLUR_BULK_UPDATE_WORKING_PATH);
+      if (workingPath != null) {
+        return workingPath;
+      }
+    }
+    throw new IOException("Table [" + table + "] does not have the property ["
+        + BlurConstants.BLUR_BULK_UPDATE_WORKING_PATH + "] setup correctly.");
+  }
+
+  private String getOutputPathStr(Iface client, String table) throws BlurException, TException, IOException {
+    TableDescriptor descriptor = client.describe(table);
+    String tableUri = descriptor.getTableUri();
+    Path tablePath = new Path(tableUri);
+    FileSystem fileSystem = tablePath.getFileSystem(getConf());
+    Path importPath = new Path(tablePath, IMPORT);
+    mkdirs(fileSystem, importPath);
+    return new Path(importPath, IMPORT + SEP + System.currentTimeMillis() + SEP + UUID.randomUUID().toString())
+        .toString();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/indexer/HdfsConfigurationNamespaceMerge.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/indexer/HdfsConfigurationNamespaceMerge.java b/blur-indexer/src/main/java/org/apache/blur/indexer/HdfsConfigurationNamespaceMerge.java
new file mode 100644
index 0000000..93ded33
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/indexer/HdfsConfigurationNamespaceMerge.java
@@ -0,0 +1,131 @@
+/**
+ * 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.blur.indexer;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class HdfsConfigurationNamespaceMerge {
+
+  private static final String DFS_NAMESERVICES = "dfs.nameservices";
+  private static final Log LOG = LogFactory.getLog(HdfsConfigurationNamespaceMerge.class);
+
+  public static void main(String[] args) throws IOException {
+    Path p = new Path("./src/main/scripts/conf/hdfs");
+
+    Configuration configuration = mergeHdfsConfigs(p.getFileSystem(new Configuration()), p);
+
+    // configuration.writeXml(System.out);
+
+    Collection<String> nameServices = configuration.getStringCollection(DFS_NAMESERVICES);
+    for (String name : nameServices) {
+      Path path = new Path("hdfs://" + name + "/");
+      FileSystem fileSystem = path.getFileSystem(configuration);
+      FileStatus[] listStatus = fileSystem.listStatus(path);
+      for (FileStatus fileStatus : listStatus) {
+        System.out.println(fileStatus.getPath());
+      }
+    }
+  }
+
+  private static boolean checkHostName(String host) {
+    try {
+      InetAddress.getAllByName(host);
+      return true;
+    } catch (UnknownHostException e) {
+      LOG.warn("Host not found [" + host + "]");
+      return false;
+    }
+  }
+
+  public static Configuration mergeHdfsConfigs(FileSystem fs, Path p) throws IOException {
+    List<Configuration> configList = new ArrayList<Configuration>();
+    gatherConfigs(fs, p, configList);
+    return merge(configList);
+  }
+
+  public static Configuration merge(List<Configuration> configList) throws IOException {
+    Configuration merge = new Configuration(false);
+    Set<String> nameServices = new HashSet<String>();
+    for (Configuration configuration : configList) {
+      String nameService = configuration.get(DFS_NAMESERVICES);
+      if (nameServices.contains(nameService)) {
+        throw new IOException("Multiple confs define namespace [" + nameService + "]");
+      }
+      nameServices.add(nameService);
+      if (shouldAdd(configuration, nameService)) {
+        for (Entry<String, String> e : configuration) {
+          String key = e.getKey();
+          if (key.contains(nameService)) {
+            String value = e.getValue();
+            merge.set(key, value);
+          }
+        }
+      }
+    }
+    merge.set(DFS_NAMESERVICES, StringUtils.join(nameServices, ","));
+    return merge;
+  }
+
+  private static boolean shouldAdd(Configuration configuration, String nameService) {
+    for (Entry<String, String> e : configuration) {
+      String key = e.getKey();
+      if (key.contains(nameService) && key.startsWith("dfs.namenode.rpc-address.")) {
+        return checkHostName(getHost(e.getValue()));
+      }
+    }
+    return false;
+  }
+
+  private static String getHost(String host) {
+    return host.substring(0, host.indexOf(":"));
+  }
+
+  public static void gatherConfigs(FileSystem fs, Path p, List<Configuration> configList) throws IOException {
+    if (fs.isFile(p)) {
+      if (p.getName().endsWith(".xml")) {
+        LOG.info("Loading file [" + p + "]");
+        Configuration configuration = new Configuration(false);
+        configuration.addResource(p);
+        configList.add(configuration);
+      } else {
+        LOG.info("Skipping file [" + p + "]");
+      }
+    } else {
+      FileStatus[] listStatus = fs.listStatus(p);
+      for (FileStatus fileStatus : listStatus) {
+        gatherConfigs(fs, fileStatus.getPath(), configList);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/indexer/IndexerJobDriver.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/indexer/IndexerJobDriver.java b/blur-indexer/src/main/java/org/apache/blur/indexer/IndexerJobDriver.java
new file mode 100644
index 0000000..9fea980
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/indexer/IndexerJobDriver.java
@@ -0,0 +1,498 @@
+/**
+ * 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.blur.indexer;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.blur.indexer.mapreduce.ExistingDataIndexLookupMapper;
+import org.apache.blur.indexer.mapreduce.ExistingDataMapper;
+import org.apache.blur.indexer.mapreduce.LookupBuilderMapper;
+import org.apache.blur.indexer.mapreduce.LookupBuilderReducer;
+import org.apache.blur.indexer.mapreduce.NewDataMapper;
+import org.apache.blur.indexer.mapreduce.PrunedBlurInputFormat;
+import org.apache.blur.indexer.mapreduce.PrunedSequenceFileInputFormat;
+import org.apache.blur.log.Log;
+import org.apache.blur.log.LogFactory;
+import org.apache.blur.mapreduce.lib.BlurInputFormat;
+import org.apache.blur.mapreduce.lib.BlurOutputFormat;
+import org.apache.blur.mapreduce.lib.update.IndexKey;
+import org.apache.blur.mapreduce.lib.update.IndexKeyPartitioner;
+import org.apache.blur.mapreduce.lib.update.IndexKeyWritableComparator;
+import org.apache.blur.mapreduce.lib.update.IndexValue;
+import org.apache.blur.mapreduce.lib.update.UpdateReducer;
+import org.apache.blur.thirdparty.thrift_0_9_0.TException;
+import org.apache.blur.thrift.BlurClient;
+import org.apache.blur.thrift.generated.Blur.Iface;
+import org.apache.blur.thrift.generated.BlurException;
+import org.apache.blur.thrift.generated.TableDescriptor;
+import org.apache.blur.thrift.generated.TableStats;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.TaskReport;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+public class IndexerJobDriver extends Configured implements Tool {
+
+  public static final String BLUR_UPDATE_ID = "blur.update.id";
+  private static final String BLUR_EXEC_TYPE = "blur.exec.type";
+  public static final String TMP = "tmp";
+
+  public enum EXEC {
+    MR_ONLY, MR_WITH_LOOKUP, AUTOMATIC
+  }
+
+  public static final String MRUPDATE_SNAPSHOT = "mrupdate-snapshot";
+  public static final String CACHE = "cache";
+  public static final String COMPLETE = "complete";
+  public static final String INPROGRESS = "inprogress";
+  public static final String NEW = "new";
+  private static final Log LOG = LogFactory.getLog(IndexerJobDriver.class);
+
+  public static void main(String[] args) throws Exception {
+    int res = ToolRunner.run(new Configuration(), new IndexerJobDriver(), args);
+    System.exit(res);
+  }
+
+  static class PartitionedInputResult {
+    final Path _partitionedInputData;
+    final Counters _counters;
+    final long[] _rowIdsFromNewData;
+    final long[] _rowIdsToUpdateFromNewData;
+    final long[] _rowIdsFromIndex;
+
+    PartitionedInputResult(Path partitionedInputData, Counters counters, int shards, TaskReport[] taskReports) {
+      _partitionedInputData = partitionedInputData;
+      _counters = counters;
+      _rowIdsFromNewData = new long[shards];
+      _rowIdsToUpdateFromNewData = new long[shards];
+      _rowIdsFromIndex = new long[shards];
+      for (TaskReport tr : taskReports) {
+        int id = tr.getTaskID().getId();
+        Counters taskCounters = tr.getTaskCounters();
+        Counter total = taskCounters.findCounter(BlurIndexCounter.ROW_IDS_FROM_NEW_DATA);
+        _rowIdsFromNewData[id] = total.getValue();
+        Counter update = taskCounters.findCounter(BlurIndexCounter.ROW_IDS_TO_UPDATE_FROM_NEW_DATA);
+        _rowIdsToUpdateFromNewData[id] = update.getValue();
+        Counter index = taskCounters.findCounter(BlurIndexCounter.ROW_IDS_FROM_INDEX);
+        _rowIdsFromIndex[id] = index.getValue();
+      }
+    }
+
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    int c = 0;
+    if (args.length < 5) {
+      System.err
+          .println("Usage Driver <table> <mr inc working path> <output path> <zk connection> <reducer multipler> <extra config files...>");
+      return 1;
+    }
+    String table = args[c++];
+    String mrIncWorkingPathStr = args[c++];
+    String outputPathStr = args[c++];
+    String blurZkConnection = args[c++];
+    int reducerMultipler = Integer.parseInt(args[c++]);
+    for (; c < args.length; c++) {
+      String externalConfigFileToAdd = args[c];
+      getConf().addResource(new Path(externalConfigFileToAdd));
+    }
+
+    Path outputPath = new Path(outputPathStr);
+    Path mrIncWorkingPath = new Path(mrIncWorkingPathStr);
+    FileSystem fileSystem = mrIncWorkingPath.getFileSystem(getConf());
+
+    Path newData = new Path(mrIncWorkingPath, NEW);
+    Path inprogressData = new Path(mrIncWorkingPath, INPROGRESS);
+    Path completeData = new Path(mrIncWorkingPath, COMPLETE);
+    Path fileCache = new Path(mrIncWorkingPath, CACHE);
+    Path tmpPathDontDelete = new Path(mrIncWorkingPath, TMP);
+
+    Path tmpPath = new Path(tmpPathDontDelete, UUID.randomUUID().toString());
+
+    fileSystem.mkdirs(newData);
+    fileSystem.mkdirs(inprogressData);
+    fileSystem.mkdirs(completeData);
+    fileSystem.mkdirs(fileCache);
+
+    List<Path> srcPathList = new ArrayList<Path>();
+    for (FileStatus fileStatus : fileSystem.listStatus(newData)) {
+      srcPathList.add(fileStatus.getPath());
+    }
+    if (srcPathList.isEmpty()) {
+      return 0;
+    }
+
+    List<Path> inprogressPathList = new ArrayList<Path>();
+    boolean success = false;
+    Iface client = null;
+
+    EXEC exec = EXEC.valueOf(getConf().get(BLUR_EXEC_TYPE, EXEC.AUTOMATIC.name()).toUpperCase());
+
+    String uuid = UUID.randomUUID().toString();
+
+    try {
+      client = BlurClient.getClientFromZooKeeperConnectionStr(blurZkConnection);
+      TableDescriptor descriptor = client.describe(table);
+      Map<String, String> tableProperties = descriptor.getTableProperties();
+      String fastDir = tableProperties.get("blur.table.disable.fast.dir");
+      if (fastDir == null || !fastDir.equals("true")) {
+        LOG.error("Table [{0}] has blur.table.disable.fast.dir enabled, not supported in fast MR update.", table);
+        return 1;
+      }
+
+      waitForOtherSnapshotsToBeRemoved(client, table, MRUPDATE_SNAPSHOT);
+      client.createSnapshot(table, MRUPDATE_SNAPSHOT);
+      TableStats tableStats = client.tableStats(table);
+
+      inprogressPathList = movePathList(fileSystem, inprogressData, srcPathList);
+
+      switch (exec) {
+      case MR_ONLY:
+        success = runMrOnly(descriptor, inprogressPathList, table, fileCache, outputPath, reducerMultipler);
+        break;
+      case MR_WITH_LOOKUP:
+        success = runMrWithLookup(uuid, descriptor, inprogressPathList, table, fileCache, outputPath, reducerMultipler,
+            tmpPath, tableStats, MRUPDATE_SNAPSHOT);
+        break;
+      case AUTOMATIC:
+        success = runAutomatic(uuid, descriptor, inprogressPathList, table, fileCache, outputPath, reducerMultipler,
+            tmpPath, tableStats, MRUPDATE_SNAPSHOT);
+        break;
+      default:
+        throw new RuntimeException("Exec type [" + exec + "] not supported.");
+      }
+    } finally {
+      if (success) {
+        LOG.info("Associate lookup cache with new data!");
+        associateLookupCache(uuid, fileCache, outputPath);
+        LOG.info("Indexing job succeeded!");
+        client.loadData(table, outputPathStr);
+        LOG.info("Load data called");
+        movePathList(fileSystem, completeData, inprogressPathList);
+        LOG.info("Input data moved to complete");
+        ClusterDriver.waitForDataToLoad(client, table);
+        LOG.info("Data loaded");
+      } else {
+        LOG.error("Indexing job failed!");
+        movePathList(fileSystem, newData, inprogressPathList);
+      }
+      fileSystem.delete(tmpPath, true);
+      if (client != null) {
+        client.removeSnapshot(table, MRUPDATE_SNAPSHOT);
+      }
+    }
+
+    if (success) {
+      return 0;
+    } else {
+      return 1;
+    }
+  }
+
+  private void associateLookupCache(String uuid, Path fileCache, Path outputPath) throws IOException {
+    FileSystem fileSystem = fileCache.getFileSystem(getConf());
+    cleanupExtraFileFromSpecX(fileSystem, uuid, fileCache);
+    associateLookupCache(fileSystem, uuid, fileSystem.getFileStatus(fileCache), outputPath);
+  }
+
+  private void cleanupExtraFileFromSpecX(FileSystem fileSystem, String uuid, Path fileCache) throws IOException {
+    FileStatus[] listStatus = fileSystem.listStatus(fileCache);
+    List<FileStatus> uuidPaths = new ArrayList<FileStatus>();
+    for (FileStatus fs : listStatus) {
+      Path path = fs.getPath();
+      if (fs.isDirectory()) {
+        cleanupExtraFileFromSpecX(fileSystem, uuid, path);
+      } else if (path.getName().startsWith(uuid)) {
+        uuidPaths.add(fs);
+      }
+    }
+    if (uuidPaths.size() > 1) {
+      deleteIncomplete(fileSystem, uuidPaths);
+    }
+  }
+
+  private void deleteIncomplete(FileSystem fileSystem, List<FileStatus> uuidPaths) throws IOException {
+    long max = 0;
+    FileStatus keeper = null;
+    for (FileStatus fs : uuidPaths) {
+      long len = fs.getLen();
+      if (len > max) {
+        keeper = fs;
+        max = len;
+      }
+    }
+    for (FileStatus fs : uuidPaths) {
+      if (fs != keeper) {
+        LOG.info("Deleteing incomplete cache file [{0}]", fs.getPath());
+        fileSystem.delete(fs.getPath(), false);
+      }
+    }
+  }
+
+  private void associateLookupCache(FileSystem fileSystem, String uuid, FileStatus fileCache, Path outputPath)
+      throws IOException {
+    Path path = fileCache.getPath();
+    if (fileCache.isDirectory()) {
+      FileStatus[] listStatus = fileSystem.listStatus(path);
+      for (FileStatus fs : listStatus) {
+        associateLookupCache(fileSystem, uuid, fs, outputPath);
+      }
+    } else if (path.getName().startsWith(uuid)) {
+      Path parent = path.getParent();
+      String shardName = parent.getName();
+      Path indexPath = findOutputDirPath(outputPath, shardName);
+      LOG.info("Path found for shard [{0}] outputPath [{1}]", shardName, outputPath);
+      String id = MergeSortRowIdMatcher.getIdForSingleSegmentIndex(getConf(), indexPath);
+      Path file = new Path(path.getParent(), id + ".seq");
+      MergeSortRowIdMatcher.commitWriter(getConf(), file, path);
+    }
+  }
+
+  private Path findOutputDirPath(Path outputPath, String shardName) throws IOException {
+    FileSystem fileSystem = outputPath.getFileSystem(getConf());
+    Path shardPath = new Path(outputPath, shardName);
+    if (!fileSystem.exists(shardPath)) {
+      throw new IOException("Shard path [" + shardPath + "]");
+    }
+    FileStatus[] listStatus = fileSystem.listStatus(shardPath, new PathFilter() {
+      @Override
+      public boolean accept(Path path) {
+        return path.getName().endsWith(".commit");          
+      }
+    });
+    if (listStatus.length == 1) {
+      FileStatus fs = listStatus[0];
+      return fs.getPath();
+    } else {
+      throw new IOException("More than one sub dir [" + shardPath + "]");
+    }
+  }
+
+  private boolean runAutomatic(String uuid, TableDescriptor descriptor, List<Path> inprogressPathList, String table,
+      Path fileCache, Path outputPath, int reducerMultipler, Path tmpPath, TableStats tableStats, String snapshot)
+      throws ClassNotFoundException, IOException, InterruptedException {
+    PartitionedInputResult result = buildPartitionedInputData(uuid, tmpPath, descriptor, inprogressPathList, snapshot,
+        fileCache);
+
+    Job job = Job.getInstance(getConf(), "Blur Row Updater for table [" + table + "]");
+
+    InputSplitPruneUtil.setBlurLookupRowIdFromNewDataCounts(job, table, result._rowIdsFromNewData);
+    InputSplitPruneUtil.setBlurLookupRowIdUpdateFromNewDataCounts(job, table, result._rowIdsToUpdateFromNewData);
+    InputSplitPruneUtil.setBlurLookupRowIdFromIndexCounts(job, table, result._rowIdsFromIndex);
+    InputSplitPruneUtil.setTable(job, table);
+
+    BlurInputFormat.setLocalCachePath(job, fileCache);
+
+    // Existing data - This adds the copy data files first open and stream
+    // through all documents.
+    {
+      Path tablePath = new Path(descriptor.getTableUri());
+      BlurInputFormat.addTable(job, descriptor, MRUPDATE_SNAPSHOT);
+      MultipleInputs.addInputPath(job, tablePath, PrunedBlurInputFormat.class, ExistingDataMapper.class);
+    }
+
+    // Existing data - This adds the row id lookup
+    {
+      ExistingDataIndexLookupMapper.setSnapshot(job, MRUPDATE_SNAPSHOT);
+      FileInputFormat.addInputPath(job, result._partitionedInputData);
+      MultipleInputs.addInputPath(job, result._partitionedInputData, PrunedSequenceFileInputFormat.class,
+          ExistingDataIndexLookupMapper.class);
+    }
+
+    // New Data
+    for (Path p : inprogressPathList) {
+      FileInputFormat.addInputPath(job, p);
+      MultipleInputs.addInputPath(job, p, SequenceFileInputFormat.class, NewDataMapper.class);
+    }
+
+    BlurOutputFormat.setOutputPath(job, outputPath);
+    BlurOutputFormat.setupJob(job, descriptor);
+
+    job.setReducerClass(UpdateReducer.class);
+    job.setMapOutputKeyClass(IndexKey.class);
+    job.setMapOutputValueClass(IndexValue.class);
+    job.setPartitionerClass(IndexKeyPartitioner.class);
+    job.setGroupingComparatorClass(IndexKeyWritableComparator.class);
+
+    BlurOutputFormat.setReducerMultiplier(job, reducerMultipler);
+
+    boolean success = job.waitForCompletion(true);
+    Counters counters = job.getCounters();
+    LOG.info("Counters [" + counters + "]");
+    return success;
+  }
+
+  private boolean runMrWithLookup(String uuid, TableDescriptor descriptor, List<Path> inprogressPathList, String table,
+      Path fileCache, Path outputPath, int reducerMultipler, Path tmpPath, TableStats tableStats, String snapshot)
+      throws ClassNotFoundException, IOException, InterruptedException {
+    PartitionedInputResult result = buildPartitionedInputData(uuid, tmpPath, descriptor, inprogressPathList, snapshot,
+        fileCache);
+
+    Job job = Job.getInstance(getConf(), "Blur Row Updater for table [" + table + "]");
+
+    ExistingDataIndexLookupMapper.setSnapshot(job, MRUPDATE_SNAPSHOT);
+    FileInputFormat.addInputPath(job, result._partitionedInputData);
+    MultipleInputs.addInputPath(job, result._partitionedInputData, SequenceFileInputFormat.class,
+        ExistingDataIndexLookupMapper.class);
+
+    for (Path p : inprogressPathList) {
+      FileInputFormat.addInputPath(job, p);
+      MultipleInputs.addInputPath(job, p, SequenceFileInputFormat.class, NewDataMapper.class);
+    }
+
+    BlurOutputFormat.setOutputPath(job, outputPath);
+    BlurOutputFormat.setupJob(job, descriptor);
+
+    job.setReducerClass(UpdateReducer.class);
+    job.setMapOutputKeyClass(IndexKey.class);
+    job.setMapOutputValueClass(IndexValue.class);
+    job.setPartitionerClass(IndexKeyPartitioner.class);
+    job.setGroupingComparatorClass(IndexKeyWritableComparator.class);
+
+    BlurOutputFormat.setReducerMultiplier(job, reducerMultipler);
+
+    boolean success = job.waitForCompletion(true);
+    Counters counters = job.getCounters();
+    LOG.info("Counters [" + counters + "]");
+    return success;
+  }
+
+  private boolean runMrOnly(TableDescriptor descriptor, List<Path> inprogressPathList, String table, Path fileCache,
+      Path outputPath, int reducerMultipler) throws IOException, ClassNotFoundException, InterruptedException {
+    Job job = Job.getInstance(getConf(), "Blur Row Updater for table [" + table + "]");
+    Path tablePath = new Path(descriptor.getTableUri());
+    BlurInputFormat.setLocalCachePath(job, fileCache);
+    BlurInputFormat.addTable(job, descriptor, MRUPDATE_SNAPSHOT);
+    MultipleInputs.addInputPath(job, tablePath, BlurInputFormat.class, ExistingDataMapper.class);
+
+    for (Path p : inprogressPathList) {
+      FileInputFormat.addInputPath(job, p);
+      MultipleInputs.addInputPath(job, p, SequenceFileInputFormat.class, NewDataMapper.class);
+    }
+
+    BlurOutputFormat.setOutputPath(job, outputPath);
+    BlurOutputFormat.setupJob(job, descriptor);
+
+    job.setReducerClass(UpdateReducer.class);
+    job.setMapOutputKeyClass(IndexKey.class);
+    job.setMapOutputValueClass(IndexValue.class);
+    job.setPartitionerClass(IndexKeyPartitioner.class);
+    job.setGroupingComparatorClass(IndexKeyWritableComparator.class);
+
+    BlurOutputFormat.setReducerMultiplier(job, reducerMultipler);
+
+    boolean success = job.waitForCompletion(true);
+    Counters counters = job.getCounters();
+    LOG.info("Counters [" + counters + "]");
+    return success;
+  }
+
+  private PartitionedInputResult buildPartitionedInputData(String uuid, Path tmpPath, TableDescriptor descriptor,
+      List<Path> inprogressPathList, String snapshot, Path fileCachePath) throws IOException, ClassNotFoundException,
+      InterruptedException {
+    Job job = Job.getInstance(getConf(), "Partitioning data for table [" + descriptor.getName() + "]");
+    job.getConfiguration().set(BLUR_UPDATE_ID, uuid);
+
+    // Needed for the bloom filter path information.
+    BlurOutputFormat.setTableDescriptor(job, descriptor);
+    BlurInputFormat.setLocalCachePath(job, fileCachePath);
+    ExistingDataIndexLookupMapper.setSnapshot(job, snapshot);
+
+    for (Path p : inprogressPathList) {
+      FileInputFormat.addInputPath(job, p);
+    }
+    Path outputPath = new Path(tmpPath, UUID.randomUUID().toString());
+    job.setJarByClass(getClass());
+    job.setMapperClass(LookupBuilderMapper.class);
+    job.setReducerClass(LookupBuilderReducer.class);
+
+    int shardCount = descriptor.getShardCount();
+    job.setNumReduceTasks(shardCount);
+    job.setInputFormatClass(SequenceFileInputFormat.class);
+    job.setMapOutputKeyClass(Text.class);
+    job.setMapOutputValueClass(NullWritable.class);
+    job.setOutputFormatClass(SequenceFileOutputFormat.class);
+    job.setOutputKeyClass(Text.class);
+    job.setOutputValueClass(BooleanWritable.class);
+    FileOutputFormat.setOutputPath(job, outputPath);
+    if (job.waitForCompletion(true)) {
+      return new PartitionedInputResult(outputPath, job.getCounters(), shardCount, job.getTaskReports(TaskType.REDUCE));
+    } else {
+      throw new IOException("Partitioning failed!");
+    }
+  }
+
+  private void waitForOtherSnapshotsToBeRemoved(Iface client, String table, String snapshot) throws BlurException,
+      TException, InterruptedException {
+    while (true) {
+      Map<String, List<String>> listSnapshots = client.listSnapshots(table);
+      boolean mrupdateSnapshots = false;
+      for (Entry<String, List<String>> e : listSnapshots.entrySet()) {
+        List<String> value = e.getValue();
+        if (value.contains(snapshot)) {
+          mrupdateSnapshots = true;
+        }
+      }
+      if (!mrupdateSnapshots) {
+        return;
+      } else {
+        LOG.info(snapshot + " Snapshot for table [{0}] already exists", table);
+        Thread.sleep(TimeUnit.SECONDS.toMillis(10));
+        LOG.info("Retrying");
+      }
+    }
+  }
+
+  private List<Path> movePathList(FileSystem fileSystem, Path dstDir, List<Path> lst) throws IOException {
+    List<Path> result = new ArrayList<Path>();
+    for (Path src : lst) {
+      Path dst = new Path(dstDir, src.getName());
+      if (fileSystem.rename(src, dst)) {
+        LOG.info("Moving [{0}] to [{1}]", src, dst);
+        result.add(dst);
+      } else {
+        LOG.error("Could not move [{0}] to [{1}]", src, dst);
+      }
+    }
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/indexer/InputSplitPruneUtil.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/indexer/InputSplitPruneUtil.java b/blur-indexer/src/main/java/org/apache/blur/indexer/InputSplitPruneUtil.java
new file mode 100644
index 0000000..043b071
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/indexer/InputSplitPruneUtil.java
@@ -0,0 +1,149 @@
+/**
+ * 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.blur.indexer;
+
+import org.apache.blur.utils.ShardUtil;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Job;
+
+public class InputSplitPruneUtil {
+
+  private static final String BLUR_LOOKUP_ROWID_UPDATE_FROM_NEW_DATA_COUNT_PREFIX = "blur.lookup.rowid.update.from.new.data.count";
+  private static final String BLUR_LOOKUP_ROWID_FROM_NEW_DATA_COUNT_PREFIX = "blur.lookup.rowid.from.new.data.count.";
+  private static final String BLUR_LOOKUP_ROWID_FROM_INDEX_COUNT_PREFIX = "blur.lookup.rowid.from.index.count.";
+
+  private static final String BLUR_LOOKUP_TABLE = "blur.lookup.table";
+  private static final String BLUR_LOOKUP_RATIO_PER_SHARD = "blur.lookup.ratio.per.shard";
+  private static final String BLUR_LOOKUP_MAX_TOTAL_PER_SHARD = "blur.lookup.max.total.per.shard";
+
+  private static final double DEFAULT_LOOKUP_RATIO = 0.5;
+  private static final long DEFAULT_LOOKUP_MAX_TOTAL = Long.MAX_VALUE;
+
+  public static boolean shouldLookupExecuteOnShard(Configuration configuration, String table, int shard) {
+    double lookupRatio = getLookupRatio(configuration);
+    long maxLookupCount = getMaxLookupCount(configuration);
+    long rowIdFromNewDataCount = getBlurLookupRowIdFromNewDataCount(configuration, table, shard);
+    long rowIdUpdateFromNewDataCount = getBlurLookupRowIdUpdateFromNewDataCount(configuration, table, shard);
+    long rowIdFromIndexCount = getBlurLookupRowIdFromIndexCount(configuration, table, shard);
+    return shouldLookupRun(rowIdFromIndexCount, rowIdFromNewDataCount, rowIdUpdateFromNewDataCount, lookupRatio,
+        maxLookupCount);
+  }
+
+  private static boolean shouldLookupRun(long rowIdFromIndexCount, long rowIdFromNewDataCount,
+      long rowIdUpdateFromNewDataCount, double lookupRatio, long maxLookupCount) {
+    if (rowIdUpdateFromNewDataCount > maxLookupCount) {
+      return false;
+    }
+    double d = (double) rowIdUpdateFromNewDataCount / (double) rowIdFromIndexCount;
+    if (d <= lookupRatio) {
+      return true;
+    }
+    return false;
+  }
+
+  public static double getLookupRatio(Configuration configuration) {
+    return configuration.getDouble(BLUR_LOOKUP_RATIO_PER_SHARD, DEFAULT_LOOKUP_RATIO);
+  }
+
+  private static long getMaxLookupCount(Configuration configuration) {
+    return configuration.getLong(BLUR_LOOKUP_MAX_TOTAL_PER_SHARD, DEFAULT_LOOKUP_MAX_TOTAL);
+  }
+
+  public static void setTable(Job job, String table) {
+    setTable(job.getConfiguration(), table);
+  }
+
+  public static void setTable(Configuration configuration, String table) {
+    configuration.set(BLUR_LOOKUP_TABLE, table);
+  }
+
+  public static String getTable(Configuration configuration) {
+    return configuration.get(BLUR_LOOKUP_TABLE);
+  }
+
+  public static String getBlurLookupRowIdFromIndexCountName(String table) {
+    return BLUR_LOOKUP_ROWID_FROM_INDEX_COUNT_PREFIX + table;
+  }
+
+  public static String getBlurLookupRowIdFromNewDataCountName(String table) {
+    return BLUR_LOOKUP_ROWID_FROM_NEW_DATA_COUNT_PREFIX + table;
+  }
+
+  public static String getBlurLookupRowIdUpdateFromNewDataCountName(String table) {
+    return BLUR_LOOKUP_ROWID_UPDATE_FROM_NEW_DATA_COUNT_PREFIX + table;
+  }
+
+  public static long getBlurLookupRowIdUpdateFromNewDataCount(Configuration configuration, String table, int shard) {
+    String[] strings = configuration.getStrings(getBlurLookupRowIdUpdateFromNewDataCountName(table));
+    return getCount(strings, shard);
+  }
+
+  public static long getBlurLookupRowIdFromNewDataCount(Configuration configuration, String table, int shard) {
+    String[] strings = configuration.getStrings(getBlurLookupRowIdFromNewDataCountName(table));
+    return getCount(strings, shard);
+  }
+
+  public static long getBlurLookupRowIdFromIndexCount(Configuration configuration, String table, int shard) {
+    String[] strings = configuration.getStrings(getBlurLookupRowIdFromIndexCountName(table));
+    return getCount(strings, shard);
+  }
+
+  public static void setBlurLookupRowIdFromNewDataCounts(Job job, String table, long[] counts) {
+    setBlurLookupRowIdFromNewDataCounts(job.getConfiguration(), table, counts);
+  }
+
+  public static void setBlurLookupRowIdFromNewDataCounts(Configuration configuration, String table, long[] counts) {
+    configuration.setStrings(getBlurLookupRowIdFromNewDataCountName(table), toStrings(counts));
+  }
+
+  public static void setBlurLookupRowIdUpdateFromNewDataCounts(Job job, String table, long[] counts) {
+    setBlurLookupRowIdUpdateFromNewDataCounts(job.getConfiguration(), table, counts);
+  }
+
+  public static void setBlurLookupRowIdUpdateFromNewDataCounts(Configuration configuration, String table, long[] counts) {
+    configuration.setStrings(getBlurLookupRowIdUpdateFromNewDataCountName(table), toStrings(counts));
+  }
+
+  public static void setBlurLookupRowIdFromIndexCounts(Job job, String table, long[] counts) {
+    setBlurLookupRowIdFromIndexCounts(job.getConfiguration(), table, counts);
+  }
+
+  public static void setBlurLookupRowIdFromIndexCounts(Configuration configuration, String table, long[] counts) {
+    configuration.setStrings(getBlurLookupRowIdFromIndexCountName(table), toStrings(counts));
+  }
+
+  public static long getCount(String[] strings, int shard) {
+    return Long.parseLong(strings[shard]);
+  }
+
+  public static int getShardFromDirectoryPath(Path path) {
+    return ShardUtil.getShardIndex(path.getName());
+  }
+
+  public static String[] toStrings(long[] counts) {
+    if (counts == null) {
+      return null;
+    }
+    String[] strs = new String[counts.length];
+    for (int i = 0; i < counts.length; i++) {
+      strs[i] = Long.toString(counts[i]);
+    }
+    return strs;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/indexer/MergeSortRowIdMatcher.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/indexer/MergeSortRowIdMatcher.java b/blur-indexer/src/main/java/org/apache/blur/indexer/MergeSortRowIdMatcher.java
new file mode 100644
index 0000000..66cd2ac
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/indexer/MergeSortRowIdMatcher.java
@@ -0,0 +1,386 @@
+/**
+ * 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.blur.indexer;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.blur.index.AtomicReaderUtil;
+import org.apache.blur.log.Log;
+import org.apache.blur.log.LogFactory;
+import org.apache.blur.store.hdfs.DirectoryDecorator;
+import org.apache.blur.store.hdfs.HdfsDirectory;
+import org.apache.blur.utils.BlurConstants;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.HdfsBlockLocation;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DeflateCodec;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.io.compress.zlib.ZlibFactory;
+import org.apache.hadoop.util.Progressable;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexCommit;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentInfoPerCommit;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.SegmentReader;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+
+public class MergeSortRowIdMatcher {
+
+  private static final String DEL = ".del";
+  private static final Log LOG = LogFactory.getLog(MergeSortRowIdMatcher.class);
+  private static final Progressable NO_OP = new Progressable() {
+    @Override
+    public void progress() {
+
+    }
+  };
+  private static final long _10_SECONDS = TimeUnit.SECONDS.toNanos(10);
+
+  public interface Action {
+    void found(Text rowId) throws IOException;
+  }
+
+  private final MyReader[] _readers;
+  private final Configuration _configuration;
+  private final Path _cachePath;
+  private final IndexCommit _indexCommit;
+  private final Directory _directory;
+  private final Progressable _progressable;
+
+  private DirectoryReader _reader;
+
+  public MergeSortRowIdMatcher(Directory directory, long generation, Configuration configuration, Path cachePath)
+      throws IOException {
+    this(directory, generation, configuration, cachePath, null);
+  }
+
+  public MergeSortRowIdMatcher(Directory directory, long generation, Configuration configuration, Path cachePath,
+      Progressable progressable) throws IOException {
+    List<IndexCommit> listCommits = DirectoryReader.listCommits(directory);
+    _indexCommit = findIndexCommit(listCommits, generation);
+    _configuration = configuration;
+    _cachePath = cachePath;
+    _directory = directory;
+    _progressable = progressable == null ? NO_OP : progressable;
+    _readers = openReaders();
+  }
+
+  public void lookup(Text rowId, Action action) throws IOException {
+    if (lookup(rowId)) {
+      action.found(rowId);
+    }
+  }
+
+  private boolean lookup(Text rowId) throws IOException {
+    advanceReadersIfNeeded(rowId);
+    sortReaders();
+    return checkReaders(rowId);
+  }
+
+  private boolean checkReaders(Text rowId) {
+    for (MyReader reader : _readers) {
+      int compareTo = reader.getCurrentRowId().compareTo(rowId);
+      if (compareTo == 0) {
+        return true;
+      } else if (compareTo > 0) {
+        return false;
+      }
+    }
+    return false;
+  }
+
+  private void advanceReadersIfNeeded(Text rowId) throws IOException {
+    _progressable.progress();
+    for (MyReader reader : _readers) {
+      if (rowId.compareTo(reader.getCurrentRowId()) > 0) {
+        advanceReader(reader, rowId);
+      }
+    }
+  }
+
+  private void advanceReader(MyReader reader, Text rowId) throws IOException {
+    while (reader.next()) {
+      if (rowId.compareTo(reader.getCurrentRowId()) <= 0) {
+        return;
+      }
+    }
+  }
+
+  private static final Comparator<MyReader> COMP = new Comparator<MyReader>() {
+    @Override
+    public int compare(MyReader o1, MyReader o2) {
+      return o1.getCurrentRowId().compareTo(o2.getCurrentRowId());
+    }
+  };
+
+  private void sortReaders() {
+    Arrays.sort(_readers, COMP);
+  }
+
+  private MyReader[] openReaders() throws IOException {
+    Collection<SegmentKey> segmentKeys = getSegmentKeys();
+    MyReader[] readers = new MyReader[segmentKeys.size()];
+    int i = 0;
+    for (SegmentKey segmentKey : segmentKeys) {
+      readers[i++] = openReader(segmentKey);
+    }
+    return readers;
+  }
+
+  private MyReader openReader(SegmentKey segmentKey) throws IOException {
+    Path file = getCacheFilePath(segmentKey);
+    FileSystem fileSystem = _cachePath.getFileSystem(_configuration);
+    if (!fileSystem.exists(file)) {
+      createCacheFile(file, segmentKey);
+    }
+    Reader reader = new SequenceFile.Reader(_configuration, SequenceFile.Reader.file(file));
+    return new MyReader(reader);
+  }
+
+  private void createCacheFile(Path file, SegmentKey segmentKey) throws IOException {
+    LOG.info("Building cache for segment [{0}] to [{1}]", segmentKey, file);
+    Path tmpPath = getTmpWriterPath(file.getParent());
+    try (Writer writer = createWriter(_configuration, tmpPath)) {
+      DirectoryReader reader = getReader();
+      for (AtomicReaderContext context : reader.leaves()) {
+        SegmentReader segmentReader = AtomicReaderUtil.getSegmentReader(context.reader());
+        if (segmentReader.getSegmentName().equals(segmentKey.getSegmentName())) {
+          writeRowIds(writer, segmentReader);
+          break;
+        }
+      }
+    }
+    commitWriter(_configuration, file, tmpPath);
+  }
+
+  public static void commitWriter(Configuration configuration, Path file, Path tmpPath) throws IOException {
+    FileSystem fileSystem = tmpPath.getFileSystem(configuration);
+    LOG.info("Commit tmp [{0}] to file [{1}]", tmpPath, file);
+    if (!fileSystem.rename(tmpPath, file)) {
+      LOG.warn("Could not commit tmp file [{0}] to file [{1}]", tmpPath, file);
+    }
+  }
+
+  public static Path getTmpWriterPath(Path dir) {
+    return new Path(dir, UUID.randomUUID().toString() + ".tmp");
+  }
+
+  public static Writer createWriter(Configuration configuration, Path tmpPath) throws IOException {
+    return SequenceFile.createWriter(configuration, SequenceFile.Writer.file(tmpPath),
+        SequenceFile.Writer.keyClass(Text.class), SequenceFile.Writer.valueClass(NullWritable.class),
+        SequenceFile.Writer.compression(CompressionType.BLOCK, getCodec(configuration)));
+  }
+
+  private static CompressionCodec getCodec(Configuration configuration) {
+    if (ZlibFactory.isNativeZlibLoaded(configuration)) {
+      return new GzipCodec();
+    }
+    return new DeflateCodec();
+  }
+
+  private void writeRowIds(Writer writer, SegmentReader segmentReader) throws IOException {
+    Terms terms = segmentReader.terms(BlurConstants.ROW_ID);
+    if (terms == null) {
+      return;
+    }
+    TermsEnum termsEnum = terms.iterator(null);
+    BytesRef rowId;
+    long s = System.nanoTime();
+    while ((rowId = termsEnum.next()) != null) {
+      long n = System.nanoTime();
+      if (n + _10_SECONDS > s) {
+        _progressable.progress();
+        s = System.nanoTime();
+      }
+      writer.append(new Text(rowId.utf8ToString()), NullWritable.get());
+    }
+  }
+
+  private IndexCommit findIndexCommit(List<IndexCommit> listCommits, long generation) throws IOException {
+    for (IndexCommit commit : listCommits) {
+      if (commit.getGeneration() == generation) {
+        return commit;
+      }
+    }
+    throw new IOException("Generation [" + generation + "] not found.");
+  }
+
+  static class SegmentKey {
+
+    final String _segmentName;
+    final String _id;
+
+    SegmentKey(String segmentName, String id) throws IOException {
+      _segmentName = segmentName;
+      _id = id;
+    }
+
+    String getSegmentName() {
+      return _segmentName;
+    }
+
+    @Override
+    public String toString() {
+      return _id;
+    }
+  }
+
+  private DirectoryReader getReader() throws IOException {
+    if (_reader == null) {
+      _reader = DirectoryReader.open(_indexCommit);
+    }
+    return _reader;
+  }
+
+  private Collection<SegmentKey> getSegmentKeys() throws IOException {
+    List<SegmentKey> keys = new ArrayList<SegmentKey>();
+    SegmentInfos segmentInfos = new SegmentInfos();
+    segmentInfos.read(_directory, _indexCommit.getSegmentsFileName());
+    for (SegmentInfoPerCommit segmentInfoPerCommit : segmentInfos) {
+      String name = segmentInfoPerCommit.info.name;
+      String id = getId(segmentInfoPerCommit.info);
+      keys.add(new SegmentKey(name, id));
+    }
+    return keys;
+  }
+
+  private String getId(SegmentInfo si) throws IOException {
+    HdfsDirectory dir = getHdfsDirectory(si.dir);
+    Set<String> files = new TreeSet<String>(si.files());
+    return getId(_configuration, dir, files);
+  }
+
+  private static String getId(Configuration configuration, HdfsDirectory dir, Set<String> files) throws IOException {
+    long ts = 0;
+    String file = null;
+    for (String f : files) {
+      if (f.endsWith(DEL)) {
+        continue;
+      }
+      long fileModified = dir.getFileModified(f);
+      if (fileModified > ts) {
+        ts = fileModified;
+        file = f;
+      }
+    }
+
+    Path path = dir.getPath();
+    FileSystem fileSystem = path.getFileSystem(configuration);
+    Path realFile = new Path(path, file);
+    if (!fileSystem.exists(realFile)) {
+      realFile = dir.getRealFilePathFromSymlink(file);
+      if (!fileSystem.exists(realFile)) {
+        throw new IOException("Lucene file [" + file + "] for dir [" + path + "] can not be found.");
+      }
+    }
+    return getFirstBlockId(fileSystem, realFile);
+  }
+
+  public static String getIdForSingleSegmentIndex(Configuration configuration, Path indexPath) throws IOException {
+    HdfsDirectory dir = new HdfsDirectory(configuration, indexPath);
+    Set<String> files = new TreeSet<String>(Arrays.asList(dir.listAll()));
+    return getId(configuration, dir, files);
+  }
+
+  private static String getFirstBlockId(FileSystem fileSystem, Path realFile) throws IOException {
+    FileStatus fileStatus = fileSystem.getFileStatus(realFile);
+    BlockLocation[] locations = fileSystem.getFileBlockLocations(fileStatus, 0, 1);
+    HdfsBlockLocation location = (HdfsBlockLocation) locations[0];
+    LocatedBlock locatedBlock = location.getLocatedBlock();
+    ExtendedBlock block = locatedBlock.getBlock();
+    return toNiceString(block.getBlockId());
+  }
+
+  private static String toNiceString(long blockId) {
+    return "b" + blockId;
+  }
+
+  private static HdfsDirectory getHdfsDirectory(Directory dir) {
+    if (dir instanceof HdfsDirectory) {
+      return (HdfsDirectory) dir;
+    } else if (dir instanceof DirectoryDecorator) {
+      DirectoryDecorator dd = (DirectoryDecorator) dir;
+      return getHdfsDirectory(dd.getOriginalDirectory());
+    } else {
+      throw new RuntimeException("Unknown directory type.");
+    }
+  }
+
+  private Path getCacheFilePath(SegmentKey segmentKey) {
+    return new Path(_cachePath, segmentKey + ".seq");
+  }
+
+  static class MyReader {
+
+    final Reader _reader;
+    final Text _rowId = new Text();
+    boolean _finished = false;
+
+    public MyReader(Reader reader) {
+      _reader = reader;
+    }
+
+    public Text getCurrentRowId() {
+      return _rowId;
+    }
+
+    public boolean next() throws IOException {
+      if (_finished) {
+        return false;
+      }
+      if (_reader.next(_rowId)) {
+        return true;
+      }
+      _finished = true;
+      return false;
+    }
+
+    public boolean isFinished() {
+      return _finished;
+    }
+  }
+
+  public static Path getCachePath(Path cachePath, String table, String shardName) {
+    return new Path(new Path(cachePath, table), shardName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/ExistingDataIndexLookupMapper.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/ExistingDataIndexLookupMapper.java b/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/ExistingDataIndexLookupMapper.java
new file mode 100644
index 0000000..90e5f9c
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/ExistingDataIndexLookupMapper.java
@@ -0,0 +1,230 @@
+/**
+ * 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.blur.indexer.mapreduce;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.blur.BlurConfiguration;
+import org.apache.blur.indexer.BlurIndexCounter;
+import org.apache.blur.manager.BlurPartitioner;
+import org.apache.blur.manager.writer.SnapshotIndexDeletionPolicy;
+import org.apache.blur.mapreduce.lib.BlurOutputFormat;
+import org.apache.blur.mapreduce.lib.BlurRecord;
+import org.apache.blur.mapreduce.lib.update.IndexKey;
+import org.apache.blur.mapreduce.lib.update.IndexValue;
+import org.apache.blur.store.BlockCacheDirectoryFactoryV2;
+import org.apache.blur.store.hdfs.HdfsDirectory;
+import org.apache.blur.thrift.generated.Column;
+import org.apache.blur.thrift.generated.FetchRecordResult;
+import org.apache.blur.thrift.generated.Record;
+import org.apache.blur.thrift.generated.TableDescriptor;
+import org.apache.blur.utils.BlurConstants;
+import org.apache.blur.utils.RowDocumentUtil;
+import org.apache.blur.utils.ShardUtil;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexCommit;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.Directory;
+
+import com.google.common.io.Closer;
+
+public class ExistingDataIndexLookupMapper extends Mapper<Text, BooleanWritable, IndexKey, IndexValue> {
+
+  private static final Log LOG = LogFactory.getLog(ExistingDataIndexLookupMapper.class);
+  private static final String BLUR_SNAPSHOT = "blur.snapshot";
+  
+  private Counter _existingRecords;
+  private Counter _rowLookup;
+  private BlurPartitioner _blurPartitioner;
+  private Path _tablePath;
+  private int _numberOfShardsInTable;
+  private Configuration _configuration;
+  private String _snapshot;
+  private int _indexShard = -1;
+  private DirectoryReader _reader;
+  private IndexSearcher _indexSearcher;
+  private long _totalNumberOfBytes;
+  private Closer _closer;
+
+  @Override
+  protected void setup(Context context) throws IOException, InterruptedException {
+    Counter counter = context.getCounter(BlurIndexCounter.LOOKUP_MAPPER);
+    counter.increment(1);
+
+    _configuration = context.getConfiguration();
+    _existingRecords = context.getCounter(BlurIndexCounter.LOOKUP_MAPPER_EXISTING_RECORDS);
+    _rowLookup = context.getCounter(BlurIndexCounter.LOOKUP_MAPPER_ROW_LOOKUP_ATTEMPT);
+    _blurPartitioner = new BlurPartitioner();
+    TableDescriptor tableDescriptor = BlurOutputFormat.getTableDescriptor(_configuration);
+    _numberOfShardsInTable = tableDescriptor.getShardCount();
+    _tablePath = new Path(tableDescriptor.getTableUri());
+    _snapshot = getSnapshot(_configuration);
+    _totalNumberOfBytes = _configuration.getLong(LookupBuilderReducer.BLUR_CACHE_DIR_TOTAL_BYTES, 128 * 1024 * 1024);
+    _closer = Closer.create();
+  }
+
+  @Override
+  protected void map(Text key, BooleanWritable value, Context context) throws IOException, InterruptedException {
+    if (value.get()) {
+      String rowId = key.toString();
+      LOG.debug("Looking up rowid [" + rowId + "]");
+      _rowLookup.increment(1);
+      IndexSearcher indexSearcher = getIndexSearcher(rowId);
+      Term term = new Term(BlurConstants.ROW_ID, rowId);
+      RowCollector collector = getCollector(context);
+      indexSearcher.search(new TermQuery(term), collector);
+      LOG.debug("Looking for rowid [" + rowId + "] has [" + collector.records + "] records");
+    }
+  }
+
+  @Override
+  protected void cleanup(Mapper<Text, BooleanWritable, IndexKey, IndexValue>.Context context) throws IOException,
+      InterruptedException {
+    _closer.close();
+  }
+
+  static class RowCollector extends Collector {
+
+    private AtomicReader reader;
+    private Mapper<Text, BooleanWritable, IndexKey, IndexValue>.Context _context;
+    private Counter _existingRecords;
+    int records;
+
+    RowCollector(Mapper<Text, BooleanWritable, IndexKey, IndexValue>.Context context, Counter existingRecords) {
+      _context = context;
+      _existingRecords = existingRecords;
+    }
+
+    @Override
+    public void setScorer(Scorer scorer) throws IOException {
+
+    }
+
+    @Override
+    public void setNextReader(AtomicReaderContext context) throws IOException {
+      reader = context.reader();
+    }
+
+    @Override
+    public void collect(int doc) throws IOException {
+      Document document = reader.document(doc);
+      FetchRecordResult result = RowDocumentUtil.getRecord(document);
+      String rowid = result.getRowid();
+      Record record = result.getRecord();
+      String recordId = record.getRecordId();
+      IndexKey oldDataKey = IndexKey.oldData(rowid, recordId);
+      try {
+        _context.write(oldDataKey, new IndexValue(toBlurRecord(rowid, record)));
+      } catch (InterruptedException e) {
+        throw new IOException(e);
+      }
+      _existingRecords.increment(1L);
+    }
+
+    private BlurRecord toBlurRecord(String rowId, Record record) {
+      BlurRecord blurRecord = new BlurRecord();
+      blurRecord.setRowId(rowId);
+      blurRecord.setRecordId(record.getRecordId());
+      blurRecord.setFamily(record.getFamily());
+      List<Column> columns = record.getColumns();
+      for (Column column : columns) {
+        blurRecord.addColumn(column.getName(), column.getValue());
+      }
+      return blurRecord;
+    }
+
+    @Override
+    public boolean acceptsDocsOutOfOrder() {
+      return false;
+    }
+  }
+
+  private RowCollector getCollector(Mapper<Text, BooleanWritable, IndexKey, IndexValue>.Context context) {
+    return new RowCollector(context, _existingRecords);
+  }
+
+  private IndexSearcher getIndexSearcher(String rowId) throws IOException {
+    int shard = _blurPartitioner.getShard(rowId, _numberOfShardsInTable);
+    if (_indexSearcher != null) {
+      if (shard != _indexShard) {
+        throw new IOException("Input data is not partitioned correctly.");
+      }
+      return _indexSearcher;
+    } else {
+      _indexShard = shard;
+      Path shardPath = new Path(_tablePath, ShardUtil.getShardName(_indexShard));
+      HdfsDirectory hdfsDirectory = new HdfsDirectory(_configuration, shardPath);
+      SnapshotIndexDeletionPolicy policy = new SnapshotIndexDeletionPolicy(_configuration,
+          SnapshotIndexDeletionPolicy.getGenerationsPath(shardPath));
+      Long generation = policy.getGeneration(_snapshot);
+      if (generation == null) {
+        hdfsDirectory.close();
+        throw new IOException("Snapshot [" + _snapshot + "] not found in shard [" + shardPath + "]");
+      }
+
+      BlurConfiguration bc = new BlurConfiguration();
+      BlockCacheDirectoryFactoryV2 blockCacheDirectoryFactoryV2 = new BlockCacheDirectoryFactoryV2(bc,
+          _totalNumberOfBytes);
+      _closer.register(blockCacheDirectoryFactoryV2);
+      Directory dir = blockCacheDirectoryFactoryV2.newDirectory("table", "shard", hdfsDirectory, null);
+
+      List<IndexCommit> listCommits = DirectoryReader.listCommits(dir);
+      IndexCommit indexCommit = findIndexCommit(listCommits, generation, shardPath);
+      _reader = DirectoryReader.open(indexCommit);
+      return _indexSearcher = new IndexSearcher(_reader);
+    }
+  }
+
+  public static IndexCommit findIndexCommit(List<IndexCommit> listCommits, long generation, Path shardDir)
+      throws IOException {
+    for (IndexCommit commit : listCommits) {
+      if (commit.getGeneration() == generation) {
+        return commit;
+      }
+    }
+    throw new IOException("Generation [" + generation + "] not found in shard [" + shardDir + "]");
+  }
+
+  public static void setSnapshot(Job job, String snapshot) {
+    setSnapshot(job.getConfiguration(), snapshot);
+  }
+
+  public static void setSnapshot(Configuration configuration, String snapshot) {
+    configuration.set(BLUR_SNAPSHOT, snapshot);
+  }
+
+  public static String getSnapshot(Configuration configuration) {
+    return configuration.get(BLUR_SNAPSHOT);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/ExistingDataMapper.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/ExistingDataMapper.java b/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/ExistingDataMapper.java
new file mode 100644
index 0000000..5cb0948
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/ExistingDataMapper.java
@@ -0,0 +1,49 @@
+/**
+ * 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.blur.indexer.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.blur.indexer.BlurIndexCounter;
+import org.apache.blur.mapreduce.lib.BlurRecord;
+import org.apache.blur.mapreduce.lib.TableBlurRecord;
+import org.apache.blur.mapreduce.lib.update.IndexKey;
+import org.apache.blur.mapreduce.lib.update.IndexValue;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.Mapper;
+
+public class ExistingDataMapper extends Mapper<Text, TableBlurRecord, IndexKey, IndexValue> {
+
+  private Counter _existingRecords;
+
+  @Override
+  protected void setup(Context context) throws IOException, InterruptedException {
+    Counter counter = context.getCounter(BlurIndexCounter.INPUT_FORMAT_MAPPER);
+    counter.increment(1);
+    _existingRecords = context.getCounter(BlurIndexCounter.INPUT_FORMAT_EXISTING_RECORDS);
+  }
+
+  @Override
+  protected void map(Text key, TableBlurRecord value, Context context) throws IOException, InterruptedException {
+    BlurRecord blurRecord = value.getBlurRecord();
+    IndexKey oldDataKey = IndexKey.oldData(blurRecord.getRowId(), blurRecord.getRecordId());
+    context.write(oldDataKey, new IndexValue(blurRecord));
+    _existingRecords.increment(1L);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/98359a40/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/LookupBuilderMapper.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/LookupBuilderMapper.java b/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/LookupBuilderMapper.java
new file mode 100644
index 0000000..8b3dcb7
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/indexer/mapreduce/LookupBuilderMapper.java
@@ -0,0 +1,34 @@
+/**
+ * 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.blur.indexer.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.blur.mapreduce.lib.BlurRecord;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+
+public class LookupBuilderMapper extends Mapper<Text, BlurRecord, Text, NullWritable> {
+
+  @Override
+  protected void map(Text key, BlurRecord value, Mapper<Text, BlurRecord, Text, NullWritable>.Context context)
+      throws IOException, InterruptedException {
+    context.write(new Text(value.getRowId()), NullWritable.get());
+  }
+
+}


[06/13] git commit: Updates to the indexer project.

Posted by am...@apache.org.
Updates to the indexer project.


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

Branch: refs/heads/master
Commit: 049e79aa97032ae7e2916a59260d0a4d9695cf08
Parents: 96a1821
Author: Aaron McCurry <am...@gmail.com>
Authored: Sun May 22 10:20:50 2016 -0400
Committer: Aaron McCurry <am...@gmail.com>
Committed: Sun May 22 10:20:50 2016 -0400

----------------------------------------------------------------------
 blur-indexer/pom.xml                            | 58 +++++++-------------
 blur-indexer/src/main/assemble/bin.xml          | 45 ---------------
 .../mapreduce/lib/update/BlurIndexCounter.java  | 32 +++++++----
 .../mapreduce/lib/update/ClusterDriver.java     | 16 ++++++
 .../update/HdfsConfigurationNamespaceMerge.java | 16 ++++++
 .../lib/update/InputSplitPruneUtil.java         | 16 ++++++
 .../lib/update/LookupBuilderMapper.java         | 16 ++++++
 .../lib/update/LookupBuilderReducer.java        | 16 ++++++
 .../lib/update/MergeSortRowIdMatcher.java       | 16 ++++++
 .../lib/update/PrunedBlurInputFormat.java       | 16 ++++++
 .../update/PrunedSequenceFileInputFormat.java   | 16 ++++++
 .../src/main/resources/blur-site.properties     |  1 -
 .../src/main/resources/program-log4j.xml        | 29 ----------
 blur-indexer/src/main/resources/test-log4j.xml  | 46 ----------------
 pom.xml                                         |  1 +
 15 files changed, 170 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/blur-indexer/pom.xml
----------------------------------------------------------------------
diff --git a/blur-indexer/pom.xml b/blur-indexer/pom.xml
index c7c1753..a402c6e 100644
--- a/blur-indexer/pom.xml
+++ b/blur-indexer/pom.xml
@@ -1,20 +1,34 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!-- 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. -->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
 	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>
+	<parent>
+		<groupId>org.apache.blur</groupId>
+		<artifactId>blur</artifactId>
+		<version>0.3.0.incubating</version>
+		<relativePath>../pom.xml</relativePath>
+	</parent>
 	<groupId>org.apache.blur</groupId>
 	<artifactId>blur-indexer</artifactId>
-	<version>0.2.8</version>
-	<name>blur-indexer</name>
+	<version>${projectVersion}</version>
+	<name>Blur Indexer</name>
 	<packaging>jar</packaging>
 
-	<properties>
-		<blur.version>0.3.0.incubating.2.5.0.cdh5.3.3-SNAPSHOT</blur.version>
-	</properties>
 	<dependencies>
 		<dependency>
 			<groupId>org.apache.blur</groupId>
 			<artifactId>blur-mapred</artifactId>
-			<version>${blur.version}</version>
+			<version>${project.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>junit</groupId>
@@ -23,36 +37,4 @@
 			<scope>test</scope>
 		</dependency>
 	</dependencies>
-
-	<build>
-		<pluginManagement>
-			<plugins>
-				<plugin>
-					<groupId>org.apache.maven.plugins</groupId>
-					<artifactId>maven-compiler-plugin</artifactId>
-					<configuration>
-						<source>1.8</source>
-						<target>1.8</target>
-					</configuration>
-				</plugin>
-			</plugins>
-		</pluginManagement>
-		<plugins>
-			<plugin>
-				<artifactId>maven-assembly-plugin</artifactId>
-				<configuration>
-					<descriptor>src/main/assemble/bin.xml</descriptor>
-					<finalName>blur-indexer-${project.version}</finalName>
-				</configuration>
-				<executions>
-					<execution>
-						<phase>package</phase>
-						<goals>
-							<goal>single</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/blur-indexer/src/main/assemble/bin.xml
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/assemble/bin.xml b/blur-indexer/src/main/assemble/bin.xml
deleted file mode 100644
index 5fddd56..0000000
--- a/blur-indexer/src/main/assemble/bin.xml
+++ /dev/null
@@ -1,45 +0,0 @@
-<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
-    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	    xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
-  <formats>
-    <format>tar.gz</format>
-  </formats>
-  <includeBaseDirectory>false</includeBaseDirectory>
-
-  <dependencySets>
-    <dependencySet>
-      <useProjectArtifact>true</useProjectArtifact>
-      <outputDirectory>blur-indexer-${project.version}/lib</outputDirectory>
-      <unpack>false</unpack>
-      <includes>
-        <include>org.apache.blur:blur-indexer</include>
-        <include>org.apache.blur:*</include>
-        <include>org.apache.zookeeper:zookeeper</include>
-        <include>org.slf4j:slf4j-api</include>
-        <include>org.slf4j:slf4j-log4j12</include>
-        <include>org.json:json</include>
-        <include>log4j:log4j</include>
-        <include>com.yammer.metrics:*</include>
-        <include>com.google.guava:guava</include>
-        <include>org.apache.httpcomponents:*</include>
-        <include>org.apache.lucene:*</include>
-        <include>com.spatial4j:spatial4j</include>
-        <include>commons-cli:commons-cli</include>
-        <include>org.eclipse.jetty:*</include>
-        <include>com.googlecode.concurrentlinkedhashmap:concurrentlinkedhashmap-lru</include>
-        <include>jline:jline</include>
-        <include>com.fasterxml.jackson.core:*</include>
-      </includes>
-    </dependencySet>
-  </dependencySets>
-
-  <fileSets>
-    <fileSet>
-      <directory>${project.build.scriptSourceDirectory}</directory>
-      <outputDirectory>blur-indexer-${project.version}</outputDirectory>
-      <excludes>
-        <exclude>**/.empty</exclude>
-      </excludes>
-    </fileSet>
-  </fileSets>
-</assembly>

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
index a9caabb..590ba83 100644
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/BlurIndexCounter.java
@@ -1,17 +1,27 @@
+/**
+ * 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.blur.mapreduce.lib.update;
 
 public enum BlurIndexCounter {
 
-  NEW_RECORDS,
-  ROW_IDS_FROM_INDEX,
-  ROW_IDS_TO_UPDATE_FROM_NEW_DATA,
-  ROW_IDS_FROM_NEW_DATA,
-  
-  INPUT_FORMAT_MAPPER, 
-  INPUT_FORMAT_EXISTING_RECORDS,
-  
-  LOOKUP_MAPPER, 
-  LOOKUP_MAPPER_EXISTING_RECORDS, 
-  LOOKUP_MAPPER_ROW_LOOKUP_ATTEMPT
+  NEW_RECORDS, ROW_IDS_FROM_INDEX, ROW_IDS_TO_UPDATE_FROM_NEW_DATA, ROW_IDS_FROM_NEW_DATA,
+
+  INPUT_FORMAT_MAPPER, INPUT_FORMAT_EXISTING_RECORDS,
+
+  LOOKUP_MAPPER, LOOKUP_MAPPER_EXISTING_RECORDS, LOOKUP_MAPPER_ROW_LOOKUP_ATTEMPT
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
index d44adf1..f56b731 100644
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/ClusterDriver.java
@@ -1,3 +1,19 @@
+/**
+ * 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.blur.mapreduce.lib.update;
 
 import java.io.ByteArrayInputStream;

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/HdfsConfigurationNamespaceMerge.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/HdfsConfigurationNamespaceMerge.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/HdfsConfigurationNamespaceMerge.java
index 34d3e99..de96d24 100644
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/HdfsConfigurationNamespaceMerge.java
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/HdfsConfigurationNamespaceMerge.java
@@ -1,3 +1,19 @@
+/**
+ * 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.blur.mapreduce.lib.update;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/InputSplitPruneUtil.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/InputSplitPruneUtil.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/InputSplitPruneUtil.java
index e295073..80d1410 100644
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/InputSplitPruneUtil.java
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/InputSplitPruneUtil.java
@@ -1,3 +1,19 @@
+/**
+ * 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.blur.mapreduce.lib.update;
 
 import org.apache.blur.utils.ShardUtil;

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderMapper.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderMapper.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderMapper.java
index ac0d91f..87a3a32 100644
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderMapper.java
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderMapper.java
@@ -1,3 +1,19 @@
+/**
+ * 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.blur.mapreduce.lib.update;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderReducer.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderReducer.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderReducer.java
index 1983cae..f3a2697 100644
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderReducer.java
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/LookupBuilderReducer.java
@@ -1,3 +1,19 @@
+/**
+ * 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.blur.mapreduce.lib.update;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MergeSortRowIdMatcher.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MergeSortRowIdMatcher.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MergeSortRowIdMatcher.java
index f376274..bd8580e 100644
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MergeSortRowIdMatcher.java
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MergeSortRowIdMatcher.java
@@ -1,3 +1,19 @@
+/**
+ * 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.blur.mapreduce.lib.update;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedBlurInputFormat.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedBlurInputFormat.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedBlurInputFormat.java
index 6ec2877..8738c5a 100644
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedBlurInputFormat.java
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedBlurInputFormat.java
@@ -1,3 +1,19 @@
+/**
+ * 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.blur.mapreduce.lib.update;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedSequenceFileInputFormat.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedSequenceFileInputFormat.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedSequenceFileInputFormat.java
index becebbd..58e9800 100644
--- a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedSequenceFileInputFormat.java
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedSequenceFileInputFormat.java
@@ -1,3 +1,19 @@
+/**
+ * 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.blur.mapreduce.lib.update;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/blur-indexer/src/main/resources/blur-site.properties
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/resources/blur-site.properties b/blur-indexer/src/main/resources/blur-site.properties
deleted file mode 100644
index 6b28452..0000000
--- a/blur-indexer/src/main/resources/blur-site.properties
+++ /dev/null
@@ -1 +0,0 @@
-blur.thrift.max.frame.size=131072000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/blur-indexer/src/main/resources/program-log4j.xml
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/resources/program-log4j.xml b/blur-indexer/src/main/resources/program-log4j.xml
deleted file mode 100644
index 30c132b..0000000
--- a/blur-indexer/src/main/resources/program-log4j.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" ?>
-<!-- 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. -->
-<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
-
-<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
-
-	<appender name="FILE" class="org.apache.log4j.DailyRollingFileAppender">
-		<param name="File" value="${BLUR_INDEXER_LOG_FILE}" />
-		<param name="DatePattern" value="'.'yyyyMMdd" />
-		<param name="Append" value="true" />
-		<layout class="org.apache.log4j.PatternLayout">
-			<param name="ConversionPattern" value="%-5p %d{yyyyMMdd_HH:mm:ss:SSS_z} [%t] %c{2}: %m%n" />
-		</layout>
-	</appender>
-
-	<root>
-		<priority value="INFO" />
-		<appender-ref ref="FILE" />
-	</root>
-</log4j:configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/blur-indexer/src/main/resources/test-log4j.xml
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/resources/test-log4j.xml b/blur-indexer/src/main/resources/test-log4j.xml
deleted file mode 100644
index bf705ca..0000000
--- a/blur-indexer/src/main/resources/test-log4j.xml
+++ /dev/null
@@ -1,46 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" ?>
-<!-- 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. -->
-<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
-
-<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
-	<appender name="console" class="org.apache.log4j.ConsoleAppender">
-		<param name="Target" value="System.out" />
-		<layout class="org.apache.log4j.PatternLayout">
-			<param name="ConversionPattern" value="%-5p %d{yyyyMMdd_HH:mm:ss:SSS_z} [%t] %c{2}: %m%n" />
-		</layout>
-	</appender>
-	<logger name="org.apache.hadoop">
-    	<level value="ERROR" />
-	    <appender-ref ref="console"/>
-	</logger>
-	<logger name="REQUEST_LOG" additivity="false">
-		<!-- Make value = "INFO"to enable -->
-    	<level value="ERROR" />
-	    <appender-ref ref="console"/>
-	</logger>
-	
-	<logger name="RESPONSE_LOG" additivity="false">
-		<!-- Make value = "INFO"to enable -->
-    	<level value="ERROR" />
-	    <appender-ref ref="console"/>
-	</logger>
-	
-	<logger name="LUCENE_WRITER_INFO_STREAM" additivity="false">
-		<!-- Make value = "INFO"to enable -->
-    	<level value="ERROR" />
-	    <appender-ref ref="console"/>
-	</logger>
-	<root>
-		<priority value="info" />
-		<appender-ref ref="console" />
-	</root>
-</log4j:configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/049e79aa/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 7d3f042..4a37560 100644
--- a/pom.xml
+++ b/pom.xml
@@ -494,6 +494,7 @@ under the License.
 		<module>blur-query</module>
 		<module>blur-store</module>
 		<module>blur-mapred</module>
+		<module>blur-indexer</module>
 		<module>blur-util</module>
 		<module>blur-status</module>
 		<module>blur-shell</module>


[03/13] git commit: Third round of updates.

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MergeSortRowIdMatcher.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MergeSortRowIdMatcher.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MergeSortRowIdMatcher.java
new file mode 100644
index 0000000..f376274
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/MergeSortRowIdMatcher.java
@@ -0,0 +1,372 @@
+package org.apache.blur.mapreduce.lib.update;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.blur.index.AtomicReaderUtil;
+import org.apache.blur.log.Log;
+import org.apache.blur.log.LogFactory;
+import org.apache.blur.store.hdfs.DirectoryDecorator;
+import org.apache.blur.store.hdfs.HdfsDirectory;
+import org.apache.blur.utils.BlurConstants;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.HdfsBlockLocation;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DeflateCodec;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.io.compress.zlib.ZlibFactory;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexCommit;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentInfoPerCommit;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.SegmentReader;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+
+public class MergeSortRowIdMatcher {
+
+  private static final String DEL = ".del";
+  private static final Log LOG = LogFactory.getLog(MergeSortRowIdMatcher.class);
+  private static final Progressable NO_OP = new Progressable() {
+    @Override
+    public void progress() {
+
+    }
+  };
+  private static final long _10_SECONDS = TimeUnit.SECONDS.toNanos(10);
+
+  public interface Action {
+    void found(Text rowId) throws IOException;
+  }
+
+  private final MyReader[] _readers;
+  private final Configuration _configuration;
+  private final Path _cachePath;
+  private final IndexCommit _indexCommit;
+  private final Directory _directory;
+  private final Progressable _progressable;
+
+  private DirectoryReader _reader;
+
+  public MergeSortRowIdMatcher(Directory directory, long generation, Configuration configuration, Path cachePath)
+      throws IOException {
+    this(directory, generation, configuration, cachePath, null);
+  }
+
+  public MergeSortRowIdMatcher(Directory directory, long generation, Configuration configuration, Path cachePath,
+      Progressable progressable) throws IOException {
+    List<IndexCommit> listCommits = DirectoryReader.listCommits(directory);
+    _indexCommit = findIndexCommit(listCommits, generation);
+    _configuration = configuration;
+    _cachePath = cachePath;
+    _directory = directory;
+    _progressable = progressable == null ? NO_OP : progressable;
+    _readers = openReaders();
+  }
+
+  public void lookup(Text rowId, Action action) throws IOException {
+    if (lookup(rowId)) {
+      action.found(rowId);
+    }
+  }
+
+  private boolean lookup(Text rowId) throws IOException {
+    advanceReadersIfNeeded(rowId);
+    sortReaders();
+    return checkReaders(rowId);
+  }
+
+  private boolean checkReaders(Text rowId) {
+    for (MyReader reader : _readers) {
+      int compareTo = reader.getCurrentRowId().compareTo(rowId);
+      if (compareTo == 0) {
+        return true;
+      } else if (compareTo > 0) {
+        return false;
+      }
+    }
+    return false;
+  }
+
+  private void advanceReadersIfNeeded(Text rowId) throws IOException {
+    _progressable.progress();
+    for (MyReader reader : _readers) {
+      if (rowId.compareTo(reader.getCurrentRowId()) > 0) {
+        advanceReader(reader, rowId);
+      }
+    }
+  }
+
+  private void advanceReader(MyReader reader, Text rowId) throws IOException {
+    while (reader.next()) {
+      if (rowId.compareTo(reader.getCurrentRowId()) <= 0) {
+        return;
+      }
+    }
+  }
+
+  private static final Comparator<MyReader> COMP = new Comparator<MyReader>() {
+    @Override
+    public int compare(MyReader o1, MyReader o2) {
+      return o1.getCurrentRowId().compareTo(o2.getCurrentRowId());
+    }
+  };
+
+  private void sortReaders() {
+    Arrays.sort(_readers, COMP);
+  }
+
+  private MyReader[] openReaders() throws IOException {
+    Collection<SegmentKey> segmentKeys = getSegmentKeys();
+    MyReader[] readers = new MyReader[segmentKeys.size()];
+    int i = 0;
+    for (SegmentKey segmentKey : segmentKeys) {
+      readers[i++] = openReader(segmentKey);
+    }
+    return readers;
+  }
+
+  private MyReader openReader(SegmentKey segmentKey) throws IOException {
+    Path file = getCacheFilePath(segmentKey);
+    FileSystem fileSystem = _cachePath.getFileSystem(_configuration);
+    if (!fileSystem.exists(file)) {
+      createCacheFile(file, segmentKey);
+    }
+    Reader reader = new SequenceFile.Reader(_configuration, SequenceFile.Reader.file(file));
+    return new MyReader(reader);
+  }
+
+  private void createCacheFile(Path file, SegmentKey segmentKey) throws IOException {
+    LOG.info("Building cache for segment [{0}] to [{1}]", segmentKey, file);
+    Path tmpPath = getTmpWriterPath(file.getParent());
+    try (Writer writer = createWriter(_configuration, tmpPath)) {
+      DirectoryReader reader = getReader();
+      for (AtomicReaderContext context : reader.leaves()) {
+        SegmentReader segmentReader = AtomicReaderUtil.getSegmentReader(context.reader());
+        if (segmentReader.getSegmentName().equals(segmentKey.getSegmentName())) {
+          writeRowIds(writer, segmentReader);
+          break;
+        }
+      }
+    }
+    commitWriter(_configuration, file, tmpPath);
+  }
+
+  public static void commitWriter(Configuration configuration, Path file, Path tmpPath) throws IOException {
+    FileSystem fileSystem = tmpPath.getFileSystem(configuration);
+    LOG.info("Commit tmp [{0}] to file [{1}]", tmpPath, file);
+    if (!fileSystem.rename(tmpPath, file)) {
+      LOG.warn("Could not commit tmp file [{0}] to file [{1}]", tmpPath, file);
+    }
+  }
+
+  public static Path getTmpWriterPath(Path dir) {
+    return new Path(dir, UUID.randomUUID().toString() + ".tmp");
+  }
+
+  public static Writer createWriter(Configuration configuration, Path tmpPath) throws IOException {
+    return SequenceFile.createWriter(configuration, SequenceFile.Writer.file(tmpPath),
+        SequenceFile.Writer.keyClass(Text.class), SequenceFile.Writer.valueClass(NullWritable.class),
+        SequenceFile.Writer.compression(CompressionType.BLOCK, getCodec(configuration)));
+  }
+
+  private static CompressionCodec getCodec(Configuration configuration) {
+    if (ZlibFactory.isNativeZlibLoaded(configuration)) {
+      return new GzipCodec();
+    }
+    return new DeflateCodec();
+  }
+
+  private void writeRowIds(Writer writer, SegmentReader segmentReader) throws IOException {
+    Terms terms = segmentReader.terms(BlurConstants.ROW_ID);
+    if (terms == null) {
+      return;
+    }
+    TermsEnum termsEnum = terms.iterator(null);
+    BytesRef rowId;
+    long s = System.nanoTime();
+    while ((rowId = termsEnum.next()) != null) {
+      long n = System.nanoTime();
+      if (n + _10_SECONDS > s) {
+        _progressable.progress();
+        s = System.nanoTime();
+      }
+      writer.append(new Text(rowId.utf8ToString()), NullWritable.get());
+    }
+  }
+
+  private IndexCommit findIndexCommit(List<IndexCommit> listCommits, long generation) throws IOException {
+    for (IndexCommit commit : listCommits) {
+      if (commit.getGeneration() == generation) {
+        return commit;
+      }
+    }
+    throw new IOException("Generation [" + generation + "] not found.");
+  }
+
+  static class SegmentKey {
+
+    final String _segmentName;
+    final String _id;
+
+    SegmentKey(String segmentName, String id) throws IOException {
+      _segmentName = segmentName;
+      _id = id;
+    }
+
+    String getSegmentName() {
+      return _segmentName;
+    }
+
+    @Override
+    public String toString() {
+      return _id;
+    }
+  }
+
+  private DirectoryReader getReader() throws IOException {
+    if (_reader == null) {
+      _reader = DirectoryReader.open(_indexCommit);
+    }
+    return _reader;
+  }
+
+  private Collection<SegmentKey> getSegmentKeys() throws IOException {
+    List<SegmentKey> keys = new ArrayList<SegmentKey>();
+    SegmentInfos segmentInfos = new SegmentInfos();
+    segmentInfos.read(_directory, _indexCommit.getSegmentsFileName());
+    for (SegmentInfoPerCommit segmentInfoPerCommit : segmentInfos) {
+      String name = segmentInfoPerCommit.info.name;
+      String id = getId(segmentInfoPerCommit.info);
+      keys.add(new SegmentKey(name, id));
+    }
+    return keys;
+  }
+
+  private String getId(SegmentInfo si) throws IOException {
+    HdfsDirectory dir = getHdfsDirectory(si.dir);
+    Set<String> files = new TreeSet<String>(si.files());
+    return getId(_configuration, dir, files);
+  }
+
+  private static String getId(Configuration configuration, HdfsDirectory dir, Set<String> files) throws IOException {
+    long ts = 0;
+    String file = null;
+    for (String f : files) {
+      if (f.endsWith(DEL)) {
+        continue;
+      }
+      long fileModified = dir.getFileModified(f);
+      if (fileModified > ts) {
+        ts = fileModified;
+        file = f;
+      }
+    }
+
+    Path path = dir.getPath();
+    FileSystem fileSystem = path.getFileSystem(configuration);
+    Path realFile = new Path(path, file);
+    if (!fileSystem.exists(realFile)) {
+      realFile = dir.getRealFilePathFromSymlink(file);
+      if (!fileSystem.exists(realFile)) {
+        throw new IOException("Lucene file [" + file + "] for dir [" + path + "] can not be found.");
+      }
+    }
+    return getFirstBlockId(fileSystem, realFile);
+  }
+
+  public static String getIdForSingleSegmentIndex(Configuration configuration, Path indexPath) throws IOException {
+    HdfsDirectory dir = new HdfsDirectory(configuration, indexPath);
+    Set<String> files = new TreeSet<String>(Arrays.asList(dir.listAll()));
+    return getId(configuration, dir, files);
+  }
+
+  private static String getFirstBlockId(FileSystem fileSystem, Path realFile) throws IOException {
+    FileStatus fileStatus = fileSystem.getFileStatus(realFile);
+    BlockLocation[] locations = fileSystem.getFileBlockLocations(fileStatus, 0, 1);
+    HdfsBlockLocation location = (HdfsBlockLocation) locations[0];
+    LocatedBlock locatedBlock = location.getLocatedBlock();
+    ExtendedBlock block = locatedBlock.getBlock();
+    return toNiceString(block.getBlockId());
+  }
+
+  private static String toNiceString(long blockId) {
+    return "b" + blockId;
+  }
+
+  private static HdfsDirectory getHdfsDirectory(Directory dir) {
+    if (dir instanceof HdfsDirectory) {
+      return (HdfsDirectory) dir;
+    } else if (dir instanceof DirectoryDecorator) {
+      DirectoryDecorator dd = (DirectoryDecorator) dir;
+      return getHdfsDirectory(dd.getOriginalDirectory());
+    } else {
+      throw new RuntimeException("Unknown directory type.");
+    }
+  }
+
+  private Path getCacheFilePath(SegmentKey segmentKey) {
+    return new Path(_cachePath, segmentKey + ".seq");
+  }
+
+  static class MyReader {
+
+    final Reader _reader;
+    final Text _rowId = new Text();
+    boolean _finished = false;
+
+    public MyReader(Reader reader) {
+      _reader = reader;
+    }
+
+    public Text getCurrentRowId() {
+      return _rowId;
+    }
+
+    public boolean next() throws IOException {
+      if (_finished) {
+        return false;
+      }
+      if (_reader.next(_rowId)) {
+        return true;
+      }
+      _finished = true;
+      return false;
+    }
+
+    public boolean isFinished() {
+      return _finished;
+    }
+  }
+
+  public static Path getCachePath(Path cachePath, String table, String shardName) {
+    return new Path(new Path(cachePath, table), shardName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedBlurInputFormat.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedBlurInputFormat.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedBlurInputFormat.java
new file mode 100644
index 0000000..6ec2877
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedBlurInputFormat.java
@@ -0,0 +1,57 @@
+package org.apache.blur.mapreduce.lib.update;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.blur.mapreduce.lib.BlurInputFormat;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+
+public class PrunedBlurInputFormat extends BlurInputFormat {
+
+  private static final Log LOG = LogFactory.getLog(PrunedBlurInputFormat.class);
+
+  @Override
+  public List<InputSplit> getSplits(JobContext context) throws IOException {
+    Path[] dirs = getInputPaths(context);
+    Configuration configuration = context.getConfiguration();
+    List<BlurInputSplit> splits = getSplits(configuration, dirs);
+    Map<Path, List<BlurInputSplit>> splitMap = new TreeMap<Path, List<BlurInputSplit>>();
+    for (BlurInputSplit split : splits) {
+      Path path = split.getDir();
+      String table = split.getTable().toString();
+      int shard = InputSplitPruneUtil.getShardFromDirectoryPath(path);
+      long rowIdUpdateFromNewDataCount = InputSplitPruneUtil.getBlurLookupRowIdUpdateFromNewDataCount(configuration,
+          table, shard);
+      long indexCount = InputSplitPruneUtil.getBlurLookupRowIdFromIndexCount(configuration, table, shard);
+      if (rowIdUpdateFromNewDataCount == 0 || indexCount == 0) {
+        LOG.info("Pruning id lookup input path [" + path + "] no overlapping ids.");
+      } else if (InputSplitPruneUtil.shouldLookupExecuteOnShard(configuration, table, shard)) {
+        LOG.info("Pruning blur input path [" + split.getDir() + "]");
+      } else {
+        LOG.debug("Keeping blur input path [" + split.getDir() + "]");
+        List<BlurInputSplit> list = splitMap.get(path);
+        if (list == null) {
+          splitMap.put(path, list = new ArrayList<BlurInputSplit>());
+        }
+        list.add(split);
+      }
+    }
+    List<InputSplit> result = new ArrayList<InputSplit>();
+    for (List<BlurInputSplit> lst : splitMap.values()) {
+      BlurInputSplitColletion blurInputSplitColletion = new BlurInputSplitColletion();
+      for (BlurInputSplit blurInputSplit : lst) {
+        blurInputSplitColletion.add(blurInputSplit);
+      }
+      result.add(blurInputSplitColletion);
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedSequenceFileInputFormat.java
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedSequenceFileInputFormat.java b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedSequenceFileInputFormat.java
new file mode 100644
index 0000000..becebbd
--- /dev/null
+++ b/blur-indexer/src/main/java/org/apache/blur/mapreduce/lib/update/PrunedSequenceFileInputFormat.java
@@ -0,0 +1,59 @@
+package org.apache.blur.mapreduce.lib.update;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+
+import com.google.common.base.Splitter;
+
+public class PrunedSequenceFileInputFormat<K, V> extends SequenceFileInputFormat<K, V> {
+
+  private static final Log LOG = LogFactory.getLog(PrunedSequenceFileInputFormat.class);
+
+  @Override
+  public List<InputSplit> getSplits(JobContext job) throws IOException {
+    List<InputSplit> splits = super.getSplits(job);
+    List<InputSplit> results = new ArrayList<InputSplit>();
+    Configuration configuration = job.getConfiguration();
+    String table = InputSplitPruneUtil.getTable(configuration);
+    for (InputSplit inputSplit : splits) {
+      FileSplit fileSplit = (FileSplit) inputSplit;
+      Path path = fileSplit.getPath();
+      LOG.debug("Getting shard index from path [" + path + "]");
+      String name = path.getName();
+      int shard = getShardIndex(name);
+      long rowIdUpdateFromNewDataCount = InputSplitPruneUtil.getBlurLookupRowIdUpdateFromNewDataCount(configuration,
+          table, shard);
+      long indexCount = InputSplitPruneUtil.getBlurLookupRowIdFromIndexCount(configuration, table, shard);
+      if (rowIdUpdateFromNewDataCount == 0 || indexCount == 0) {
+        LOG.info("Pruning id lookup input path [" + path + "] no overlapping ids.");
+      } else if (InputSplitPruneUtil.shouldLookupExecuteOnShard(configuration, table, shard)) {
+        LOG.debug("Keeping id lookup input path [" + path + "]");
+        results.add(inputSplit);
+      } else {
+        LOG.info("Pruning id lookup input path [" + path + "]");
+      }
+    }
+    return results;
+  }
+
+  private int getShardIndex(String name) {
+    // based on file format of "part-r-00000", etc
+    Iterable<String> split = Splitter.on('-').split(name);
+    List<String> parts = new ArrayList<String>();
+    for (String s : split) {
+      parts.add(s);
+    }
+    return Integer.parseInt(parts.get(2));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/resources/blur-site.properties
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/resources/blur-site.properties b/blur-indexer/src/main/resources/blur-site.properties
new file mode 100644
index 0000000..6b28452
--- /dev/null
+++ b/blur-indexer/src/main/resources/blur-site.properties
@@ -0,0 +1 @@
+blur.thrift.max.frame.size=131072000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/resources/program-log4j.xml
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/resources/program-log4j.xml b/blur-indexer/src/main/resources/program-log4j.xml
new file mode 100644
index 0000000..30c132b
--- /dev/null
+++ b/blur-indexer/src/main/resources/program-log4j.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!-- 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. -->
+<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
+
+<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
+
+	<appender name="FILE" class="org.apache.log4j.DailyRollingFileAppender">
+		<param name="File" value="${BLUR_INDEXER_LOG_FILE}" />
+		<param name="DatePattern" value="'.'yyyyMMdd" />
+		<param name="Append" value="true" />
+		<layout class="org.apache.log4j.PatternLayout">
+			<param name="ConversionPattern" value="%-5p %d{yyyyMMdd_HH:mm:ss:SSS_z} [%t] %c{2}: %m%n" />
+		</layout>
+	</appender>
+
+	<root>
+		<priority value="INFO" />
+		<appender-ref ref="FILE" />
+	</root>
+</log4j:configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/ea50630a/blur-indexer/src/main/resources/test-log4j.xml
----------------------------------------------------------------------
diff --git a/blur-indexer/src/main/resources/test-log4j.xml b/blur-indexer/src/main/resources/test-log4j.xml
new file mode 100644
index 0000000..bf705ca
--- /dev/null
+++ b/blur-indexer/src/main/resources/test-log4j.xml
@@ -0,0 +1,46 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!-- 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. -->
+<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
+
+<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
+	<appender name="console" class="org.apache.log4j.ConsoleAppender">
+		<param name="Target" value="System.out" />
+		<layout class="org.apache.log4j.PatternLayout">
+			<param name="ConversionPattern" value="%-5p %d{yyyyMMdd_HH:mm:ss:SSS_z} [%t] %c{2}: %m%n" />
+		</layout>
+	</appender>
+	<logger name="org.apache.hadoop">
+    	<level value="ERROR" />
+	    <appender-ref ref="console"/>
+	</logger>
+	<logger name="REQUEST_LOG" additivity="false">
+		<!-- Make value = "INFO"to enable -->
+    	<level value="ERROR" />
+	    <appender-ref ref="console"/>
+	</logger>
+	
+	<logger name="RESPONSE_LOG" additivity="false">
+		<!-- Make value = "INFO"to enable -->
+    	<level value="ERROR" />
+	    <appender-ref ref="console"/>
+	</logger>
+	
+	<logger name="LUCENE_WRITER_INFO_STREAM" additivity="false">
+		<!-- Make value = "INFO"to enable -->
+    	<level value="ERROR" />
+	    <appender-ref ref="console"/>
+	</logger>
+	<root>
+		<priority value="info" />
+		<appender-ref ref="console" />
+	</root>
+</log4j:configuration>
\ No newline at end of file


[11/13] git commit: Removing thread local variables.

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-thrift/src/main/java/org/apache/blur/thrift/BlurClientManager.java
----------------------------------------------------------------------
diff --git a/blur-thrift/src/main/java/org/apache/blur/thrift/BlurClientManager.java b/blur-thrift/src/main/java/org/apache/blur/thrift/BlurClientManager.java
index 8c0e9ba..766cc59 100644
--- a/blur-thrift/src/main/java/org/apache/blur/thrift/BlurClientManager.java
+++ b/blur-thrift/src/main/java/org/apache/blur/thrift/BlurClientManager.java
@@ -50,6 +50,7 @@ import org.apache.blur.trace.Trace;
 import org.apache.blur.trace.Trace.TraceId;
 import org.apache.blur.trace.Tracer;
 import org.apache.blur.user.UserContext;
+import org.apache.blur.utils.ThreadValue;
 
 public class BlurClientManager {
 
@@ -141,7 +142,7 @@ public class BlurClientManager {
     List<Connection> shuffledConnections = new ArrayList<Connection>();
   }
 
-  private static ThreadLocal<Random> _random = new ThreadLocal<Random>() {
+  private static ThreadValue<Random> _random = new ThreadValue<Random>() {
     @Override
     protected Random initialValue() {
       return new Random();

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-thrift/src/main/java/org/apache/blur/thrift/sasl/TSaslTransport.java
----------------------------------------------------------------------
diff --git a/blur-thrift/src/main/java/org/apache/blur/thrift/sasl/TSaslTransport.java b/blur-thrift/src/main/java/org/apache/blur/thrift/sasl/TSaslTransport.java
index 3cf80a4..15361f0 100644
--- a/blur-thrift/src/main/java/org/apache/blur/thrift/sasl/TSaslTransport.java
+++ b/blur-thrift/src/main/java/org/apache/blur/thrift/sasl/TSaslTransport.java
@@ -38,6 +38,7 @@ import org.apache.blur.thirdparty.thrift_0_9_0.transport.TMemoryInputTransport;
 import org.apache.blur.thirdparty.thrift_0_9_0.transport.TSocket;
 import org.apache.blur.thirdparty.thrift_0_9_0.transport.TTransport;
 import org.apache.blur.thirdparty.thrift_0_9_0.transport.TTransportException;
+import org.apache.blur.utils.ThreadValue;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -311,7 +312,7 @@ abstract class TSaslTransport extends TTransport {
     _currentConnection.set(null);
   }
 
-  static ThreadLocal<InetSocketAddress> _currentConnection = new ThreadLocal<InetSocketAddress>();
+  static ThreadValue<InetSocketAddress> _currentConnection = new ThreadValue<InetSocketAddress>();
 
   private void setupConnectionInfo() {
     if (underlyingTransport instanceof TSocket) {

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-util/src/main/java/org/apache/blur/trace/Trace.java
----------------------------------------------------------------------
diff --git a/blur-util/src/main/java/org/apache/blur/trace/Trace.java b/blur-util/src/main/java/org/apache/blur/trace/Trace.java
index 65c2bfa..8f4bbce 100644
--- a/blur-util/src/main/java/org/apache/blur/trace/Trace.java
+++ b/blur-util/src/main/java/org/apache/blur/trace/Trace.java
@@ -27,6 +27,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.blur.utils.ThreadValue;
 import org.json.JSONException;
 import org.json.JSONObject;
 
@@ -77,10 +78,10 @@ public class Trace {
 
     }
   };
-  private static ThreadLocal<TraceCollector> _tracer = new ThreadLocal<TraceCollector>();
+  private static ThreadValue<TraceCollector> _tracer = new ThreadValue<TraceCollector>();
   private static TraceStorage _storage;
   private static String _nodeName;
-  private static ThreadLocal<Random> _random = new ThreadLocal<Random>() {
+  private static ThreadValue<Random> _random = new ThreadValue<Random>() {
     @Override
     protected Random initialValue() {
       return new Random();

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-util/src/main/java/org/apache/blur/user/UserContext.java
----------------------------------------------------------------------
diff --git a/blur-util/src/main/java/org/apache/blur/user/UserContext.java b/blur-util/src/main/java/org/apache/blur/user/UserContext.java
index 6ae373b..1b4fb1a 100644
--- a/blur-util/src/main/java/org/apache/blur/user/UserContext.java
+++ b/blur-util/src/main/java/org/apache/blur/user/UserContext.java
@@ -19,6 +19,8 @@ package org.apache.blur.user;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.blur.utils.ThreadValue;
+
 public class UserContext {
 
   private static User _defaultUser;
@@ -45,7 +47,7 @@ public class UserContext {
     return new User(user.getUsername(), null);
   }
 
-  private static ThreadLocal<User> _user = new ThreadLocal<User>() {
+  private static ThreadValue<User> _user = new ThreadValue<User>() {
     @Override
     protected User initialValue() {
       return getDefaultUser();

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-util/src/main/java/org/apache/blur/utils/ThreadValue.java
----------------------------------------------------------------------
diff --git a/blur-util/src/main/java/org/apache/blur/utils/ThreadValue.java b/blur-util/src/main/java/org/apache/blur/utils/ThreadValue.java
new file mode 100644
index 0000000..80c59b6
--- /dev/null
+++ b/blur-util/src/main/java/org/apache/blur/utils/ThreadValue.java
@@ -0,0 +1,56 @@
+/**
+ * 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.blur.utils;
+
+import java.util.concurrent.ConcurrentMap;
+
+import com.google.common.collect.MapMaker;
+
+public class ThreadValue<T> {
+
+  static class Value<T> {
+    T value;
+
+    Value(T value) {
+      this.value = value;
+    }
+  }
+
+  private final ConcurrentMap<Thread, Value<T>> refs = new MapMaker().weakKeys().makeMap();
+
+  protected T initialValue() {
+    return null;
+  }
+
+  public T get() {
+    Value<T> value = refs.get(Thread.currentThread());
+    if (value == null) {
+      refs.put(Thread.currentThread(), value = new Value<>(initialValue()));
+    }
+    return value.value;
+  }
+
+  public void set(T v) {
+    Value<T> value = refs.get(Thread.currentThread());
+    if (value == null) {
+      refs.put(Thread.currentThread(), value = new Value<>(v));
+    } else {
+      value.value = v;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/991fb043/blur-util/src/test/java/org/apache/blur/utils/BlurConstantsTest.java
----------------------------------------------------------------------
diff --git a/blur-util/src/test/java/org/apache/blur/utils/BlurConstantsTest.java b/blur-util/src/test/java/org/apache/blur/utils/BlurConstantsTest.java
index 25b5967..139afb2 100644
--- a/blur-util/src/test/java/org/apache/blur/utils/BlurConstantsTest.java
+++ b/blur-util/src/test/java/org/apache/blur/utils/BlurConstantsTest.java
@@ -43,7 +43,7 @@ public class BlurConstantsTest {
         "DELETE_MARKER_VALUE", "DELETE_MARKER", "BLUR_ZOOKEEPER_TIMEOUT_DEFAULT", "BLUR_THRIFT_DEFAULT_MAX_FRAME_SIZE",
         "ZK_WAIT_TIME", "ACL_DISCOVER", "ACL_READ", "FAST_DECOMPRESSION", "FAST", "HIGH_COMPRESSION", "DEFAULT_VALUE",
         "OFF_HEAP", "DEFAULT", "BLUR_CLUSTER", "BLUR_NODENAME", "BLUR_HTTP_STATUS_RUNNING_PORT",
-        "BLUR_STREAM_SERVER_RUNNING_PORT", "SHARED_MERGE_SCHEDULER_PREFIX"));
+        "BLUR_STREAM_SERVER_RUNNING_PORT", "SHARED_MERGE_SCHEDULER_PREFIX", "UPDATE_ROW", "NEW_ROW", "INTERNAL"));
     _emptyDefaultProperties = new HashSet<String>();
     _emptyDefaultProperties.addAll(Arrays.asList("BLUR_HDFS_TRACE_PATH", "BLUR_SHARD_HOSTNAME",
         "BLUR_SHARD_BLOCK_CACHE_TOTAL_SIZE", "BLUR_CONTROLLER_HOSTNAME", "BLUR_SHARD_READ_INTERCEPTOR",


[07/13] git commit: Fixing api issue.

Posted by am...@apache.org.
Fixing api issue.


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

Branch: refs/heads/master
Commit: 1d103e48b6ecf24f27f6009ea6060709cbb8700a
Parents: 049e79a
Author: Aaron McCurry <am...@gmail.com>
Authored: Sun May 22 10:21:23 2016 -0400
Committer: Aaron McCurry <am...@gmail.com>
Committed: Sun May 22 10:21:23 2016 -0400

----------------------------------------------------------------------
 .../main/java/org/apache/blur/thrift/ThriftBlurShardServer.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/1d103e48/blur-core/src/main/java/org/apache/blur/thrift/ThriftBlurShardServer.java
----------------------------------------------------------------------
diff --git a/blur-core/src/main/java/org/apache/blur/thrift/ThriftBlurShardServer.java b/blur-core/src/main/java/org/apache/blur/thrift/ThriftBlurShardServer.java
index 46bd8b0..0b4e290 100644
--- a/blur-core/src/main/java/org/apache/blur/thrift/ThriftBlurShardServer.java
+++ b/blur-core/src/main/java/org/apache/blur/thrift/ThriftBlurShardServer.java
@@ -316,7 +316,7 @@ public class ThriftBlurShardServer extends ThriftServer {
     StreamServer streamServer;
     int streamThreadCount = configuration.getInt(BLUR_STREAM_SERVER_THREADS, 100);
     if (streamThreadCount > 0) {
-      StreamProcessor streamProcessor = new StreamProcessor(indexServer, tmpPath, config);
+      StreamProcessor streamProcessor = new StreamProcessor(indexServer, tmpPath);
       streamServer = new StreamServer(0, streamThreadCount, streamProcessor);
       streamServer.start();
       configuration.setInt(BLUR_STREAM_SERVER_RUNNING_PORT, streamServer.getPort());