You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by mu...@apache.org on 2014/02/15 01:07:34 UTC

[01/15] Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Repository: incubator-phoenix
Updated Branches:
  refs/heads/master eebeed4b8 -> bbacf6e03


http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/recovery/TestPerRegionIndexWriteCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/recovery/TestPerRegionIndexWriteCache.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/recovery/TestPerRegionIndexWriteCache.java
new file mode 100644
index 0000000..0cde541
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/recovery/TestPerRegionIndexWriteCache.java
@@ -0,0 +1,168 @@
+/*
+ * 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.phoenix.hbase.index.write.recovery;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.write.recovery.PerRegionIndexWriteCache;
+
+public class TestPerRegionIndexWriteCache {
+
+  private static final byte[] row = Bytes.toBytes("row");
+  private static final byte[] family = Bytes.toBytes("family");
+  private static final byte[] qual = Bytes.toBytes("qual");
+  private static final byte[] val = Bytes.toBytes("val");
+
+  Put p = new Put(row);
+  Put p2 = new Put(Bytes.toBytes("other row"));
+  {
+    p.add(family, qual, val);
+    p2.add(family, qual, val);
+  }
+
+
+  HRegion r1 = new HRegion() {
+    @Override
+    public int hashCode() {
+      return 1;
+    }
+
+    @Override
+    public String toString() {
+      return "testRegion1";
+    }
+  };
+  HRegion r2 = new HRegion() {
+    @Override
+    public int hashCode() {
+      return 2;
+    }
+
+    @Override
+    public String toString() {
+      return "testRegion1";
+    }
+  };
+
+  @Test
+  public void testAddRemoveSingleRegion() {
+    PerRegionIndexWriteCache cache = new PerRegionIndexWriteCache();
+    HTableInterfaceReference t1 = new HTableInterfaceReference(new ImmutableBytesPtr(Bytes.toBytes("t1")));
+    List<Mutation> mutations = new ArrayList<Mutation>();
+    mutations.add(p);
+    cache.addEdits(r1, t1, mutations);
+    Multimap<HTableInterfaceReference, Mutation> edits = cache.getEdits(r1);
+    Set<Entry<HTableInterfaceReference, Collection<Mutation>>> entries = edits.asMap().entrySet();
+    assertEquals("Got more than one table in the the edit map!", 1, entries.size());
+    for (Entry<HTableInterfaceReference, Collection<Mutation>> entry : entries) {
+     //ensure that we are still storing a list here - otherwise it breaks the parallel writer implementation
+      final List<Mutation> stored = (List<Mutation>) entry.getValue();
+      assertEquals("Got an unexpected amount of mutations in the entry", 1, stored.size());
+      assertEquals("Got an unexpected mutation in the entry", p, stored.get(0));
+    }
+
+    // ensure that a second get doesn't have any more edits. This ensures that we don't keep
+    // references around to these edits and have a memory leak
+    assertNull("Got an entry for a region we removed", cache.getEdits(r1));
+  }
+
+  @Test
+  public void testMultipleAddsForSingleRegion() {
+    PerRegionIndexWriteCache cache = new PerRegionIndexWriteCache();
+    HTableInterfaceReference t1 =
+        new HTableInterfaceReference(new ImmutableBytesPtr(Bytes.toBytes("t1")));
+    List<Mutation> mutations = Lists.<Mutation> newArrayList(p);
+    cache.addEdits(r1, t1, mutations);
+
+    // add a second set
+    mutations = Lists.<Mutation> newArrayList(p2);
+    cache.addEdits(r1, t1, mutations);
+
+    Multimap<HTableInterfaceReference, Mutation> edits = cache.getEdits(r1);
+    Set<Entry<HTableInterfaceReference, Collection<Mutation>>> entries = edits.asMap().entrySet();
+    assertEquals("Got more than one table in the the edit map!", 1, entries.size());
+    for (Entry<HTableInterfaceReference, Collection<Mutation>> entry : entries) {
+      // ensure that we are still storing a list here - otherwise it breaks the parallel writer
+      // implementation
+      final List<Mutation> stored = (List<Mutation>) entry.getValue();
+      assertEquals("Got an unexpected amount of mutations in the entry", 2, stored.size());
+      assertEquals("Got an unexpected mutation in the entry", p, stored.get(0));
+      assertEquals("Got an unexpected mutation in the entry", p2, stored.get(1));
+    }
+  }
+
+  @Test
+  public void testMultipleRegions() {
+    PerRegionIndexWriteCache cache = new PerRegionIndexWriteCache();
+    HTableInterfaceReference t1 =
+        new HTableInterfaceReference(new ImmutableBytesPtr(Bytes.toBytes("t1")));
+    List<Mutation> mutations = Lists.<Mutation> newArrayList(p);
+    List<Mutation> m2 = Lists.<Mutation> newArrayList(p2);
+    // add each region
+    cache.addEdits(r1, t1, mutations);
+    cache.addEdits(r2, t1, m2);
+
+    // check region1
+    Multimap<HTableInterfaceReference, Mutation> edits = cache.getEdits(r1);
+    Set<Entry<HTableInterfaceReference, Collection<Mutation>>> entries = edits.asMap().entrySet();
+    assertEquals("Got more than one table in the the edit map!", 1, entries.size());
+    for (Entry<HTableInterfaceReference, Collection<Mutation>> entry : entries) {
+      // ensure that we are still storing a list here - otherwise it breaks the parallel writer
+      // implementation
+      final List<Mutation> stored = (List<Mutation>) entry.getValue();
+      assertEquals("Got an unexpected amount of mutations in the entry for region1", 1,
+        stored.size());
+      assertEquals("Got an unexpected mutation in the entry for region2", p, stored.get(0));
+    }
+
+    // check region2
+    edits = cache.getEdits(r2);
+    entries = edits.asMap().entrySet();
+    assertEquals("Got more than one table in the the edit map!", 1, entries.size());
+    for (Entry<HTableInterfaceReference, Collection<Mutation>> entry : entries) {
+      // ensure that we are still storing a list here - otherwise it breaks the parallel writer
+      // implementation
+      final List<Mutation> stored = (List<Mutation>) entry.getValue();
+      assertEquals("Got an unexpected amount of mutations in the entry for region2", 1,
+        stored.size());
+      assertEquals("Got an unexpected mutation in the entry for region2", p2, stored.get(0));
+    }
+
+
+    // ensure that a second get doesn't have any more edits. This ensures that we don't keep
+    // references around to these edits and have a memory leak
+    assertNull("Got an entry for a region we removed", cache.getEdits(r1));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java
index c314ce1..49438b8 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java
@@ -41,13 +41,13 @@ import org.apache.hadoop.hbase.util.VersionInfo;
 import org.junit.Test;
 
 import com.google.common.collect.Maps;
-import org.apache.hadoop.hbase.index.ValueGetter;
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.client.ClientKeyValueBuilder;
 import org.apache.phoenix.client.GenericKeyValueBuilder;
 import org.apache.phoenix.client.KeyValueBuilder;
 import org.apache.phoenix.end2end.index.IndexTestUtil;
+import org.apache.phoenix.hbase.index.ValueGetter;
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.BaseConnectionlessQueryTest;
 import org.apache.phoenix.schema.PTable;


[05/15] Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestCoveredColumnIndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestCoveredColumnIndexCodec.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestCoveredColumnIndexCodec.java
deleted file mode 100644
index 0f2ed37..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestCoveredColumnIndexCodec.java
+++ /dev/null
@@ -1,248 +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.hadoop.hbase.index.covered.example;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-import com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.index.covered.IndexCodec;
-import org.apache.hadoop.hbase.index.covered.IndexUpdate;
-import org.apache.hadoop.hbase.index.covered.LocalTableState;
-import org.apache.hadoop.hbase.index.covered.data.LocalHBaseState;
-import org.apache.hadoop.hbase.index.covered.example.CoveredColumnIndexCodec.ColumnEntry;
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-
-public class TestCoveredColumnIndexCodec {
-  private static final byte[] PK = new byte[] { 'a' };
-  private static final String FAMILY_STRING = "family";
-  private static final byte[] FAMILY = Bytes.toBytes(FAMILY_STRING);
-  private static final byte[] QUAL = Bytes.toBytes("qual");
-  private static final CoveredColumn COLUMN_REF = new CoveredColumn(FAMILY_STRING, QUAL);
-  private static final byte[] EMPTY_INDEX_KEY = CoveredColumnIndexCodec.composeRowKey(PK, 0,
-    Arrays.asList(toColumnEntry(new byte[0])));
-  private static final byte[] BLANK_INDEX_KEY = CoveredColumnIndexCodec.composeRowKey(PK, 0,
-    Collections.<ColumnEntry> emptyList());
-
-  private static ColumnEntry toColumnEntry(byte[] bytes) {
-    return new ColumnEntry(bytes, COLUMN_REF);
-  }
-
-  /**
-   * Convert between an index and a bunch of values
-   * @throws Exception
-   */
-  @Test
-  public void toFromIndexKey() throws Exception {
-    // start with empty values
-    byte[] indexKey = BLANK_INDEX_KEY;
-    List<byte[]> stored = CoveredColumnIndexCodec.getValues(indexKey);
-    assertEquals("Found some stored values in an index row key that wasn't created with values!",
-      0, stored.size());
-
-    // a single, empty value
-    indexKey = EMPTY_INDEX_KEY;
-    stored = CoveredColumnIndexCodec.getValues(indexKey);
-    assertEquals("Found some stored values in an index row key that wasn't created with values!",
-      1, stored.size());
-    assertEquals("Found a non-zero length value: " + Bytes.toString(stored.get(0)), 0,
-      stored.get(0).length);
-
-    // try with a couple values, some different lengths
-    byte[] v1 = new byte[] { 'a' };
-    byte[] v2 = new byte[] { 'b' };
-    byte[] v3 = Bytes.toBytes("v3");
-    int len = v1.length + v2.length + v3.length;
-    indexKey =
-        CoveredColumnIndexCodec.composeRowKey(PK, len,
-          Arrays.asList(toColumnEntry(v1), toColumnEntry(v2), toColumnEntry(v3)));
-    stored = CoveredColumnIndexCodec.getValues(indexKey);
-    assertEquals("Didn't find expected number of values in index key!", 3, stored.size());
-    assertTrue("First index keys don't match!", Bytes.equals(v1, stored.get(0)));
-    assertTrue("Second index keys don't match!", Bytes.equals(v2, stored.get(1)));
-    assertTrue("Third index keys don't match!", Bytes.equals(v3, stored.get(2)));
-  }
-
-  /**
-   * Ensure that we correctly can determine when a row key is empty (no values).
-   */
-  @Test
-  public void testCheckRowKeyForAllNulls() {
-    byte[] pk = new byte[] { 'a', 'b', 'z' };
-    // check positive cases first
-    byte[] result = EMPTY_INDEX_KEY;
-    assertTrue("Didn't correctly read single element as being null in row key",
-      CoveredColumnIndexCodec.checkRowKeyForAllNulls(result));
-    result =
-        CoveredColumnIndexCodec.composeRowKey(pk, 0,
-          Lists.newArrayList(toColumnEntry(new byte[0]), toColumnEntry(new byte[0])));
-    assertTrue("Didn't correctly read two elements as being null in row key",
-      CoveredColumnIndexCodec.checkRowKeyForAllNulls(result));
-
-    // check cases where it isn't null
-    result =
-        CoveredColumnIndexCodec.composeRowKey(pk, 2,
-          Arrays.asList(toColumnEntry(new byte[] { 1, 2 })));
-    assertFalse("Found a null key, when it wasn't!",
-      CoveredColumnIndexCodec.checkRowKeyForAllNulls(result));
-    result =
-        CoveredColumnIndexCodec.composeRowKey(pk, 2,
-          Arrays.asList(toColumnEntry(new byte[] { 1, 2 }), toColumnEntry(new byte[0])));
-    assertFalse("Found a null key, when it wasn't!",
-      CoveredColumnIndexCodec.checkRowKeyForAllNulls(result));
-  }
-
-  private static class SimpleTableState implements LocalHBaseState {
-
-    private Result r;
-
-    public SimpleTableState(Result r) {
-      this.r = r;
-    }
-
-    @Override
-    public Result getCurrentRowState(Mutation m, Collection<? extends ColumnReference> toCover)
-        throws IOException {
-      return r;
-    }
-
-  }
-
-  /**
-   * Test that we get back the correct index updates for a given column group
-   * @throws Exception on failure
-   */
-  @Test
-  public void testGeneratedIndexUpdates() throws Exception {
-    ColumnGroup group = new ColumnGroup("test-column-group");
-    group.add(COLUMN_REF);
-
-    final Result emptyState = new Result(Collections.<KeyValue> emptyList());
-    
-    // setup the state we expect for the codec
-    RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
-    Configuration conf = new Configuration(false);
-    Mockito.when(env.getConfiguration()).thenReturn(conf);
-    LocalHBaseState table = new SimpleTableState(emptyState);
-
-    // make a new codec on those kvs
-    CoveredColumnIndexCodec codec =
-        CoveredColumnIndexCodec.getCodecForTesting(Arrays.asList(group));
-
-    // start with a basic put that has some keyvalues
-    Put p = new Put(PK);
-    // setup the kvs to add
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
-    byte[] v1 = Bytes.toBytes("v1");
-    KeyValue kv = new KeyValue(PK, FAMILY, QUAL, 1, v1);
-    kvs.add(kv);
-    p.add(kv);
-    byte[] v2 = Bytes.toBytes("v2");
-    kv = new KeyValue(PK, Bytes.toBytes("family2"), QUAL, 1, v2);
-    kvs.add(kv);
-    p.add(kv);
-
-    // check the codec for deletes it should send
-    LocalTableState state = new LocalTableState(env, table, p);
-    Iterable<IndexUpdate> updates = codec.getIndexDeletes(state);
-    assertFalse("Found index updates without any existing kvs in table!", updates.iterator().next()
-        .isValid());
-
-    // get the updates with the pending update
-    state.setCurrentTimestamp(1);
-    state.addPendingUpdates(kvs);
-    updates = codec.getIndexUpserts(state);
-    assertTrue("Didn't find index updates for pending primary table update!", updates.iterator()
-        .hasNext());
-    for (IndexUpdate update : updates) {
-      assertTrue("Update marked as invalid, but should be a pending index write!", update.isValid());
-      Put m = (Put) update.getUpdate();
-      // should just be the single update for the column reference
-      byte[] expected =
-          CoveredColumnIndexCodec.composeRowKey(PK, v1.length, Arrays.asList(toColumnEntry(v1)));
-      assertArrayEquals("Didn't get expected index value", expected, m.getRow());
-    }
-
-    // then apply a delete
-    Delete d = new Delete(PK, 2);
-    // need to set the timestamp here, as would actually happen on the server, unlike what happens
-    // with puts, where the get the constructor specified timestamp for unspecified methods.
-    d.deleteFamily(FAMILY, 2);
-    // setup the next batch of 'current state', basically just ripping out the current state from
-    // the last round
-    table = new SimpleTableState(new Result(kvs));
-    state = new LocalTableState(env, table, d);
-    state.setCurrentTimestamp(2);
-    // check the cleanup of the current table, after the puts (mocking a 'next' update)
-    updates = codec.getIndexDeletes(state);
-    for (IndexUpdate update : updates) {
-      assertTrue("Didn't have any index cleanup, even though there is current state",
-        update.isValid());
-      Delete m = (Delete) update.getUpdate();
-      // should just be the single update for the column reference
-      byte[] expected =
-          CoveredColumnIndexCodec.composeRowKey(PK, v1.length, Arrays.asList(toColumnEntry(v1)));
-      assertArrayEquals("Didn't get expected index value", expected, m.getRow());
-    }
-    ensureNoUpdatesWhenCoveredByDelete(env, codec, kvs, d);
-
-    // now with the delete of the columns
-    d = new Delete(PK, 2);
-    d.deleteColumns(FAMILY, QUAL, 2);
-    ensureNoUpdatesWhenCoveredByDelete(env, codec, kvs, d);
-
-    // this delete needs to match timestamps exactly, by contract, to have any effect
-    d = new Delete(PK, 1);
-    d.deleteColumn(FAMILY, QUAL, 1);
-    ensureNoUpdatesWhenCoveredByDelete(env, codec, kvs, d);
-  }
-
-  private void ensureNoUpdatesWhenCoveredByDelete(RegionCoprocessorEnvironment env, IndexCodec codec, List<KeyValue> currentState,
-      Delete d) throws IOException {
-    LocalHBaseState table = new SimpleTableState(new Result(currentState));
-    LocalTableState state = new LocalTableState(env, table, d);
-    state.setCurrentTimestamp(d.getTimeStamp());
-    // now we shouldn't see anything when getting the index update
-    state.addPendingUpdates(d.getFamilyMap().get(FAMILY));
-    Iterable<IndexUpdate> updates = codec.getIndexUpserts(state);
-    for (IndexUpdate update : updates) {
-      assertFalse("Had some index updates, though it should have been covered by the delete",
-        update.isValid());
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestCoveredIndexSpecifierBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestCoveredIndexSpecifierBuilder.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestCoveredIndexSpecifierBuilder.java
deleted file mode 100644
index ae06f3b..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestCoveredIndexSpecifierBuilder.java
+++ /dev/null
@@ -1,72 +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.hadoop.hbase.index.covered.example;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-
-public class TestCoveredIndexSpecifierBuilder {
-  private static final String FAMILY = "FAMILY";
-  private static final String FAMILY2 = "FAMILY2";
-  private static final String INDEX_TABLE = "INDEX_TABLE";
-  private static final String INDEX_TABLE2 = "INDEX_TABLE2";
-
-
-  @Test
-  public void testSimpleSerialziationDeserialization() throws Exception {
-    byte[] indexed_qualifer = Bytes.toBytes("indexed_qual");
-
-    //setup the index 
-    CoveredColumnIndexSpecifierBuilder builder = new CoveredColumnIndexSpecifierBuilder();
-    ColumnGroup fam1 = new ColumnGroup(INDEX_TABLE);
-    // match a single family:qualifier pair
-    CoveredColumn col1 = new CoveredColumn(FAMILY, indexed_qualifer);
-    fam1.add(col1);
-    // matches the family2:* columns
-    CoveredColumn col2 = new CoveredColumn(FAMILY2, null);
-    fam1.add(col2);
-    builder.addIndexGroup(fam1);
-    ColumnGroup fam2 = new ColumnGroup(INDEX_TABLE2);
-    // match a single family2:qualifier pair
-    CoveredColumn col3 = new CoveredColumn(FAMILY2, indexed_qualifer);
-    fam2.add(col3);
-    builder.addIndexGroup(fam2);
-    
-    Configuration conf = new Configuration(false);
-    //convert the map that HTableDescriptor gets into the conf the coprocessor receives
-    Map<String, String> map = builder.convertToMap();
-    for(Entry<String, String> entry: map.entrySet()){
-      conf.set(entry.getKey(), entry.getValue());
-    }
-
-    List<ColumnGroup> columns = CoveredColumnIndexSpecifierBuilder.getColumns(conf);
-    assertEquals("Didn't deserialize the expected number of column groups", 2, columns.size());
-    ColumnGroup group = columns.get(0);
-    assertEquals("Didn't deserialize expected column in first group", col1, group.getColumnForTesting(0));
-    assertEquals("Didn't deserialize expected column in first group", col2, group.getColumnForTesting(1));
-    group = columns.get(1);
-    assertEquals("Didn't deserialize expected column in second group", col3, group.getColumnForTesting(0));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestEndToEndCoveredIndexing.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestEndToEndCoveredIndexing.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestEndToEndCoveredIndexing.java
deleted file mode 100644
index 8107ff2..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestEndToEndCoveredIndexing.java
+++ /dev/null
@@ -1,877 +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.hadoop.hbase.index.covered.example;
-
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.commons.lang.ArrayUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-import org.apache.hadoop.hbase.index.IndexTestingUtils;
-import org.apache.hadoop.hbase.index.Indexer;
-import org.apache.hadoop.hbase.index.TableName;
-
-/**
- * Test Covered Column indexing in an 'end-to-end' manner on a minicluster. This covers cases where
- * we manage custom timestamped updates that arrive in and out of order as well as just using the
- * generically timestamped updates.
- */
-public class TestEndToEndCoveredIndexing {
-  private static final Log LOG = LogFactory.getLog(TestEndToEndCoveredIndexing.class);
-  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-  private static final String FAM_STRING = "FAMILY";
-  private static final byte[] FAM = Bytes.toBytes(FAM_STRING);
-  private static final String FAM2_STRING = "FAMILY2";
-  private static final byte[] FAM2 = Bytes.toBytes(FAM2_STRING);
-  private static final byte[] EMPTY_BYTES = new byte[0];
-  private static final byte[] indexed_qualifer = Bytes.toBytes("indexed_qual");
-  private static final byte[] regular_qualifer = Bytes.toBytes("reg_qual");
-  private static final byte[] row1 = Bytes.toBytes("row1");
-  private static final byte[] value1 = Bytes.toBytes("val1");
-  private static final byte[] value2 = Bytes.toBytes("val2");
-  private static final byte[] value3 = Bytes.toBytes("val3");
-  // match a single family:qualifier pair
-  private static final CoveredColumn col1 = new CoveredColumn(FAM_STRING, indexed_qualifer);
-  // matches the family2:* columns
-  private static final CoveredColumn col2 = new CoveredColumn(FAM2_STRING, null);
-  private static final CoveredColumn col3 = new CoveredColumn(FAM2_STRING, indexed_qualifer);
-  
-  @Rule
-  public TableName TestTable = new TableName();
-  
-  private ColumnGroup fam1;
-  private ColumnGroup fam2;
-
-  // setup a couple of index columns
-  private void setupColumns() {
-    fam1 = new ColumnGroup(getIndexTableName());
-    fam2 = new ColumnGroup(getIndexTableName() + "2");
-    // values are [col1][col2_1]...[col2_n]
-    fam1.add(col1);
-    fam1.add(col2);
-    // value is [col2]
-    fam2.add(col3);
-  }
-
-  private String getIndexTableName() {
-    return Bytes.toString(TestTable.getTableName()) + "_index";
-  }
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    Configuration conf = UTIL.getConfiguration();
-    IndexTestingUtils.setupConfig(conf);
-    // disable version checking, so we can test against whatever version of HBase happens to be
-    // installed (right now, its generally going to be SNAPSHOT versions).
-    conf.setBoolean(Indexer.CHECK_VERSION_CONF_KEY, false);
-    UTIL.startMiniCluster();
-  }
-
-  @AfterClass
-  public static void teardownCluster() throws Exception {
-    UTIL.shutdownMiniCluster();
-  }
-
-  @Before
-  public void setup() throws Exception {
-    setupColumns();
-  }
-
-  /**
-   * Test that a bunch of puts with a single timestamp across all the puts builds and inserts index
-   * entries as expected
-   * @throws Exception on failure
-   */
-  @Test
-  public void testSimpleTimestampedUpdates() throws Exception {
-    HTable primary = createSetupTables(fam1);
-
-    // do a put to the primary table
-    Put p = new Put(row1);
-    long ts = 10;
-    p.add(FAM, indexed_qualifer, ts, value1);
-    p.add(FAM, regular_qualifer, ts, value2);
-    primary.put(p);
-    primary.flushCommits();
-
-    // read the index for the expected values
-    HTable index1 = new HTable(UTIL.getConfiguration(), getIndexTableName());
-
-    // build the expected kvs
-    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
-    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
-    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts, pairs);
-
-    // verify that the index matches
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value1);
-
-    // cleanup
-    closeAndCleanupTables(primary, index1);
-  }
-
-  /**
-   * Test that the multiple timestamps in a single put build the correct index updates.
-   * @throws Exception on failure
-   */
-  @Test
-  public void testMultipleTimestampsInSinglePut() throws Exception {
-    HTable primary = createSetupTables(fam1);
-
-    // do a put to the primary table
-    Put p = new Put(row1);
-    long ts1 = 10;
-    long ts2 = 11;
-    p.add(FAM, indexed_qualifer, ts1, value1);
-    p.add(FAM, regular_qualifer, ts1, value2);
-    // our group indexes all columns in the this family, so any qualifier here is ok
-    p.add(FAM2, regular_qualifer, ts2, value3);
-    primary.put(p);
-    primary.flushCommits();
-
-    // read the index for the expected values
-    HTable index1 = new HTable(UTIL.getConfiguration(), getIndexTableName());
-
-    // build the expected kvs
-    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
-    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
-
-    // check the first entry at ts1
-    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts1, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts1, value1);
-
-    // check the second entry at ts2
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(value3, col2));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts2, value1);
-
-    // cleanup
-    closeAndCleanupTables(primary, index1);
-  }
-
-  /**
-   * Test that we make updates to multiple {@link ColumnGroup}s across a single put/delete 
-   * @throws Exception on failure
-   */
-  @Test
-  public void testMultipleConcurrentGroupsUpdated() throws Exception {
-    HTable primary = createSetupTables(fam1, fam2);
-
-    // do a put to the primary table
-    Put p = new Put(row1);
-    long ts = 10;
-    p.add(FAM, indexed_qualifer, ts, value1);
-    p.add(FAM, regular_qualifer, ts, value2);
-    p.add(FAM2, indexed_qualifer, ts, value3);
-    primary.put(p);
-    primary.flushCommits();
-
-    // read the index for the expected values
-    HTable index1 = new HTable(UTIL.getConfiguration(), fam1.getTable());
-    HTable index2 = new HTable(UTIL.getConfiguration(), fam2.getTable());
-
-    // build the expected kvs
-    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
-    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(value3, col2));
-    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value1);
-
-    // and check the second index as well
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value3, col3));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index2, expected, ts, value3);
-
-    // cleanup
-    closeAndCleanupTables(primary, index1, index2);
-  }
-
-  /**
-   * HBase has a 'fun' property wherein you can completely clobber an existing row if you make a
-   * {@link Put} at the exact same dimension (row, cf, cq, ts) as an existing row. The old row
-   * disappears and the new value (since the rest of the row is the same) completely subsumes it.
-   * This test ensures that we remove the old entry and put a new entry in its place.
-   * @throws Exception on failure
-   */
-  @Test
-  public void testOverwritingPutsCorrectlyGetIndexed() throws Exception {
-    HTable primary = createSetupTables(fam1);
-
-    // do a put to the primary table
-    Put p = new Put(row1);
-    long ts = 10;
-    p.add(FAM, indexed_qualifer, ts, value1);
-    p.add(FAM, regular_qualifer, ts, value2);
-    primary.put(p);
-    primary.flushCommits();
-
-    // read the index for the expected values
-    HTable index1 = new HTable(UTIL.getConfiguration(), fam1.getTable());
-
-    // build the expected kvs
-    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
-    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
-
-    // check the first entry at ts
-    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value1);
-
-    // now overwrite the put in the primary table with a new value
-    p = new Put(row1);
-    p.add(FAM, indexed_qualifer, ts, value3);
-    primary.put(p);
-    primary.flushCommits();
-
-    pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
-    pairs.add(new Pair<byte[], CoveredColumn>(value3, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
-
-    // check the first entry at ts
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value3);
-    // and verify that a scan at the first entry returns nothing (ignore the updated row)
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, Collections.<KeyValue> emptyList(), ts,
-      value1, value2);
-
-    // cleanup
-    closeAndCleanupTables(primary, index1);
-  }
-  
-  @Test
-  public void testSimpleDeletes() throws Exception {
-    HTable primary = createSetupTables(fam1);
-
-    // do a simple Put
-    long ts = 10;
-    Put p = new Put(row1);
-    p.add(FAM, indexed_qualifer, ts, value1);
-    p.add(FAM, regular_qualifer, ts, value2);
-    primary.put(p);
-    primary.flushCommits();
-
-    Delete d = new Delete(row1);
-    primary.delete(d);
-
-    HTable index = new HTable(UTIL.getConfiguration(), fam1.getTable());
-    List<KeyValue> expected = Collections.<KeyValue> emptyList();
-    // scan over all time should cause the delete to be covered
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index, expected, 0, Long.MAX_VALUE, value1,
-      HConstants.EMPTY_END_ROW);
-
-    // scan at the older timestamp should still show the older value
-    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
-    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index, expected, ts, value1);
-
-    // cleanup
-    closeAndCleanupTables(index, primary);
-  }
-
-  /**
-   * If we don't have any updates to make to the index, we don't take a lock on the WAL. However, we
-   * need to make sure that we don't try to unlock the WAL on write time when we don't write
-   * anything, since that will cause an java.lang.IllegalMonitorStateException
-   * @throws Exception on failure
-   */
-  @Test
-  public void testDeletesWithoutPreviousState() throws Exception {
-    HTable primary = createSetupTables(fam1);
-
-    // do a delete on the primary table (no data, so no index updates...hopefully).
-    long ts = 10;
-    Delete d = new Delete(row1);
-    primary.delete(d);
-
-    HTable index1 = new HTable(UTIL.getConfiguration(), fam1.getTable());
-    List<KeyValue> expected = Collections.<KeyValue> emptyList();
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value1);
-
-    // a delete of a specific family/column should also not show any index updates
-    d = new Delete(row1);
-    d.deleteColumn(FAM, indexed_qualifer);
-    primary.delete(d);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value1);
-
-    // also just a family marker should have the same effect
-    d = new Delete(row1);
-    d.deleteFamily(FAM);
-    primary.delete(d);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value1);
-
-    // also just a family marker should have the same effect
-    d = new Delete(row1);
-    d.deleteColumns(FAM, indexed_qualifer);
-    primary.delete(d);
-    primary.flushCommits();
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value1);
-
-    // cleanup
-    closeAndCleanupTables(primary, index1);
-  }
-
-  /**
-   * Similar to the {@link #testMultipleTimestampsInSinglePut()}, this check the same with deletes
-   * @throws Exception on failure
-   */
-  @Test
-  public void testMultipleTimestampsInSingleDelete() throws Exception {
-    HTable primary = createSetupTables(fam1);
-
-    // do a put to the primary table
-    Put p = new Put(row1);
-    long ts1 = 10, ts2 = 11, ts3 = 12;
-    p.add(FAM, indexed_qualifer, ts1, value1);
-    // our group indexes all columns in the this family, so any qualifier here is ok
-    p.add(FAM2, regular_qualifer, ts2, value3);
-    primary.put(p);
-    primary.flushCommits();
-
-    // check to make sure everything we expect is there
-    HTable index1 = new HTable(UTIL.getConfiguration(), fam1.getTable());
-
-    // ts1, we just have v1
-    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
-    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts1, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts1, value1);
-
-    // at ts2, don't have the above anymore
-    pairs.clear();
-    expected = Collections.emptyList();
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts2, ts2 + 1, value1, value1);
-    // but we do have the new entry at ts2
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(value3, col2));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts2, value1);
-
-    // now build up a delete with a couple different timestamps
-    Delete d = new Delete(row1);
-    // these deletes have to match the exact ts since we are doing an exact match (deleteColumn).
-    d.deleteColumn(FAM, indexed_qualifer, ts1);
-    // since this doesn't match exactly, we actually shouldn't see a change in table state
-    d.deleteColumn(FAM2, regular_qualifer, ts3);
-    primary.delete(d);
-
-    // at ts1, we should have the put covered exactly by the delete and into the entire future
-    expected = Collections.emptyList();
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts1, Long.MAX_VALUE, value1,
-      value1);
-
-    // at ts2, we should just see value3
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(value3, col2));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts2, value1);
-
-    // the later delete is a point delete, so we shouldn't see any change at ts3
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts2, ts3, value1,
-      HConstants.EMPTY_END_ROW);
-
-    // cleanup
-    closeAndCleanupTables(primary, index1);
-  }
-
-  /**
-   * Covering deletes (via {@link Delete#deleteColumns}) cover everything back in time from the
-   * given time. If its modifying the latest state, we don't need to do anything but add deletes. If
-   * its modifying back in time state, we need to just fix up the surrounding elements as anything
-   * else ahead of it will be fixed up by later updates.
-   * <p>
-   * similar to {@link #testMultipleTimestampsInSingleDelete()}, but with covering deletes.
-   * @throws Exception on failure
-   */
-  @Test
-  public void testDeleteColumnsInThePast() throws Exception {
-    HTable primary = createSetupTables(fam1);
-
-    // do a put to the primary table
-    Put p = new Put(row1);
-    long ts1 = 10, ts2 = 11, ts3 = 12;
-    p.add(FAM, indexed_qualifer, ts1, value1);
-    p.add(FAM2, regular_qualifer, ts2, value3);
-    primary.put(p);
-    primary.flushCommits();
-
-    // now build up a delete with a couple different timestamps
-    Delete d = new Delete(row1);
-    // these deletes don't need to match the exact ts because they cover everything earlier
-    d.deleteColumns(FAM, indexed_qualifer, ts2);
-    d.deleteColumns(FAM2, regular_qualifer, ts3);
-    primary.delete(d);
-
-    // read the index for the expected values
-    HTable index1 = new HTable(UTIL.getConfiguration(), fam1.getTable());
-
-    // build the expected kvs
-    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
-    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
-
-    // check the first entry at ts1
-    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts1, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts1, value1);
-
-    // delete at ts2 changes what the put would insert
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(value3, col2));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts2, value1);
-
-    // final delete clears out everything
-    expected = Collections.emptyList();
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts3, value1);
-
-    // cleanup
-    closeAndCleanupTables(primary, index1);
-  }
-  
-  /**
-   * If the client is using custom timestamps is possible that the updates come out-of-order (i.e.
-   * update to ts 10 comes after the update to ts 12). In the case, we need to be sure that the
-   * index is correctly updated when the out of order put arrives.
-   * @throws Exception
-   */
-  @Test
-  public void testOutOfOrderUpdates() throws Exception {
-    HTable primary = createSetupTables(fam1);
-
-    // do a put to the primary table
-    Put p = new Put(row1);
-    long ts = 12;
-    p.add(FAM, indexed_qualifer, ts, value1);
-    primary.put(p);
-    primary.flushCommits();
-
-    // read the index for the expected values
-    HTable index1 = new HTable(UTIL.getConfiguration(), fam1.getTable());
-
-    // build the expected kvs
-    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
-    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
-
-    // check the first entry at ts
-    List<KeyValue> expectedTs1 = CoveredColumnIndexCodec
-        .getIndexKeyValueForTesting(row1, ts, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expectedTs1, ts, value1);
-
-    // now make a put back in time
-    long ts2 = ts - 2;
-    p = new Put(row1);
-    p.add(FAM, indexed_qualifer, ts2, value2);
-    primary.put(p);
-    primary.flushCommits();
-
-    pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
-    pairs.add(new Pair<byte[], CoveredColumn>(value2, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
-
-    // check to make sure the back in time entry exists
-    List<KeyValue> expectedTs2 = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2,
-      pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expectedTs2, ts2, value2);
-    // then it should be gone at the newer ts (because it deletes itself)
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, Collections.<KeyValue> emptyList(), ts2,
-      ts + 1, value2, HConstants.EMPTY_END_ROW);
-
-    // but that the original index entry is still visible at ts, just fine
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expectedTs1, ts, value1);
-
-    // cleanup
-    closeAndCleanupTables(primary, index1);
-  }
-
-  /**
-   * Its possible (i.e. from a fast, frequently writing client) that they put more than the
-   * 'visible' number of versions in a row before a client make a put 'back in time' on that row. If
-   * we don't scan the current table properly, we won't see an index update for that 'back in time'
-   * update since the usual lookup will only see the regular number of versions. This ability to see
-   * back in time depends on running HBase 0.94.9
-   * @throws Exception on failure
-   */
-  @Test
-  public void testExceedVersionsOutOfOrderPut() throws Exception {
-    // setup the index
-    HTable primary = createSetupTables(fam2);
-
-    // do a put to the primary table
-    Put p = new Put(row1);
-    long ts1 = 1, ts2 = 2, ts3 = 3, ts4 = 4, ts5 = 5;
-    byte[] value4 = Bytes.toBytes("val4");
-    byte[] value5 = Bytes.toBytes("val5");
-    p.add(FAM2, indexed_qualifer, ts1, value1);
-    primary.put(p);
-    primary.flushCommits();
-
-    p = new Put(row1);
-    p.add(FAM2, indexed_qualifer, ts3, value3);
-    primary.put(p);
-    primary.flushCommits();
-
-    p = new Put(row1);
-    p.add(FAM2, indexed_qualifer, ts4, value4);
-    primary.put(p);
-    primary.flushCommits();
-
-    p = new Put(row1);
-    p.add(FAM2, indexed_qualifer, ts5, value5);
-    primary.put(p);
-    primary.flushCommits();
-
-    // read the index for the expected values
-    HTable index = new HTable(UTIL.getConfiguration(), fam2.getTable());
-
-    // do a raw scan of everything in the table
-    if (LOG.isDebugEnabled()) {
-      // the whole table, all the keys
-      Scan s = new Scan();
-      s.setRaw(true);
-      ResultScanner scanner = index.getScanner(s);
-      for (Result r : scanner) {
-        LOG.debug("Found row:" + r);
-      }
-      scanner.close();
-    }
-
-    /*
-     * now we have definitely exceeded the number of versions visible to a usual client of the
-     * primary table, so we should try doing a put 'back in time' an make sure that has the correct
-     * index values and cleanup
-     */
-    p = new Put(row1);
-    p.add(FAM2, indexed_qualifer, ts2, value2);
-    primary.put(p);
-    primary.flushCommits();
-
-    // // read the index for the expected values
-    // HTable index = new HTable(UTIL.getConfiguration(), fam2.getTable());
-    //
-    // do a raw scan of everything in the table
-    if (LOG.isDebugEnabled()) {
-      // the whole table, all the keys
-      Scan s = new Scan();
-      s.setRaw(true);
-      ResultScanner scanner = index.getScanner(s);
-      for (Result r : scanner) {
-        LOG.debug("Found row:" + r);
-      }
-      scanner.close();
-    }
-
-    // build the expected kvs
-    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
-    pairs.add(new Pair<byte[], CoveredColumn>(value1, col3));
-
-    // check the value1 should be present at the earliest timestamp
-    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts1, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index, expected, ts1, value1, value2);
-
-    // and value1 should be removed at ts2 (even though it came later)
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index, Collections.<KeyValue> emptyList(), ts1,
-      ts2 + 1, value1, value2); // timestamp + 1 since its exclusive end timestamp
-
-    // late added column should be there just fine at its timestamp
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value2, col3));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index, expected, ts2, value2);
-
-    // and check that the late entry also removes its self at the next timestamp up
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index, Collections.<KeyValue> emptyList(), ts3,
-      value2, value3);
-
-    // then we should have the rest of the inserts at their appropriate timestamps. Everything else
-    // should be exactly the same, except we shouldn't see ts0 anymore at ts2
-
-    // check the third entry
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value3, col3));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts3, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index, expected, ts3, value3);
-
-    // check the fourth entry
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value4, col3));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts4, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index, expected, ts4, value4);
-
-    // check the first entry at ts4
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value2, col3));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index, expected, ts2, value2);
-    // verify that we remove the entry, even though its too far 'back in time'
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index, Collections.<KeyValue> emptyList(), ts3,
-      value4);
-
-    // cleanup
-    closeAndCleanupTables(primary, index);
-  }
-
-  /**
-   * Similar to {@link #testExceedVersionsOutOfOrderPut()}, but mingles deletes and puts.
-   * @throws Exception on failure
-   */
-  @Test
-  public void testExceedVersionsOutOfOrderUpdates() throws Exception {
-    HTable primary = createSetupTables(fam1);
-
-    // setup the data to store
-    long ts1 = 1, ts2 = 2, ts3 = 3, ts4 = 4, ts5 = 5, ts6 = 6;
-    byte[] value4 = Bytes.toBytes("val4"), value5 = Bytes.toBytes("val5"), value6 =
-        Bytes.toBytes("val6");
-    // values for the other column to index
-    byte[] v1_1 = ArrayUtils.addAll(value1, Bytes.toBytes("_otherCol")), v3_1 =
-        ArrayUtils.addAll(value3, Bytes.toBytes("_otherCol")), v5_1 =
-        ArrayUtils.addAll(value5, Bytes.toBytes("_otherCol")), v6_1 =
-        ArrayUtils.addAll(value6, Bytes.toBytes("_otherCol"));
-
-    // make some puts to the primary table
-    Put p = new Put(row1);
-    p.add(FAM, indexed_qualifer, ts1, value1);
-    p.add(FAM2, indexed_qualifer, ts1, v1_1);
-    primary.put(p);
-    primary.flushCommits();
-
-    p = new Put(row1);
-    p.add(FAM, indexed_qualifer, ts3, value3);
-    p.add(FAM2, indexed_qualifer, ts3, v3_1);
-    primary.put(p);
-    primary.flushCommits();
-
-    p = new Put(row1);
-    p.add(FAM, indexed_qualifer, ts5, value5);
-    p.add(FAM2, indexed_qualifer, ts5, v5_1);
-    primary.put(p);
-    primary.flushCommits();
-
-    p = new Put(row1);
-    p.add(FAM, indexed_qualifer, ts6, value6);
-    p.add(FAM2, indexed_qualifer, ts6, v6_1);
-    primary.put(p);
-    primary.flushCommits();
-
-    /*
-     * now we have definitely exceeded the number of versions visible to a usual client of the
-     * primary table, so we should try doing a put 'back in time' an make sure that has the correct
-     * index values and cleanup
-     */
-    p = new Put(row1);
-    p.add(FAM, indexed_qualifer, ts2, value2);
-    primary.put(p);
-    primary.flushCommits();
-
-    // read the index for the expected values
-    HTable index1 = new HTable(UTIL.getConfiguration(), fam1.getTable());
-
-    // do a raw scan of everything in the table
-    if (LOG.isDebugEnabled()) {
-      Scan s = new Scan();
-      s.setRaw(true);
-      ResultScanner scanner = index1.getScanner(s);
-      for (Result r : scanner) {
-        LOG.debug("Found row:" + r);
-      }
-      scanner.close();
-    }
-
-    // build the expected kvs
-    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
-    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(v1_1, col2));
-
-    // check the value1 should be present at the earliest timestamp
-    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts1, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts1, value1, value2);
-
-    // and value1 should be removed at ts2 (even though it came later)
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, Collections.<KeyValue> emptyList(), ts1,
-      ts2 + 1, value1, value2); // timestamp + 1 since its exclusive end timestamp
-
-    // late added column should be there just fine at its timestamp
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value2, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(v1_1, col2));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts2, value2);
-
-    // and check that the late entry also removes its self at the next timestamp up
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, Collections.<KeyValue> emptyList(), ts3,
-      value2, value3);
-
-    // -----------------------------------------------
-    // Check Delete intermingled
-    // -----------------------------------------------
-
-    // verify that the old row is there
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value3, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(v3_1, col2));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts3, pairs);
-    // scan from the start key forward (should only include [value3][v3_3])
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts3, expected.get(0).getKey(),
-      value4);
-
-    // then do a delete of just one of the indexed columns. This should insert a delete for all just
-    // the single value, then a put & a later corresponding in the past for the new value
-    Delete d = new Delete(row1);
-    d.deleteColumn(FAM2, indexed_qualifer, ts3);
-    primary.delete(d);
-
-    // we shouldn't find that entry, but we should find [value3][v1_1] since that is next entry back
-    // in time from the current
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value3, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(v1_1, col2));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts3, pairs);
-    // it should be re-written at 3
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts3, value3, value4);
-
-    // but we shouldn't find it at ts5 since it should be covered again
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, Collections.<KeyValue> emptyList(), ts5,
-      value3, value4);
-
-    // now remove all the older columns in FAM2 at 4
-    d = new Delete(row1);
-    d.deleteColumns(FAM2, indexed_qualifer, ts4);
-    primary.delete(d);
-
-    // we shouldn't find that entry, but we should find [value3][null] since deleteColumns removes
-    // all the entries for that column
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value3, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts4, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts4, value3, value4);
-
-    // same as above, but now do it at 3 (on earlier)
-    d = new Delete(row1);
-    d.deleteColumns(FAM2, indexed_qualifer, ts3);
-    primary.delete(d);
-
-    // we shouldn't find that entry, but we should find [value3][null] since deleteColumns removes
-    // all the entries for that column
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value3, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts3, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts3, value3, value4);
-
-    // -----------------------------------------------
-    // then we should have the rest of the inserts at their appropriate timestamps. Everything else
-    // should be exactly the same, except we shouldn't see ts0 anymore at ts2
-    // -----------------------------------------------
-
-    // check the entry at 5
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value5, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(v5_1, col2));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts5, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts5, value5);
-
-    // check the entry at 6
-    pairs.clear();
-    pairs.add(new Pair<byte[], CoveredColumn>(value6, col1));
-    pairs.add(new Pair<byte[], CoveredColumn>(v6_1, col2));
-    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts6, pairs);
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts6, value5);
-
-    // cleanup
-    closeAndCleanupTables(primary, index1);
-  }
-
-  /**
-   * Create the primary table (to which you should write), setup properly for indexing the given
-   * {@link ColumnGroup}s. Also creates the necessary index tables to match the passes groups.
-   * @param groups {@link ColumnGroup}s to index, creating one index table per column group.
-   * @return reference to the primary table
-   * @throws IOException if there is an issue communicating with HBase
-   */
-  private HTable createSetupTables(ColumnGroup... groups) throws IOException {
-    HBaseAdmin admin = UTIL.getHBaseAdmin();
-    // setup the index
-    CoveredColumnIndexSpecifierBuilder builder = new CoveredColumnIndexSpecifierBuilder();
-    for (ColumnGroup group : groups) {
-      builder.addIndexGroup(group);
-      // create the index tables
-      CoveredColumnIndexer.createIndexTable(admin, group.getTable());
-    }
-
-    // setup the primary table
-    String indexedTableName = Bytes.toString(TestTable.getTableName());
-    HTableDescriptor pTable = new HTableDescriptor(indexedTableName);
-    pTable.addFamily(new HColumnDescriptor(FAM));
-    pTable.addFamily(new HColumnDescriptor(FAM2));
-    builder.build(pTable);
-
-    // create the primary table
-    admin.createTable(pTable);
-    HTable primary = new HTable(UTIL.getConfiguration(), indexedTableName);
-    primary.setAutoFlush(false);
-    return primary;
-  }
-
-  private void closeAndCleanupTables(HTable... tables) throws IOException {
-    if (tables == null) {
-      return;
-    }
-
-    for (HTable table : tables) {
-      table.close();
-      UTIL.deleteTable(table.getTableName());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestEndtoEndIndexingWithCompression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestEndtoEndIndexingWithCompression.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestEndtoEndIndexingWithCompression.java
deleted file mode 100644
index 570c1d9..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestEndtoEndIndexingWithCompression.java
+++ /dev/null
@@ -1,49 +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.hadoop.hbase.index.covered.example;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec;
-import org.apache.hadoop.hbase.regionserver.wal.WALEditCodec;
-import org.junit.BeforeClass;
-
-import org.apache.hadoop.hbase.index.IndexTestingUtils;
-import org.apache.hadoop.hbase.index.Indexer;
-
-/**
- * Test secondary indexing from an end-to-end perspective (client to server to index table).
- */
-public class TestEndtoEndIndexingWithCompression extends TestEndToEndCoveredIndexing {
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    //add our codec and enable WAL compression
-    Configuration conf = UTIL.getConfiguration();
-    IndexTestingUtils.setupConfig(conf);
-    // disable version checking, so we can test against whatever version of HBase happens to be
-    // installed (right now, its generally going to be SNAPSHOT versions).
-    conf.setBoolean(Indexer.CHECK_VERSION_CONF_KEY, false);
-    conf.set(WALEditCodec.WAL_EDIT_CODEC_CLASS_KEY,
-    IndexedWALEditCodec.class.getName());
-    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
-    
-    //start the mini-cluster
-    UTIL.startMiniCluster();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestFailWithoutRetries.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestFailWithoutRetries.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestFailWithoutRetries.java
deleted file mode 100644
index 099755a..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestFailWithoutRetries.java
+++ /dev/null
@@ -1,145 +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.hadoop.hbase.index.covered.example;
-
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-import org.apache.hadoop.hbase.index.IndexTestingUtils;
-import org.apache.hadoop.hbase.index.TableName;
-import org.apache.hadoop.hbase.index.covered.IndexUpdate;
-import org.apache.hadoop.hbase.index.covered.TableState;
-import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
-import org.apache.phoenix.index.BaseIndexCodec;
-
-/**
- * If {@link DoNotRetryIOException} is not subclassed correctly (with the {@link String}
- * constructor), {@link MultiResponse#readFields(java.io.DataInput)} will not correctly deserialize
- * the exception, and just return <tt>null</tt> to the client, which then just goes and retries.
- */
-public class TestFailWithoutRetries {
-
-  private static final Log LOG = LogFactory.getLog(TestFailWithoutRetries.class);
-  @Rule
-  public TableName table = new TableName();
-
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-  private String getIndexTableName() {
-    return Bytes.toString(table.getTableName()) + "_index";
-  }
-
-  public static class FailingTestCodec extends BaseIndexCodec {
-
-    @Override
-    public Iterable<IndexUpdate> getIndexDeletes(TableState state) throws IOException {
-      throw new RuntimeException("Intentionally failing deletes for "
-          + TestFailWithoutRetries.class.getName());
-    }
-
-    @Override
-    public Iterable<IndexUpdate> getIndexUpserts(TableState state) throws IOException {
-      throw new RuntimeException("Intentionally failing upserts for "
-          + TestFailWithoutRetries.class.getName());
-    }
-
-  }
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    // setup and verify the config
-    Configuration conf = UTIL.getConfiguration();
-    IndexTestingUtils.setupConfig(conf);
-    IndexManagementUtil.ensureMutableIndexingCorrectlyConfigured(conf);
-    // start the cluster
-    UTIL.startMiniCluster();
-  }
-
-  @AfterClass
-  public static void teardownCluster() throws Exception {
-    UTIL.shutdownMiniCluster();
-  }
-
-  /**
-   * If this test times out, then we didn't fail quickly enough. {@link Indexer} maybe isn't
-   * rethrowing the exception correctly?
-   * <p>
-   * We use a custom codec to enforce the thrown exception.
-   * @throws Exception
-   */
-  @Test(timeout = 300000)
-  public void testQuickFailure() throws Exception {
-    // incorrectly setup indexing for the primary table - target index table doesn't exist, which
-    // should quickly return to the client
-    byte[] family = Bytes.toBytes("family");
-    ColumnGroup fam1 = new ColumnGroup(getIndexTableName());
-    // values are [col1]
-    fam1.add(new CoveredColumn(family, CoveredColumn.ALL_QUALIFIERS));
-    CoveredColumnIndexSpecifierBuilder builder = new CoveredColumnIndexSpecifierBuilder();
-    // add the index family
-    builder.addIndexGroup(fam1);
-    // usually, we would create the index table here, but we don't for the sake of the test.
-
-    // setup the primary table
-    String primaryTable = Bytes.toString(table.getTableName());
-    HTableDescriptor pTable = new HTableDescriptor(primaryTable);
-    pTable.addFamily(new HColumnDescriptor(family));
-    // override the codec so we can use our test one
-    builder.build(pTable, FailingTestCodec.class);
-
-    // create the primary table
-    HBaseAdmin admin = UTIL.getHBaseAdmin();
-    admin.createTable(pTable);
-    Configuration conf = new Configuration(UTIL.getConfiguration());
-    // up the number of retries/wait time to make it obvious that we are failing with retries here
-    conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 20);
-    conf.setLong(HConstants.HBASE_CLIENT_PAUSE, 1000);
-    HTable primary = new HTable(conf, primaryTable);
-    primary.setAutoFlush(false, true);
-
-    // do a simple put that should be indexed
-    Put p = new Put(Bytes.toBytes("row"));
-    p.add(family, null, Bytes.toBytes("value"));
-    primary.put(p);
-    try {
-      primary.flushCommits();
-      fail("Shouldn't have gotten a successful write to the primary table");
-    } catch (RetriesExhaustedWithDetailsException e) {
-      LOG.info("Correclty got a failure of the put!");
-    }
-    primary.close();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/filter/TestApplyAndFilterDeletesFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/filter/TestApplyAndFilterDeletesFilter.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/filter/TestApplyAndFilterDeletesFilter.java
deleted file mode 100644
index a84477a..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/filter/TestApplyAndFilterDeletesFilter.java
+++ /dev/null
@@ -1,210 +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.hadoop.hbase.index.covered.filter;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-/**
- * Test filter to ensure that it correctly handles KVs of different types correctly
- */
-public class TestApplyAndFilterDeletesFilter {
-
-  private static final Set<ImmutableBytesPtr> EMPTY_SET = Collections
-      .<ImmutableBytesPtr> emptySet();
-  private byte[] row = Bytes.toBytes("row");
-  private byte[] family = Bytes.toBytes("family");
-  private byte[] qualifier = Bytes.toBytes("qualifier");
-  private byte[] value = Bytes.toBytes("value");
-  private long ts = 10;
-
-  @Test
-  public void testDeletesAreNotReturned() {
-    KeyValue kv = createKvForType(Type.Delete);
-    ApplyAndFilterDeletesFilter filter = new ApplyAndFilterDeletesFilter(EMPTY_SET);
-    assertEquals("Didn't skip point delete!", ReturnCode.SKIP, filter.filterKeyValue(kv));
-
-    filter.reset();
-    kv = createKvForType(Type.DeleteColumn);
-    assertEquals("Didn't skip from column delete!", ReturnCode.SKIP, filter.filterKeyValue(kv));
-
-    filter.reset();
-    kv = createKvForType(Type.DeleteFamily);
-    assertEquals("Didn't skip from family delete!", ReturnCode.SKIP, filter.filterKeyValue(kv));
-  }
-
-  /**
-   * Hinting with this filter is a little convoluted as we binary search the list of families to
-   * attempt to find the right one to seek.
-   */
-  @Test
-  public void testHintCorrectlyToNextFamily() {
-    // start with doing a family delete, so we will seek to the next column
-    KeyValue kv = createKvForType(Type.DeleteFamily);
-    ApplyAndFilterDeletesFilter filter = new ApplyAndFilterDeletesFilter(EMPTY_SET);
-    assertEquals(ReturnCode.SKIP, filter.filterKeyValue(kv));
-    KeyValue next = createKvForType(Type.Put);
-    // make sure the hint is our attempt at the end key, because we have no more families to seek
-    assertEquals("Didn't get a hint from a family delete", ReturnCode.SEEK_NEXT_USING_HINT,
-      filter.filterKeyValue(next));
-    assertEquals("Didn't get END_KEY with no families to match", KeyValue.LOWESTKEY,
-      filter.getNextKeyHint(next));
-
-    // check for a family that comes before our family, so we always seek to the end as well
-    filter = new ApplyAndFilterDeletesFilter(asSet(Bytes.toBytes("afamily")));
-    assertEquals(ReturnCode.SKIP, filter.filterKeyValue(kv));
-    // make sure the hint is our attempt at the end key, because we have no more families to seek
-    assertEquals("Didn't get a hint from a family delete", ReturnCode.SEEK_NEXT_USING_HINT,
-      filter.filterKeyValue(next));
-    assertEquals("Didn't get END_KEY with no families to match", KeyValue.LOWESTKEY,
-      filter.getNextKeyHint(next));
-
-    // check that we seek to the correct family that comes after our family
-    byte[] laterFamily = Bytes.toBytes("zfamily");
-    filter = new ApplyAndFilterDeletesFilter(asSet(laterFamily));
-    assertEquals(ReturnCode.SKIP, filter.filterKeyValue(kv));
-    KeyValue expected = KeyValue.createFirstOnRow(kv.getRow(), laterFamily, new byte[0]);
-    assertEquals("Didn't get a hint from a family delete", ReturnCode.SEEK_NEXT_USING_HINT,
-      filter.filterKeyValue(next));
-    assertEquals("Didn't get correct next key with a next family", expected,
-      filter.getNextKeyHint(next));
-  }
-
-  /**
-   * Point deletes should only cover the exact entry they are tied to. Earlier puts should always
-   * show up.
-   */
-  @Test
-  public void testCoveringPointDelete() {
-    // start with doing a family delete, so we will seek to the next column
-    KeyValue kv = createKvForType(Type.Delete);
-    ApplyAndFilterDeletesFilter filter = new ApplyAndFilterDeletesFilter(EMPTY_SET);
-    filter.filterKeyValue(kv);
-    KeyValue put = createKvForType(Type.Put);
-    assertEquals("Didn't filter out put with same timestamp!", ReturnCode.SKIP,
-      filter.filterKeyValue(put));
-    // we should filter out the exact same put again, which could occur with the kvs all kept in the
-    // same memstore
-    assertEquals("Didn't filter out put with same timestamp on second call!", ReturnCode.SKIP,
-      filter.filterKeyValue(put));
-
-    // ensure then that we don't filter out a put with an earlier timestamp (though everything else
-    // matches)
-    put = createKvForType(Type.Put, ts - 1);
-    assertEquals("Didn't accept put that has an earlier ts than the covering delete!",
-      ReturnCode.INCLUDE, filter.filterKeyValue(put));
-  }
-
-  private KeyValue createKvForType(Type t) {
-    return createKvForType(t, this.ts);
-  }
-
-  private KeyValue createKvForType(Type t, long timestamp) {
-    return new KeyValue(row, family, qualifier, 0, qualifier.length, timestamp, t, value, 0,
-        value.length);
-  }
-
-  /**
-   * Test that when we do a column delete at a given timestamp that we delete the entire column.
-   * @throws Exception
-   */
-  @Test
-  public void testCoverForDeleteColumn() throws Exception {
-    ApplyAndFilterDeletesFilter filter = new ApplyAndFilterDeletesFilter(EMPTY_SET);
-    KeyValue dc = createKvForType(Type.DeleteColumn, 11);
-    KeyValue put = createKvForType(Type.Put, 10);
-    assertEquals("Didn't filter out delete column.", ReturnCode.SKIP, filter.filterKeyValue(dc));
-    assertEquals("Didn't get a seek hint for the deleted column", ReturnCode.SEEK_NEXT_USING_HINT,
-      filter.filterKeyValue(put));
-    // seek past the given put
-    KeyValue seek = filter.getNextKeyHint(put);
-    assertTrue("Seeked key wasn't past the expected put - didn't skip the column",
-      KeyValue.COMPARATOR.compare(seek, put) > 0);
-  }
-
-  /**
-   * DeleteFamily markers should delete everything from that timestamp backwards, but not hide
-   * anything forwards
-   */
-  @Test
-  public void testDeleteFamilyCorrectlyCoversColumns() {
-    ApplyAndFilterDeletesFilter filter = new ApplyAndFilterDeletesFilter(EMPTY_SET);
-    KeyValue df = createKvForType(Type.DeleteFamily, 11);
-    KeyValue put = createKvForType(Type.Put, 12);
-
-    assertEquals("Didn't filter out delete family", ReturnCode.SKIP, filter.filterKeyValue(df));
-    assertEquals("Filtered out put with newer TS than delete family", ReturnCode.INCLUDE,
-      filter.filterKeyValue(put));
-
-    // older kv shouldn't be visible
-    put = createKvForType(Type.Put, 10);
-    assertEquals("Didn't filter out older put, covered by DeleteFamily marker",
-      ReturnCode.SEEK_NEXT_USING_HINT, filter.filterKeyValue(put));
-
-    // next seek should be past the families
-    assertEquals(KeyValue.LOWESTKEY, filter.getNextKeyHint(put));
-  }
-
-  /**
-   * Test that we don't cover other columns when we have a delete column.
-   */
-  @Test
-  public void testDeleteColumnCorrectlyCoversColumns() {
-    ApplyAndFilterDeletesFilter filter = new ApplyAndFilterDeletesFilter(EMPTY_SET);
-    KeyValue d = createKvForType(Type.DeleteColumn, 12);
-    byte[] qual2 = Bytes.add(qualifier, Bytes.toBytes("-other"));
-    KeyValue put =
-        new KeyValue(row, family, qual2, 0, qual2.length, 11, Type.Put, value, 0,
-            value.length);
-
-    assertEquals("Didn't filter out delete column", ReturnCode.SKIP, filter.filterKeyValue(d));
-    // different column put should still be visible
-    assertEquals("Filtered out put with different column than the delete", ReturnCode.INCLUDE,
-      filter.filterKeyValue(put));
-
-    // set a delete family, but in the past
-    d = createKvForType(Type.DeleteFamily, 10);
-    assertEquals("Didn't filter out delete column", ReturnCode.SKIP, filter.filterKeyValue(d));
-    // add back in the original delete column
-    d = createKvForType(Type.DeleteColumn, 11);
-    assertEquals("Didn't filter out delete column", ReturnCode.SKIP, filter.filterKeyValue(d));
-    // onto a different family, so that must be visible too
-    assertEquals("Filtered out put with different column than the delete", ReturnCode.INCLUDE,
-      filter.filterKeyValue(put));
-  }
-
-  private static Set<ImmutableBytesPtr> asSet(byte[]... strings) {
-    Set<ImmutableBytesPtr> set = new HashSet<ImmutableBytesPtr>();
-    for (byte[] s : strings) {
-      set.add(new ImmutableBytesPtr(s));
-    }
-    return set;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/filter/TestFamilyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/filter/TestFamilyOnlyFilter.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/filter/TestFamilyOnlyFilter.java
deleted file mode 100644
index 4412d75..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/filter/TestFamilyOnlyFilter.java
+++ /dev/null
@@ -1,105 +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.hadoop.hbase.index.covered.filter;
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-
-/**
- * Test that the family only filter only allows a single family through
- */
-public class TestFamilyOnlyFilter {
-
-  byte[] row = new byte[] { 'a' };
-  byte[] qual = new byte[] { 'b' };
-  byte[] val = Bytes.toBytes("val");
-
-  @Test
-  public void testPassesFirstFamily() {
-    byte[] fam = Bytes.toBytes("fam");
-    byte[] fam2 = Bytes.toBytes("fam2");
-
-    FamilyOnlyFilter filter = new FamilyOnlyFilter(fam);
-
-    KeyValue kv = new KeyValue(row, fam, qual, 10, val);
-    ReturnCode code = filter.filterKeyValue(kv);
-    assertEquals("Didn't pass matching family!", ReturnCode.INCLUDE, code);
-
-    kv = new KeyValue(row, fam2, qual, 10, val);
-    code = filter.filterKeyValue(kv);
-    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
-  }
-
-  @Test
-  public void testPassesTargetFamilyAsNonFirstFamily() {
-    byte[] fam = Bytes.toBytes("fam");
-    byte[] fam2 = Bytes.toBytes("fam2");
-    byte[] fam3 = Bytes.toBytes("way_after_family");
-
-    FamilyOnlyFilter filter = new FamilyOnlyFilter(fam2);
-
-    KeyValue kv = new KeyValue(row, fam, qual, 10, val);
-
-    ReturnCode code = filter.filterKeyValue(kv);
-    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
-
-    kv = new KeyValue(row, fam2, qual, 10, val);
-    code = filter.filterKeyValue(kv);
-    assertEquals("Didn't pass matching family", ReturnCode.INCLUDE, code);
-
-    kv = new KeyValue(row, fam3, qual, 10, val);
-    code = filter.filterKeyValue(kv);
-    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
-  }
-
-  @Test
-  public void testResetFilter() {
-    byte[] fam = Bytes.toBytes("fam");
-    byte[] fam2 = Bytes.toBytes("fam2");
-    byte[] fam3 = Bytes.toBytes("way_after_family");
-
-    FamilyOnlyFilter filter = new FamilyOnlyFilter(fam2);
-
-    KeyValue kv = new KeyValue(row, fam, qual, 10, val);
-
-    ReturnCode code = filter.filterKeyValue(kv);
-    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
-
-    KeyValue accept = new KeyValue(row, fam2, qual, 10, val);
-    code = filter.filterKeyValue(accept);
-    assertEquals("Didn't pass matching family", ReturnCode.INCLUDE, code);
-
-    kv = new KeyValue(row, fam3, qual, 10, val);
-    code = filter.filterKeyValue(kv);
-    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
-
-    // we shouldn't match the family again - everything after a switched family should be ignored
-    code = filter.filterKeyValue(accept);
-    assertEquals("Should have skipped a 'matching' family if it arrives out of order",
-      ReturnCode.SKIP, code);
-
-    // reset the filter and we should accept it again
-    filter.reset();
-    code = filter.filterKeyValue(accept);
-    assertEquals("Didn't pass matching family after reset", ReturnCode.INCLUDE, code);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/filter/TestNewerTimestampFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/filter/TestNewerTimestampFilter.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/filter/TestNewerTimestampFilter.java
deleted file mode 100644
index 6e61733..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/filter/TestNewerTimestampFilter.java
+++ /dev/null
@@ -1,47 +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.hadoop.hbase.index.covered.filter;
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-
-public class TestNewerTimestampFilter {
-  byte[] row = new byte[] { 'a' };
-  byte[] fam = Bytes.toBytes("family");
-  byte[] qual = new byte[] { 'b' };
-  byte[] val = Bytes.toBytes("val");
-
-  @Test
-  public void testOnlyAllowsOlderTimestamps() {
-    long ts = 100;
-    NewerTimestampFilter filter = new NewerTimestampFilter(ts);
-
-    KeyValue kv = new KeyValue(row, fam, qual, ts, val);
-    assertEquals("Didn't accept kv with matching ts", ReturnCode.INCLUDE, filter.filterKeyValue(kv));
-
-    kv = new KeyValue(row, fam, qual, ts + 1, val);
-    assertEquals("Didn't skip kv with greater ts", ReturnCode.SKIP, filter.filterKeyValue(kv));
-
-    kv = new KeyValue(row, fam, qual, ts - 1, val);
-    assertEquals("Didn't accept kv with lower ts", ReturnCode.INCLUDE, filter.filterKeyValue(kv));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/update/TestIndexUpdateManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/update/TestIndexUpdateManager.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/update/TestIndexUpdateManager.java
deleted file mode 100644
index 633ffd5..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/update/TestIndexUpdateManager.java
+++ /dev/null
@@ -1,140 +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.hadoop.hbase.index.covered.update;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.junit.Test;
-
-import org.apache.hadoop.hbase.index.covered.update.IndexUpdateManager;
-
-public class TestIndexUpdateManager {
-
-  private static final byte[] row = Bytes.toBytes("row");
-  private static final String TABLE_NAME = "table";
-  private static final byte[] table = Bytes.toBytes(TABLE_NAME);
-
-  @Test
-  public void testMutationComparator() throws Exception {
-    IndexUpdateManager manager = new IndexUpdateManager();
-    Comparator<Mutation> comparator = manager.COMPARATOR;
-    Put p = new Put(row, 10);
-    // lexigraphically earlier should sort earlier
-    Put p1 = new Put(Bytes.toBytes("ro"), 10);
-    assertTrue("lexigraphically later sorting first, should be earlier first.",
-      comparator.compare(p, p1) > 0);
-    p1 = new Put(Bytes.toBytes("row1"), 10);
-    assertTrue("lexigraphically later sorting first, should be earlier first.",
-      comparator.compare(p1, p) > 0);
-
-    // larger ts sorts before smaller, for the same row
-    p1 = new Put(row, 11);
-    assertTrue("Smaller timestamp sorting first, should be larger first.",
-      comparator.compare(p, p1) > 0);
-    // still true, even for deletes
-    Delete d = new Delete(row, 11);
-    assertTrue("Smaller timestamp sorting first, should be larger first.",
-      comparator.compare(p, d) > 0);
-
-    // for the same row, t1, the delete should sort earlier
-    d = new Delete(row, 10);
-    assertTrue("Delete doesn't sort before put, for the same row and ts",
-      comparator.compare(p, d) > 0);
-
-    // but for different rows, we still respect the row sorting.
-    d = new Delete(Bytes.toBytes("row1"), 10);
-    assertTrue("Delete doesn't sort before put, for the same row and ts",
-      comparator.compare(p, d) < 0);
-  }
-
-  /**
-   * When making updates we need to cancel out {@link Delete} and {@link Put}s for the same row.
-   * @throws Exception on failure
-   */
-  @Test
-  public void testCancelingUpdates() throws Exception {
-    IndexUpdateManager manager = new IndexUpdateManager();
-
-    long ts1 = 10, ts2 = 11;
-    // at different timestamps, so both should be retained
-    Delete d = new Delete(row, ts1);
-    Put p = new Put(row, ts2);
-    manager.addIndexUpdate(table, d);
-    manager.addIndexUpdate(table, p);
-    List<Mutation> pending = new ArrayList<Mutation>();
-    pending.add(p);
-    pending.add(d);
-    validate(manager, pending);
-
-    // add a delete that should cancel out the put, leading to only one delete remaining
-    Delete d2 = new Delete(row, ts2);
-    manager.addIndexUpdate(table, d2);
-    pending.add(d);
-    validate(manager, pending);
-
-    // double-deletes of the same row only retain the existing one, which was already canceled out
-    // above
-    Delete d3 = new Delete(row, ts2);
-    manager.addIndexUpdate(table, d3);
-    pending.add(d);
-    validate(manager, pending);
-
-    // if there is just a put and a delete at the same ts, no pending updates should be returned
-    manager = new IndexUpdateManager();
-    manager.addIndexUpdate(table, d2);
-    manager.addIndexUpdate(table, p);
-    validate(manager, Collections.<Mutation> emptyList());
-
-    // different row insertions can be tricky too, if you don't get the base cases right
-    manager = new IndexUpdateManager();
-    manager.addIndexUpdate(table, p);
-    // this row definitely sorts after the current row
-    byte[] row1 = Bytes.toBytes("row1");
-    Put p1 = new Put(row1, ts1);
-    manager.addIndexUpdate(table, p1);
-    // this delete should completely cover the given put and both should be removed
-    Delete d4 = new Delete(row1, ts1);
-    manager.addIndexUpdate(table, d4);
-    pending.clear();
-    pending.add(p);
-    validate(manager, pending);
-  }
-
-  private void validate(IndexUpdateManager manager, List<Mutation> pending) {
-    for (Pair<Mutation, byte[]> entry : manager.toMap()) {
-      assertEquals("Table name didn't match for stored entry!", table, entry.getSecond());
-      Mutation m = pending.remove(0);
-      // test with == to match the exact entries, Mutation.equals just checks the row
-      assertTrue(
-        "Didn't get the expected mutation! Expected: " + m + ", but got: " + entry.getFirst(),
-        m == entry.getFirst());
-    }
-    assertTrue("Missing pending updates: " + pending, pending.isEmpty());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/parallel/TestThreadPoolBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/parallel/TestThreadPoolBuilder.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/parallel/TestThreadPoolBuilder.java
deleted file mode 100644
index ce1a539..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/parallel/TestThreadPoolBuilder.java
+++ /dev/null
@@ -1,63 +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.hadoop.hbase.index.parallel;
-
-import static org.junit.Assert.*;
-
-import org.apache.hadoop.conf.Configuration;
-import org.junit.Rule;
-import org.junit.Test;
-
-import org.apache.hadoop.hbase.index.TableName;
-
-public class TestThreadPoolBuilder {
-
-  @Rule
-  public TableName name = new TableName();
-
-  @Test
-  public void testCoreThreadTimeoutNonZero() {
-    Configuration conf = new Configuration(false);
-    String key = name.getTableNameString()+"-key";
-    ThreadPoolBuilder builder = new ThreadPoolBuilder(name.getTableNameString(), conf);
-    assertTrue("core threads not set, but failed return", builder.getKeepAliveTime() > 0);
-    // set an negative value
-    builder.setCoreTimeout(key, -1);
-    assertTrue("core threads not set, but failed return", builder.getKeepAliveTime() > 0);
-    // set a positive value
-    builder.setCoreTimeout(key, 1234);
-    assertEquals("core threads not set, but failed return", 1234, builder.getKeepAliveTime());
-    // set an empty value
-    builder.setCoreTimeout(key);
-    assertTrue("core threads not set, but failed return", builder.getKeepAliveTime() > 0);
-  }
-  
-  @Test
-  public void testMaxThreadsNonZero() {
-    Configuration conf = new Configuration(false);
-    String key = name.getTableNameString()+"-key";
-    ThreadPoolBuilder builder = new ThreadPoolBuilder(name.getTableNameString(), conf);
-    assertTrue("core threads not set, but failed return", builder.getMaxThreads() > 0);
-    // set an negative value
-    builder.setMaxThread(key, -1);
-    assertTrue("core threads not set, but failed return", builder.getMaxThreads() > 0);
-    // set a positive value
-    builder.setMaxThread(key, 1234);
-    assertEquals("core threads not set, but failed return", 1234, builder.getMaxThreads());
-  }
-}
\ No newline at end of file


[15/15] git commit: Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Posted by mu...@apache.org.
Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38


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

Branch: refs/heads/master
Commit: bbacf6e03045bab692aecbea00a4b63ee7593e73
Parents: eebeed4
Author: Mujtaba <mu...@apache.org>
Authored: Fri Feb 14 16:04:38 2014 -0800
Committer: Mujtaba <mu...@apache.org>
Committed: Fri Feb 14 16:04:38 2014 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/index/CapturingAbortable.java  |  66 --
 .../hbase/index/IndexLogRollSynchronizer.java   | 123 ---
 .../org/apache/hadoop/hbase/index/Indexer.java  | 704 ---------------
 .../apache/hadoop/hbase/index/ValueGetter.java  |  36 -
 .../hbase/index/builder/BaseIndexBuilder.java   |  97 --
 .../hbase/index/builder/IndexBuildManager.java  | 214 -----
 .../hbase/index/builder/IndexBuilder.java       | 142 ---
 .../builder/IndexBuildingFailureException.java  |  48 -
 .../hadoop/hbase/index/covered/Batch.java       |  60 --
 .../hbase/index/covered/CoveredColumns.java     |  49 --
 .../covered/CoveredColumnsIndexBuilder.java     | 488 ----------
 .../hadoop/hbase/index/covered/IndexCodec.java  | 110 ---
 .../hadoop/hbase/index/covered/IndexUpdate.java |  77 --
 .../hbase/index/covered/KeyValueStore.java      |  33 -
 .../hbase/index/covered/LocalTableState.java    | 242 -----
 .../hadoop/hbase/index/covered/TableState.java  | 116 ---
 .../hbase/index/covered/data/IndexMemStore.java | 331 -------
 .../index/covered/data/LazyValueGetter.java     |  87 --
 .../index/covered/data/LocalHBaseState.java     |  47 -
 .../hbase/index/covered/data/LocalTable.java    |  72 --
 .../index/covered/example/ColumnGroup.java      | 112 ---
 .../index/covered/example/CoveredColumn.java    | 107 ---
 .../example/CoveredColumnIndexCodec.java        | 367 --------
 .../CoveredColumnIndexSpecifierBuilder.java     | 184 ----
 .../covered/example/CoveredColumnIndexer.java   | 164 ----
 .../filter/ApplyAndFilterDeletesFilter.java     | 308 -------
 ...olumnTrackingNextLargestTimestampFilter.java |  72 --
 .../index/covered/filter/FamilyOnlyFilter.java  |  80 --
 .../covered/filter/MaxTimestampFilter.java      |  74 --
 .../covered/filter/NewerTimestampFilter.java    |  55 --
 .../index/covered/update/ColumnReference.java   | 166 ----
 .../index/covered/update/ColumnTracker.java     | 115 ---
 .../covered/update/IndexUpdateManager.java      | 238 -----
 .../covered/update/IndexedColumnGroup.java      |  28 -
 .../index/covered/update/SortedCollection.java  | 128 ---
 .../index/exception/IndexWriteException.java    |  43 -
 .../MultiIndexWriteFailureException.java        |  44 -
 .../SingleIndexWriteFailureException.java       |  61 --
 .../hbase/index/parallel/BaseTaskRunner.java    | 129 ---
 .../hbase/index/parallel/EarlyExitFailure.java  |  34 -
 .../index/parallel/QuickFailingTaskRunner.java  |  49 --
 .../hadoop/hbase/index/parallel/Task.java       |  40 -
 .../hadoop/hbase/index/parallel/TaskBatch.java  |  74 --
 .../hadoop/hbase/index/parallel/TaskRunner.java |  60 --
 .../hbase/index/parallel/ThreadPoolBuilder.java |  90 --
 .../hbase/index/parallel/ThreadPoolManager.java | 146 ---
 .../parallel/WaitForCompletionTaskRunner.java   |  51 --
 .../hbase/index/scanner/EmptyScanner.java       |  50 --
 .../index/scanner/FilteredKeyValueScanner.java  | 162 ----
 .../hadoop/hbase/index/scanner/Scanner.java     |  55 --
 .../hbase/index/scanner/ScannerBuilder.java     | 165 ----
 .../hbase/index/table/CachingHTableFactory.java | 110 ---
 .../index/table/CoprocessorHTableFactory.java   |  68 --
 .../hadoop/hbase/index/table/HTableFactory.java |  32 -
 .../index/table/HTableInterfaceReference.java   |  64 --
 .../hbase/index/util/ImmutableBytesPtr.java     | 111 ---
 .../hbase/index/util/IndexManagementUtil.java   | 244 -----
 .../hadoop/hbase/index/wal/IndexedKeyValue.java | 173 ----
 .../hadoop/hbase/index/wal/KeyValueCodec.java   |  97 --
 .../hbase/index/write/IndexCommitter.java       |  37 -
 .../hbase/index/write/IndexFailurePolicy.java   |  44 -
 .../hadoop/hbase/index/write/IndexWriter.java   | 224 -----
 .../hbase/index/write/IndexWriterUtils.java     |  72 --
 .../index/write/KillServerOnFailurePolicy.java  |  80 --
 .../write/ParallelWriterIndexCommitter.java     | 210 -----
 .../recovery/PerRegionIndexWriteCache.java      |  63 --
 .../recovery/StoreFailuresInCachePolicy.java    |  84 --
 .../TrackingParallelWriterIndexCommitter.java   | 226 -----
 .../hbase/regionserver/wal/IndexedWALEdit.java  |   2 +-
 .../regionserver/wal/IndexedWALEditCodec.java   |   4 +-
 .../org/apache/phoenix/cache/GlobalCache.java   |   2 +-
 .../org/apache/phoenix/cache/HashCache.java     |   2 +-
 .../org/apache/phoenix/cache/TenantCache.java   |   2 +-
 .../apache/phoenix/cache/TenantCacheImpl.java   |   2 +-
 .../phoenix/cache/aggcache/SpillManager.java    |   2 +-
 .../apache/phoenix/cache/aggcache/SpillMap.java |   3 +-
 .../cache/aggcache/SpillableGroupByCache.java   |   2 +-
 .../phoenix/client/GenericKeyValueBuilder.java  |   2 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |   2 +-
 .../apache/phoenix/compile/QueryCompiler.java   |   2 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   2 +-
 .../GroupedAggregateRegionObserver.java         |   2 +-
 .../coprocessor/HashJoinRegionScanner.java      |   2 +-
 .../coprocessor/MetaDataEndpointImpl.java       |   4 +-
 .../coprocessor/ServerCachingEndpointImpl.java  |   2 +-
 .../phoenix/exception/SQLExceptionCode.java     |   2 +-
 .../apache/phoenix/execute/HashJoinPlan.java    |   2 +-
 .../apache/phoenix/execute/MutationState.java   |   2 +-
 .../phoenix/expression/InListExpression.java    |   2 +-
 .../aggregator/BaseDecimalStddevAggregator.java |   2 +-
 .../aggregator/BaseStddevAggregator.java        |   2 +-
 .../DistinctValueWithCountClientAggregator.java |   2 +-
 .../DistinctValueWithCountServerAggregator.java |   2 +-
 .../filter/MultiCQKeyValueComparisonFilter.java |   2 +-
 .../phoenix/hbase/index/CapturingAbortable.java |  66 ++
 .../hbase/index/IndexLogRollSynchronizer.java   | 123 +++
 .../org/apache/phoenix/hbase/index/Indexer.java | 704 +++++++++++++++
 .../apache/phoenix/hbase/index/ValueGetter.java |  36 +
 .../hbase/index/builder/BaseIndexBuilder.java   |  97 ++
 .../hbase/index/builder/IndexBuildManager.java  | 214 +++++
 .../hbase/index/builder/IndexBuilder.java       | 142 +++
 .../builder/IndexBuildingFailureException.java  |  48 +
 .../phoenix/hbase/index/covered/Batch.java      |  60 ++
 .../hbase/index/covered/CoveredColumns.java     |  49 ++
 .../covered/CoveredColumnsIndexBuilder.java     | 488 ++++++++++
 .../phoenix/hbase/index/covered/IndexCodec.java | 110 +++
 .../hbase/index/covered/IndexUpdate.java        |  77 ++
 .../hbase/index/covered/KeyValueStore.java      |  33 +
 .../hbase/index/covered/LocalTableState.java    | 242 +++++
 .../phoenix/hbase/index/covered/TableState.java | 116 +++
 .../hbase/index/covered/data/IndexMemStore.java | 331 +++++++
 .../index/covered/data/LazyValueGetter.java     |  87 ++
 .../index/covered/data/LocalHBaseState.java     |  47 +
 .../hbase/index/covered/data/LocalTable.java    |  72 ++
 .../index/covered/example/ColumnGroup.java      | 112 +++
 .../index/covered/example/CoveredColumn.java    | 107 +++
 .../example/CoveredColumnIndexCodec.java        | 367 ++++++++
 .../CoveredColumnIndexSpecifierBuilder.java     | 184 ++++
 .../covered/example/CoveredColumnIndexer.java   | 164 ++++
 .../filter/ApplyAndFilterDeletesFilter.java     | 308 +++++++
 ...olumnTrackingNextLargestTimestampFilter.java |  72 ++
 .../index/covered/filter/FamilyOnlyFilter.java  |  80 ++
 .../covered/filter/MaxTimestampFilter.java      |  74 ++
 .../covered/filter/NewerTimestampFilter.java    |  55 ++
 .../index/covered/update/ColumnReference.java   | 166 ++++
 .../index/covered/update/ColumnTracker.java     | 115 +++
 .../covered/update/IndexUpdateManager.java      | 239 +++++
 .../covered/update/IndexedColumnGroup.java      |  28 +
 .../index/covered/update/SortedCollection.java  | 128 +++
 .../index/exception/IndexWriteException.java    |  43 +
 .../MultiIndexWriteFailureException.java        |  44 +
 .../SingleIndexWriteFailureException.java       |  61 ++
 .../hbase/index/parallel/BaseTaskRunner.java    | 129 +++
 .../hbase/index/parallel/EarlyExitFailure.java  |  34 +
 .../index/parallel/QuickFailingTaskRunner.java  |  49 ++
 .../phoenix/hbase/index/parallel/Task.java      |  40 +
 .../phoenix/hbase/index/parallel/TaskBatch.java |  74 ++
 .../hbase/index/parallel/TaskRunner.java        |  60 ++
 .../hbase/index/parallel/ThreadPoolBuilder.java |  90 ++
 .../hbase/index/parallel/ThreadPoolManager.java | 146 +++
 .../parallel/WaitForCompletionTaskRunner.java   |  51 ++
 .../hbase/index/scanner/EmptyScanner.java       |  50 ++
 .../index/scanner/FilteredKeyValueScanner.java  | 162 ++++
 .../phoenix/hbase/index/scanner/Scanner.java    |  55 ++
 .../hbase/index/scanner/ScannerBuilder.java     | 165 ++++
 .../hbase/index/table/CachingHTableFactory.java | 110 +++
 .../index/table/CoprocessorHTableFactory.java   |  68 ++
 .../hbase/index/table/HTableFactory.java        |  32 +
 .../index/table/HTableInterfaceReference.java   |  64 ++
 .../hbase/index/util/ImmutableBytesPtr.java     | 111 +++
 .../hbase/index/util/IndexManagementUtil.java   | 244 +++++
 .../hbase/index/wal/IndexedKeyValue.java        | 173 ++++
 .../phoenix/hbase/index/wal/KeyValueCodec.java  |  97 ++
 .../hbase/index/write/IndexCommitter.java       |  37 +
 .../hbase/index/write/IndexFailurePolicy.java   |  45 +
 .../phoenix/hbase/index/write/IndexWriter.java  | 224 +++++
 .../hbase/index/write/IndexWriterUtils.java     |  72 ++
 .../index/write/KillServerOnFailurePolicy.java  |  81 ++
 .../write/ParallelWriterIndexCommitter.java     | 210 +++++
 .../recovery/PerRegionIndexWriteCache.java      |  64 ++
 .../recovery/StoreFailuresInCachePolicy.java    |  84 ++
 .../TrackingParallelWriterIndexCommitter.java   | 226 +++++
 .../apache/phoenix/index/BaseIndexCodec.java    |   2 +-
 .../apache/phoenix/index/IndexMaintainer.java   |   6 +-
 .../phoenix/index/PhoenixIndexBuilder.java      |   4 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java |  16 +-
 .../index/PhoenixIndexFailurePolicy.java        |   4 +-
 .../apache/phoenix/join/HashCacheFactory.java   |   2 +-
 .../org/apache/phoenix/join/HashJoinInfo.java   |   2 +-
 .../query/ConnectionQueryServicesImpl.java      |   4 +-
 .../apache/phoenix/query/QueryConstants.java    |   2 +-
 .../java/org/apache/phoenix/schema/PName.java   |   2 +-
 .../org/apache/phoenix/schema/PNameImpl.java    |   2 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |   2 +-
 .../java/org/apache/phoenix/util/ByteUtil.java  |   2 +-
 .../java/org/apache/phoenix/util/IndexUtil.java |   6 +-
 .../org/apache/phoenix/util/SchemaUtil.java     |   2 +-
 .../java/org/apache/phoenix/util/TupleUtil.java |   2 +-
 .../hadoop/hbase/index/IndexTestingUtils.java   |  94 --
 .../hadoop/hbase/index/StubAbortable.java       |  41 -
 .../apache/hadoop/hbase/index/TableName.java    |  45 -
 .../TestFailForUnsupportedHBaseVersions.java    | 155 ----
 .../covered/CoveredIndexCodecForTesting.java    |  71 --
 .../hbase/index/covered/TestCoveredColumns.java |  45 -
 .../TestEndToEndCoveredColumnsIndexBuilder.java | 339 -------
 .../index/covered/TestLocalTableState.java      | 196 -----
 .../index/covered/data/TestIndexMemStore.java   |  93 --
 .../covered/example/TestColumnTracker.java      |  61 --
 .../example/TestCoveredColumnIndexCodec.java    | 248 ------
 .../TestCoveredIndexSpecifierBuilder.java       |  72 --
 .../example/TestEndToEndCoveredIndexing.java    | 877 ------------------
 .../TestEndtoEndIndexingWithCompression.java    |  49 --
 .../covered/example/TestFailWithoutRetries.java | 145 ---
 .../filter/TestApplyAndFilterDeletesFilter.java | 210 -----
 .../covered/filter/TestFamilyOnlyFilter.java    | 105 ---
 .../filter/TestNewerTimestampFilter.java        |  47 -
 .../covered/update/TestIndexUpdateManager.java  | 140 ---
 .../index/parallel/TestThreadPoolBuilder.java   |  63 --
 .../index/parallel/TestThreadPoolManager.java   |  93 --
 .../index/util/TestIndexManagementUtil.java     |  66 --
 .../hbase/index/write/FakeTableFactory.java     |  51 --
 .../index/write/TestCachingHTableFactory.java   |  58 --
 .../hbase/index/write/TestIndexWriter.java      | 284 ------
 .../index/write/TestParalleIndexWriter.java     | 117 ---
 .../write/TestParalleWriterIndexCommitter.java  | 117 ---
 .../index/write/TestWALRecoveryCaching.java     | 368 --------
 .../recovery/TestPerRegionIndexWriteCache.java  | 168 ----
 .../wal/TestReadWriteKeyValuesWithCodec.java    |   4 +-
 ...ALReplayWithIndexWritesAndCompressedWAL.java |  12 +-
 ...exWritesAndUncompressedWALInHBase_094_9.java |   2 +-
 .../org/apache/phoenix/end2end/QueryTest.java   |   3 +-
 .../phoenix/hbase/index/IndexTestingUtils.java  |  94 ++
 .../phoenix/hbase/index/StubAbortable.java      |  41 +
 .../apache/phoenix/hbase/index/TableName.java   |  45 +
 .../TestFailForUnsupportedHBaseVersions.java    | 156 ++++
 .../covered/CoveredIndexCodecForTesting.java    |  74 ++
 .../hbase/index/covered/TestCoveredColumns.java |  46 +
 .../TestEndToEndCoveredColumnsIndexBuilder.java | 344 ++++++++
 .../index/covered/TestLocalTableState.java      | 198 +++++
 .../index/covered/data/TestIndexMemStore.java   |  94 ++
 .../covered/example/TestColumnTracker.java      |  61 ++
 .../example/TestCoveredColumnIndexCodec.java    | 251 ++++++
 .../TestCoveredIndexSpecifierBuilder.java       |  75 ++
 .../example/TestEndToEndCoveredIndexing.java    | 882 +++++++++++++++++++
 .../TestEndtoEndIndexingWithCompression.java    |  49 ++
 .../covered/example/TestFailWithoutRetries.java | 148 ++++
 .../filter/TestApplyAndFilterDeletesFilter.java | 211 +++++
 .../covered/filter/TestFamilyOnlyFilter.java    | 106 +++
 .../filter/TestNewerTimestampFilter.java        |  48 +
 .../covered/update/TestIndexUpdateManager.java  | 140 +++
 .../index/parallel/TestThreadPoolBuilder.java   |  64 ++
 .../index/parallel/TestThreadPoolManager.java   |  95 ++
 .../index/util/TestIndexManagementUtil.java     |  67 ++
 .../hbase/index/write/FakeTableFactory.java     |  51 ++
 .../index/write/TestCachingHTableFactory.java   |  58 ++
 .../hbase/index/write/TestIndexWriter.java      | 287 ++++++
 .../index/write/TestParalleIndexWriter.java     | 119 +++
 .../write/TestParalleWriterIndexCommitter.java  | 119 +++
 .../index/write/TestWALRecoveryCaching.java     | 369 ++++++++
 .../recovery/TestPerRegionIndexWriteCache.java  | 168 ++++
 .../phoenix/index/IndexMaintainerTest.java      |   6 +-
 241 files changed, 13069 insertions(+), 13021 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/CapturingAbortable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/CapturingAbortable.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/CapturingAbortable.java
deleted file mode 100644
index f918c90..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/CapturingAbortable.java
+++ /dev/null
@@ -1,66 +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.hadoop.hbase.index;
-
-import org.apache.hadoop.hbase.Abortable;
-
-/**
- * {@link Abortable} that can rethrow the cause of the abort.
- */
-public class CapturingAbortable implements Abortable {
-
-  private Abortable delegate;
-  private Throwable cause;
-  private String why;
-
-  public CapturingAbortable(Abortable delegate) {
-    this.delegate = delegate;
-  }
-
-  @Override
-  public void abort(String why, Throwable e) {
-    if (delegate.isAborted()) {
-      return;
-    }
-    this.why = why;
-    this.cause = e;
-    delegate.abort(why, e);
-
-  }
-
-  @Override
-  public boolean isAborted() {
-    return delegate.isAborted();
-  }
-
-  /**
-   * Throw the cause of the abort, if <tt>this</tt> was aborted. If there was an exception causing
-   * the abort, re-throws that. Otherwise, just throws a generic {@link Exception} with the reason
-   * why the abort was caused.
-   * @throws Throwable the cause of the abort.
-   */
-  public void throwCauseIfAborted() throws Throwable {
-    if (!this.isAborted()) {
-      return;
-    }
-    if (cause == null) {
-      throw new Exception(why);
-    }
-    throw cause;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/IndexLogRollSynchronizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/IndexLogRollSynchronizer.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/IndexLogRollSynchronizer.java
deleted file mode 100644
index a4ef58e..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/IndexLogRollSynchronizer.java
+++ /dev/null
@@ -1,123 +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.hadoop.hbase.index;
-
-import java.io.IOException;
-import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-
-/**
- * Ensure that the log isn't rolled while we are the in middle of doing a pending index write.
- * <p>
- * The problem we are trying to solve is the following sequence:
- * <ol>
- * <li>Write to the indexed table</li>
- * <li>Write the index-containing WALEdit</li>
- * <li>Start writing to the index tables in the postXXX hook</li>
- * <li>WAL gets rolled and archived</li>
- * <li>An index update fails, in which case we should kill ourselves to get WAL replay</li>
- * <li>Since the WAL got archived, we won't get the replay of the index writes</li>
- * </ol>
- * <p>
- * The usual course of events should be:
- * <ol>
- * <li>In a preXXX hook,
- * <ol>
- * <li>Build the {@link WALEdit} + index information</li>
- * <li>Lock the {@link IndexLogRollSynchronizer#logArchiveLock}</li>
- * <ul>
- * <li>This is a reentrant readlock on the WAL archiving, so we can make multiple WAL/index updates
- * concurrently</li>
- * </ul>
- * </li>
- * </ol>
- * </li>
- * <li>Pass that {@link WALEdit} to the WAL, ensuring its durable and replayable</li>
- * <li>In the corresponding postXXX,
- * <ol>
- * <li>make the updates to the index tables</li>
- * <li>Unlock {@link IndexLogRollSynchronizer#logArchiveLock}</li>
- * </ol>
- * </li> </ol>
- * <p>
- * <tt>this</tt> should be added as a {@link WALActionsListener} by updating
- */
-public class IndexLogRollSynchronizer implements WALActionsListener {
-
-  private static final Log LOG = LogFactory.getLog(IndexLogRollSynchronizer.class);
-  private WriteLock logArchiveLock;
-
-  public IndexLogRollSynchronizer(WriteLock logWriteLock){
-    this.logArchiveLock = logWriteLock;
-  }
-
-
-  @Override
-  public void preLogArchive(Path oldPath, Path newPath) throws IOException {
-    //take a write lock on the index - any pending index updates will complete before we finish
-    LOG.debug("Taking INDEX_UPDATE writelock");
-    logArchiveLock.lock();
-    LOG.debug("Got the INDEX_UPDATE writelock");
-  }
-  
-  @Override
-  public void postLogArchive(Path oldPath, Path newPath) throws IOException {
-    // done archiving the logs, any WAL updates will be replayed on failure
-    LOG.debug("Releasing INDEX_UPDATE writelock");
-    logArchiveLock.unlock();
-  }
-
-  @Override
-  public void logCloseRequested() {
-    // don't care- before this is called, all the HRegions are closed, so we can't get any new
-    // requests and all pending request can finish before the WAL closes.
-  }
-
-  @Override
-  public void preLogRoll(Path oldPath, Path newPath) throws IOException {
-    // noop
-  }
-
-  @Override
-  public void postLogRoll(Path oldPath, Path newPath) throws IOException {
-    // noop
-  }
-
-  @Override
-  public void logRollRequested() {
-    // noop
-  }
-
-  @Override
-  public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit) {
-    // noop
-  }
-
-  @Override
-  public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, WALEdit logEdit) {
-    // noop
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/Indexer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/Indexer.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/Indexer.java
deleted file mode 100644
index ebe685d..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/Indexer.java
+++ /dev/null
@@ -1,704 +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.hadoop.hbase.index;
-
-import static org.apache.hadoop.hbase.index.util.IndexManagementUtil.rethrowIndexingException;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Coprocessor;
-import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
-import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
-import org.apache.hadoop.hbase.regionserver.ScanType;
-import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.wal.HLog;
-import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
-import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-import org.apache.hadoop.hbase.util.Pair;
-
-import com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.index.builder.IndexBuildManager;
-import org.apache.hadoop.hbase.index.builder.IndexBuilder;
-import org.apache.hadoop.hbase.index.builder.IndexBuildingFailureException;
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
-import org.apache.hadoop.hbase.index.wal.IndexedKeyValue;
-import org.apache.hadoop.hbase.index.write.IndexFailurePolicy;
-import org.apache.hadoop.hbase.index.write.IndexWriter;
-import org.apache.hadoop.hbase.index.write.recovery.PerRegionIndexWriteCache;
-import org.apache.hadoop.hbase.index.write.recovery.StoreFailuresInCachePolicy;
-import org.apache.hadoop.hbase.index.write.recovery.TrackingParallelWriterIndexCommitter;
-import org.apache.phoenix.util.MetaDataUtil;
-
-/**
- * Do all the work of managing index updates from a single coprocessor. All Puts/Delets are passed
- * to an {@link IndexBuilder} to determine the actual updates to make.
- * <p>
- * If the WAL is enabled, these updates are then added to the WALEdit and attempted to be written to
- * the WAL after the WALEdit has been saved. If any of the index updates fail, this server is
- * immediately terminated and we rely on WAL replay to attempt the index updates again (see
- * {@link #preWALRestore(ObserverContext, HRegionInfo, HLogKey, WALEdit)}).
- * <p>
- * If the WAL is disabled, the updates are attempted immediately. No consistency guarantees are made
- * if the WAL is disabled - some or none of the index updates may be successful. All updates in a
- * single batch must have the same durability level - either everything gets written to the WAL or
- * nothing does. Currently, we do not support mixed-durability updates within a single batch. If you
- * want to have different durability levels, you only need to split the updates into two different
- * batches.
- */
-public class Indexer extends BaseRegionObserver {
-
-  private static final Log LOG = LogFactory.getLog(Indexer.class);
-
-  /** WAL on this server */
-  private HLog log;
-  protected IndexWriter writer;
-  protected IndexBuildManager builder;
-
-  /** Configuration key for the {@link IndexBuilder} to use */
-  public static final String INDEX_BUILDER_CONF_KEY = "index.builder";
-
-  // Setup out locking on the index edits/WAL so we can be sure that we don't lose a roll a WAL edit
-  // before an edit is applied to the index tables
-  private static final ReentrantReadWriteLock INDEX_READ_WRITE_LOCK = new ReentrantReadWriteLock(
-      true);
-  public static final ReadLock INDEX_UPDATE_LOCK = INDEX_READ_WRITE_LOCK.readLock();
-
-  /**
-   * Configuration key for if the indexer should check the version of HBase is running. Generally,
-   * you only want to ignore this for testing or for custom versions of HBase.
-   */
-  public static final String CHECK_VERSION_CONF_KEY = "com.saleforce.hbase.index.checkversion";
-
-  private static final String INDEX_RECOVERY_FAILURE_POLICY_KEY = "org.apache.hadoop.hbase.index.recovery.failurepolicy";
-
-  /**
-   * Marker {@link KeyValue} to indicate that we are doing a batch operation. Needed because the
-   * coprocessor framework throws away the WALEdit from the prePut/preDelete hooks when checking a
-   * batch if there were no {@link KeyValue}s attached to the {@link WALEdit}. When you get down to
-   * the preBatch hook, there won't be any WALEdits to which to add the index updates.
-   */
-  private static KeyValue BATCH_MARKER = new KeyValue();
-
-  /**
-   * cache the failed updates to the various regions. Used for making the WAL recovery mechanisms
-   * more robust in the face of recoverying index regions that were on the same server as the
-   * primary table region
-   */
-  private PerRegionIndexWriteCache failedIndexEdits = new PerRegionIndexWriteCache();
-
-  /**
-   * IndexWriter for writing the recovered index edits. Separate from the main indexer since we need
-   * different write/failure policies
-   */
-  private IndexWriter recoveryWriter;
-
-  private boolean stopped;
-  private boolean disabled;
-
-  public static final String RecoveryFailurePolicyKeyForTesting = INDEX_RECOVERY_FAILURE_POLICY_KEY;
-
-    public static final int INDEXING_SUPPORTED_MAJOR_VERSION = MetaDataUtil
-            .encodeMaxPatchVersion(0, 94);
-    public static final int INDEXING_SUPPORTED__MIN_MAJOR_VERSION = MetaDataUtil
-            .encodeVersion("0.94.0");
-    private static final int INDEX_WAL_COMPRESSION_MINIMUM_SUPPORTED_VERSION = MetaDataUtil
-            .encodeVersion("0.94.9");
-
-  @Override
-  public void start(CoprocessorEnvironment e) throws IOException {
-      try {
-        final RegionCoprocessorEnvironment env = (RegionCoprocessorEnvironment) e;
-        String serverName = env.getRegionServerServices().getServerName().getServerName();
-        if (env.getConfiguration().getBoolean(CHECK_VERSION_CONF_KEY, true)) {
-          // make sure the right version <-> combinations are allowed.
-          String errormsg = Indexer.validateVersion(env.getHBaseVersion(), env.getConfiguration());
-          if (errormsg != null) {
-            IOException ioe = new IOException(errormsg);
-            env.getRegionServerServices().abort(errormsg, ioe);
-            throw ioe;
-          }
-        }
-    
-        this.builder = new IndexBuildManager(env);
-    
-        // get a reference to the WAL
-        log = env.getRegionServerServices().getWAL();
-        // add a synchronizer so we don't archive a WAL that we need
-        log.registerWALActionsListener(new IndexLogRollSynchronizer(INDEX_READ_WRITE_LOCK.writeLock()));
-    
-        // setup the actual index writer
-        this.writer = new IndexWriter(env, serverName + "-index-writer");
-    
-        // setup the recovery writer that does retries on the failed edits
-        TrackingParallelWriterIndexCommitter recoveryCommmiter =
-            new TrackingParallelWriterIndexCommitter();
-    
-        try {
-          // get the specified failure policy. We only ever override it in tests, but we need to do it
-          // here
-          Class<? extends IndexFailurePolicy> policyClass =
-              env.getConfiguration().getClass(INDEX_RECOVERY_FAILURE_POLICY_KEY,
-                StoreFailuresInCachePolicy.class, IndexFailurePolicy.class);
-          IndexFailurePolicy policy =
-              policyClass.getConstructor(PerRegionIndexWriteCache.class).newInstance(failedIndexEdits);
-          LOG.debug("Setting up recovery writter with committer: " + recoveryCommmiter.getClass()
-              + " and failure policy: " + policy.getClass());
-          recoveryWriter =
-              new IndexWriter(recoveryCommmiter, policy, env, serverName + "-recovery-writer");
-        } catch (Exception ex) {
-          throw new IOException("Could not instantiate recovery failure policy!", ex);
-        }
-      } catch (NoSuchMethodError ex) {
-          disabled = true;
-          super.start(e);
-          LOG.error("Must be too early a version of HBase. Disabled coprocessor ", ex);
-      }
-  }
-
-  @Override
-  public void stop(CoprocessorEnvironment e) throws IOException {
-    if (this.stopped) {
-      return;
-    }
-    if (this.disabled) {
-        super.stop(e);
-        return;
-      }
-    this.stopped = true;
-    String msg = "Indexer is being stopped";
-    this.builder.stop(msg);
-    this.writer.stop(msg);
-    this.recoveryWriter.stop(msg);
-  }
-
-  @Override
-  public void prePut(final ObserverContext<RegionCoprocessorEnvironment> c, final Put put,
-      final WALEdit edit, final boolean writeToWAL) throws IOException {
-      if (this.disabled) {
-          super.prePut(c, put, edit, writeToWAL);
-          return;
-        }
-    // just have to add a batch marker to the WALEdit so we get the edit again in the batch
-    // processing step. We let it throw an exception here because something terrible has happened.
-    edit.add(BATCH_MARKER);
-  }
-
-  @Override
-  public void preDelete(ObserverContext<RegionCoprocessorEnvironment> e, Delete delete,
-      WALEdit edit, boolean writeToWAL) throws IOException {
-      if (this.disabled) {
-          super.preDelete(e, delete, edit, writeToWAL);
-          return;
-        }
-    try {
-      preDeleteWithExceptions(e, delete, edit, writeToWAL);
-      return;
-    } catch (Throwable t) {
-      rethrowIndexingException(t);
-    }
-    throw new RuntimeException(
-        "Somehow didn't return an index update but also didn't propagate the failure to the client!");
-  }
-
-  public void preDeleteWithExceptions(ObserverContext<RegionCoprocessorEnvironment> e,
-      Delete delete, WALEdit edit, boolean writeToWAL) throws Exception {
-    // if we are making the update as part of a batch, we need to add in a batch marker so the WAL
-    // is retained
-    if (this.builder.getBatchId(delete) != null) {
-      edit.add(BATCH_MARKER);
-      return;
-    }
-
-    // get the mapping for index column -> target index table
-    Collection<Pair<Mutation, byte[]>> indexUpdates = this.builder.getIndexUpdate(delete);
-
-    if (doPre(indexUpdates, edit, writeToWAL)) {
-      takeUpdateLock("delete");
-    }
-  }
-
-  @Override
-  public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
-      MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
-      if (this.disabled) {
-          super.preBatchMutate(c, miniBatchOp);
-          return;
-        }
-    try {
-      preBatchMutateWithExceptions(c, miniBatchOp);
-      return;
-    } catch (Throwable t) {
-      rethrowIndexingException(t);
-    }
-    throw new RuntimeException(
-        "Somehow didn't return an index update but also didn't propagate the failure to the client!");
-  }
-
-  @SuppressWarnings("deprecation")
-  public void preBatchMutateWithExceptions(ObserverContext<RegionCoprocessorEnvironment> c,
-      MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws Throwable {
-
-    // first group all the updates for a single row into a single update to be processed
-    Map<ImmutableBytesPtr, MultiMutation> mutations =
-        new HashMap<ImmutableBytesPtr, MultiMutation>();
-    boolean durable = false;
-    for (int i = 0; i < miniBatchOp.size(); i++) {
-      // remove the batch keyvalue marker - its added for all puts
-      WALEdit edit = miniBatchOp.getWalEdit(i);
-      // we don't have a WALEdit for immutable index cases, which still see this path
-      // we could check is indexing is enable for the mutation in prePut and then just skip this
-      // after checking here, but this saves us the checking again.
-      if (edit != null) {
-        KeyValue kv = edit.getKeyValues().remove(0);
-        assert kv == BATCH_MARKER : "Expected batch marker from the WALEdit, but got: " + kv;
-      }
-      Pair<Mutation, Integer> op = miniBatchOp.getOperation(i);
-      Mutation m = op.getFirst();
-      // skip this mutation if we aren't enabling indexing
-      // unfortunately, we really should ask if the raw mutation (rather than the combined mutation)
-      // should be indexed, which means we need to expose another method on the builder. Such is the
-      // way optimization go though.
-      if (!this.builder.isEnabled(m)) {
-        continue;
-      }
-      
-      // figure out if this is batch is durable or not
-      if(!durable){
-        durable = m.getDurability() != Durability.SKIP_WAL;
-      }
-
-      // add the mutation to the batch set
-      ImmutableBytesPtr row = new ImmutableBytesPtr(m.getRow());
-      MultiMutation stored = mutations.get(row);
-      // we haven't seen this row before, so add it
-      if (stored == null) {
-        stored = new MultiMutation(row, m.getWriteToWAL());
-        mutations.put(row, stored);
-      }
-      stored.addAll(m);
-    }
-    
-    // early exit if it turns out we don't have any edits
-    if (mutations.entrySet().size() == 0) {
-      return;
-    }
-
-    // dump all the index updates into a single WAL. They will get combined in the end anyways, so
-    // don't worry which one we get
-    WALEdit edit = miniBatchOp.getWalEdit(0);
-
-    // get the index updates for all elements in this batch
-    Collection<Pair<Mutation, byte[]>> indexUpdates =
-        this.builder.getIndexUpdate(miniBatchOp, mutations.values());
-    // write them
-    if (doPre(indexUpdates, edit, durable)) {
-      takeUpdateLock("batch mutation");
-    }
-  }
-
-  private void takeUpdateLock(String opDesc) throws IndexBuildingFailureException {
-    boolean interrupted = false;
-    // lock the log, so we are sure that index write gets atomically committed
-    LOG.debug("Taking INDEX_UPDATE readlock for " + opDesc);
-    // wait for the update lock
-    while (!this.stopped) {
-      try {
-        INDEX_UPDATE_LOCK.lockInterruptibly();
-        LOG.debug("Got the INDEX_UPDATE readlock for " + opDesc);
-        // unlock the lock so the server can shutdown, if we find that we have stopped since getting
-        // the lock
-        if (this.stopped) {
-          INDEX_UPDATE_LOCK.unlock();
-          throw new IndexBuildingFailureException(
-              "Found server stop after obtaining the update lock, killing update attempt");
-        }
-        break;
-      } catch (InterruptedException e) {
-        LOG.info("Interrupted while waiting for update lock. Ignoring unless stopped");
-        interrupted = true;
-      }
-    }
-    if (interrupted) {
-      Thread.currentThread().interrupt();
-    }
-  }
-
-  private class MultiMutation extends Mutation {
-
-    private ImmutableBytesPtr rowKey;
-
-    public MultiMutation(ImmutableBytesPtr rowkey, boolean writeToWal) {
-      this.rowKey = rowkey;
-      this.writeToWAL = writeToWal;
-    }
-
-    /**
-     * @param stored
-     */
-    @SuppressWarnings("deprecation")
-    public void addAll(Mutation stored) {
-      // add all the kvs
-      for (Entry<byte[], List<KeyValue>> kvs : stored.getFamilyMap().entrySet()) {
-        byte[] family = kvs.getKey();
-        List<KeyValue> list = getKeyValueList(family, kvs.getValue().size());
-        list.addAll(kvs.getValue());
-        familyMap.put(family, list);
-      }
-
-      // add all the attributes, not overriding already stored ones
-      for (Entry<String, byte[]> attrib : stored.getAttributesMap().entrySet()) {
-        if (this.getAttribute(attrib.getKey()) == null) {
-          this.setAttribute(attrib.getKey(), attrib.getValue());
-        }
-      }
-      if (stored.getWriteToWAL()) {
-        this.writeToWAL = true;
-      }
-    }
-
-    private List<KeyValue> getKeyValueList(byte[] family, int hint) {
-      List<KeyValue> list = familyMap.get(family);
-      if (list == null) {
-        list = new ArrayList<KeyValue>(hint);
-      }
-      return list;
-    }
-
-    @Override
-    public byte[] getRow(){
-      return this.rowKey.copyBytesIfNecessary();
-    }
-
-    @Override
-    public int hashCode() {
-      return this.rowKey.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      return o == null ? false : o.hashCode() == this.hashCode();
-    }
-
-    @Override
-    public void readFields(DataInput arg0) throws IOException {
-      throw new UnsupportedOperationException("MultiMutations cannot be read/written");
-    }
-
-    @Override
-    public void write(DataOutput arg0) throws IOException {
-      throw new UnsupportedOperationException("MultiMutations cannot be read/written");
-    }
-  }
-
-  /**
-   * Add the index updates to the WAL, or write to the index table, if the WAL has been disabled
-   * @return <tt>true</tt> if the WAL has been updated.
-   * @throws IOException
-   */
-  private boolean doPre(Collection<Pair<Mutation, byte[]>> indexUpdates, final WALEdit edit,
-      final boolean writeToWAL) throws IOException {
-    // no index updates, so we are done
-    if (indexUpdates == null || indexUpdates.size() == 0) {
-      return false;
-    }
-
-    // if writing to wal is disabled, we never see the WALEdit updates down the way, so do the index
-    // update right away
-    if (!writeToWAL) {
-      try {
-        this.writer.write(indexUpdates);
-        return false;
-      } catch (Throwable e) {
-        LOG.error("Failed to update index with entries:" + indexUpdates, e);
-        IndexManagementUtil.rethrowIndexingException(e);
-      }
-    }
-
-    // we have all the WAL durability, so we just update the WAL entry and move on
-    for (Pair<Mutation, byte[]> entry : indexUpdates) {
-      edit.add(new IndexedKeyValue(entry.getSecond(), entry.getFirst()));
-    }
-
-    return true;
-  }
-
-  @Override
-  public void postPut(ObserverContext<RegionCoprocessorEnvironment> e, Put put, WALEdit edit,
-      boolean writeToWAL) throws IOException {
-      if (this.disabled) {
-          super.postPut(e, put, edit, writeToWAL);
-          return;
-        }
-    doPost(edit, put, writeToWAL);
-  }
-
-  @Override
-  public void postDelete(ObserverContext<RegionCoprocessorEnvironment> e, Delete delete,
-      WALEdit edit, boolean writeToWAL) throws IOException {
-      if (this.disabled) {
-          super.postDelete(e, delete, edit, writeToWAL);
-          return;
-        }
-    doPost(edit,delete, writeToWAL);
-  }
-
-  @Override
-  public void postBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
-      MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
-      if (this.disabled) {
-          super.postBatchMutate(c, miniBatchOp);
-          return;
-        }
-    this.builder.batchCompleted(miniBatchOp);
-    // noop for the rest of the indexer - its handled by the first call to put/delete
-  }
-
-  private void doPost(WALEdit edit, Mutation m, boolean writeToWAL) throws IOException {
-    try {
-      doPostWithExceptions(edit, m, writeToWAL);
-      return;
-    } catch (Throwable e) {
-      rethrowIndexingException(e);
-    }
-    throw new RuntimeException(
-        "Somehow didn't complete the index update, but didn't return succesfully either!");
-  }
-
-  private void doPostWithExceptions(WALEdit edit, Mutation m, boolean writeToWAL) throws Exception {
-    //short circuit, if we don't need to do any work
-    if (!writeToWAL || !this.builder.isEnabled(m)) {
-      // already did the index update in prePut, so we are done
-      return;
-    }
-
-    // there is a little bit of excess here- we iterate all the non-indexed kvs for this check first
-    // and then do it again later when getting out the index updates. This should be pretty minor
-    // though, compared to the rest of the runtime
-    IndexedKeyValue ikv = getFirstIndexedKeyValue(edit);
-    /*
-     * early exit - we have nothing to write, so we don't need to do anything else. NOTE: we don't
-     * release the WAL Rolling lock (INDEX_UPDATE_LOCK) since we never take it in doPre if there are
-     * no index updates.
-     */
-    if (ikv == null) {
-      return;
-    }
-
-    /*
-     * only write the update if we haven't already seen this batch. We only want to write the batch
-     * once (this hook gets called with the same WALEdit for each Put/Delete in a batch, which can
-     * lead to writing all the index updates for each Put/Delete).
-     */
-    if (!ikv.getBatchFinished()) {
-      Collection<Pair<Mutation, byte[]>> indexUpdates = extractIndexUpdate(edit);
-
-      // the WAL edit is kept in memory and we already specified the factory when we created the
-      // references originally - therefore, we just pass in a null factory here and use the ones
-      // already specified on each reference
-      try {
-          writer.writeAndKillYourselfOnFailure(indexUpdates);
-      } finally {
-        // With a custom kill policy, we may throw instead of kill the server.
-        // Without doing this in a finally block (at least with the mini cluster),
-        // the region server never goes down.
-
-        // mark the batch as having been written. In the single-update case, this never gets check
-        // again, but in the batch case, we will check it again (see above).
-        ikv.markBatchFinished();
-      
-        // release the lock on the index, we wrote everything properly
-        // we took the lock for each Put/Delete, so we have to release it a matching number of times
-        // batch cases only take the lock once, so we need to make sure we don't over-release the
-        // lock.
-        LOG.debug("Releasing INDEX_UPDATE readlock");
-        INDEX_UPDATE_LOCK.unlock();
-      }
-    }
-  }
-
-  /**
-   * Search the {@link WALEdit} for the first {@link IndexedKeyValue} present
-   * @param edit {@link WALEdit}
-   * @return the first {@link IndexedKeyValue} in the {@link WALEdit} or <tt>null</tt> if not
-   *         present
-   */
-  private IndexedKeyValue getFirstIndexedKeyValue(WALEdit edit) {
-    for (KeyValue kv : edit.getKeyValues()) {
-      if (kv instanceof IndexedKeyValue) {
-        return (IndexedKeyValue) kv;
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Extract the index updates from the WAL Edit
-   * @param edit to search for index updates
-   * @return the mutations to apply to the index tables
-   */
-  private Collection<Pair<Mutation, byte[]>> extractIndexUpdate(WALEdit edit) {
-    Collection<Pair<Mutation, byte[]>> indexUpdates = new ArrayList<Pair<Mutation, byte[]>>();
-    for (KeyValue kv : edit.getKeyValues()) {
-      if (kv instanceof IndexedKeyValue) {
-        IndexedKeyValue ikv = (IndexedKeyValue) kv;
-        indexUpdates.add(new Pair<Mutation, byte[]>(ikv.getMutation(), ikv.getIndexTable()));
-      }
-    }
-
-    return indexUpdates;
-  }
-
-  @Override
-  public void postOpen(final ObserverContext<RegionCoprocessorEnvironment> c) {
-    Multimap<HTableInterfaceReference, Mutation> updates = failedIndexEdits.getEdits(c.getEnvironment().getRegion());
-    
-    if (this.disabled) {
-        super.postOpen(c);
-        return;
-      }
-    LOG.info("Found some outstanding index updates that didn't succeed during"
-        + " WAL replay - attempting to replay now.");
-    //if we have no pending edits to complete, then we are done
-    if (updates == null || updates.size() == 0) {
-      return;
-    }
-    
-    // do the usual writer stuff, killing the server again, if we can't manage to make the index
-    // writes succeed again
-    try {
-        writer.writeAndKillYourselfOnFailure(updates);
-    } catch (IOException e) {
-        LOG.error("Exception thrown instead of killing server during index writing", e);
-    }
-  }
-
-  @Override
-  public void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info,
-      HLogKey logKey, WALEdit logEdit) throws IOException {
-      if (this.disabled) {
-          super.preWALRestore(env, info, logKey, logEdit);
-          return;
-        }
-    // TODO check the regions in transition. If the server on which the region lives is this one,
-    // then we should rety that write later in postOpen.
-    // we might be able to get even smarter here and pre-split the edits that are server-local
-    // into their own recovered.edits file. This then lets us do a straightforward recovery of each
-    // region (and more efficiently as we aren't writing quite as hectically from this one place).
-
-    /*
-     * Basically, we let the index regions recover for a little while long before retrying in the
-     * hopes they come up before the primary table finishes.
-     */
-    Collection<Pair<Mutation, byte[]>> indexUpdates = extractIndexUpdate(logEdit);
-    recoveryWriter.writeAndKillYourselfOnFailure(indexUpdates);
-  }
-
-  /**
-   * Create a custom {@link InternalScanner} for a compaction that tracks the versions of rows that
-   * are removed so we can clean then up from the the index table(s).
-   * <p>
-   * This is not yet implemented - its not clear if we should even mess around with the Index table
-   * for these rows as those points still existed. TODO: v2 of indexing
-   */
-  @Override
-  public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
-      Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
-      InternalScanner s) throws IOException {
-    return super.preCompactScannerOpen(c, store, scanners, scanType, earliestPutTs, s);
-  }
-
-  /**
-   * Exposed for testing!
-   * @return the currently instantiated index builder
-   */
-  public IndexBuilder getBuilderForTesting() {
-    return this.builder.getBuilderForTesting();
-  }
-
-    /**
-     * Validate that the version and configuration parameters are supported
-     * @param hbaseVersion current version of HBase on which <tt>this</tt> coprocessor is installed
-     * @param conf configuration to check for allowed parameters (e.g. WAL Compression only if >=
-     *            0.94.9)
-     * @return <tt>null</tt> if the version is supported, the error message to display otherwise
-     */
-    public static String validateVersion(String hbaseVersion, Configuration conf) {
-        int encodedVersion = MetaDataUtil.encodeVersion(hbaseVersion);
-        // above 0.94 everything should be supported
-        if (encodedVersion > INDEXING_SUPPORTED_MAJOR_VERSION) {
-            return null;
-        }
-        // check to see if its at least 0.94
-        if (encodedVersion < INDEXING_SUPPORTED__MIN_MAJOR_VERSION) {
-            return "Indexing not supported for versions older than 0.94.X";
-        }
-        // if less than 0.94.9, we need to check if WAL Compression is enabled
-        if (encodedVersion < INDEX_WAL_COMPRESSION_MINIMUM_SUPPORTED_VERSION) {
-            if (conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false)) {
-                return "Indexing not supported with WAL Compression for versions of HBase older than 0.94.9 - found version:"
-                        + hbaseVersion;
-            }
-        }
-        return null;
-    }
-
-  /**
-   * Enable indexing on the given table
-   * @param desc {@link HTableDescriptor} for the table on which indexing should be enabled
-   * @param builder class to use when building the index for this table
-   * @param properties map of custom configuration options to make available to your
-   *          {@link IndexBuilder} on the server-side
-   * @throws IOException the Indexer coprocessor cannot be added
-   */
-  public static void enableIndexing(HTableDescriptor desc, Class<? extends IndexBuilder> builder,
-      Map<String, String> properties) throws IOException {
-    if (properties == null) {
-      properties = new HashMap<String, String>();
-    }
-    properties.put(Indexer.INDEX_BUILDER_CONF_KEY, builder.getName());
-    desc.addCoprocessor(Indexer.class.getName(), null, Coprocessor.PRIORITY_USER, properties);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/ValueGetter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/ValueGetter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/ValueGetter.java
deleted file mode 100644
index b328ff2..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/ValueGetter.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.index;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-public interface ValueGetter {
-
-  /**
-   * Get the most recent (largest timestamp) for the given column reference
-   * @param ref to match against an underlying key value. Uses the passed object to match the
-   *          keyValue via {@link ColumnReference#matches}
-   * @return the stored value for the given {@link ColumnReference}, or <tt>null</tt> if no value is
-   *         present.
-   * @throws IOException if there is an error accessing the underlying data storage
-   */
-  public ImmutableBytesPtr getLatestValue(ColumnReference ref) throws IOException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/BaseIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/BaseIndexBuilder.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/BaseIndexBuilder.java
deleted file mode 100644
index d369bd8..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/BaseIndexBuilder.java
+++ /dev/null
@@ -1,97 +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.hadoop.hbase.index.builder;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.hadoop.hbase.index.covered.CoveredColumnsIndexBuilder;
-
-/**
- * Basic implementation of the {@link IndexBuilder} that doesn't do any actual work of indexing.
- * <p>
- * You should extend this class, rather than implementing IndexBuilder directly to maintain
- * compatability going forward.
- * <p>
- * Generally, you should consider using one of the implemented IndexBuilders (e.g
- * {@link CoveredColumnsIndexBuilder}) as there is a lot of work required to keep an index table
- * up-to-date.
- */
-public abstract class BaseIndexBuilder implements IndexBuilder {
-
-  private static final Log LOG = LogFactory.getLog(BaseIndexBuilder.class);
-  protected boolean stopped;
-
-  @Override
-  public void extendBaseIndexBuilderInstead() { }
-  
-  @Override
-  public void setup(RegionCoprocessorEnvironment conf) throws IOException {
-    // noop
-  }
-
-  @Override
-  public void batchStarted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
-    // noop
-  }
-
-  @Override
-  public void batchCompleted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) {
-    // noop
-  }
-  
-  /**
-   * By default, we always attempt to index the mutation. Commonly this can be slow (because the
-   * framework spends the time to do the indexing, only to realize that you don't need it) or not
-   * ideal (if you want to turn on/off indexing on a table without completely reloading it).
- * @throws IOException 
-   */
-  @Override
-  public boolean isEnabled(Mutation m) throws IOException {
-    return true; 
-  }
-
-  /**
-   * {@inheritDoc}
-   * <p>
-   * By default, assumes that all mutations should <b>not be batched</b>. That is to say, each
-   * mutation always applies to different rows, even if they are in the same batch, or are
-   * independent updates.
-   */
-  @Override
-  public byte[] getBatchId(Mutation m) {
-    return null;
-  }
-
-  @Override
-  public void stop(String why) {
-    LOG.debug("Stopping because: " + why);
-    this.stopped = true;
-  }
-
-  @Override
-  public boolean isStopped() {
-    return this.stopped;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildManager.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildManager.java
deleted file mode 100644
index d450353..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildManager.java
+++ /dev/null
@@ -1,214 +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.hadoop.hbase.index.builder;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.ExecutionException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.hadoop.hbase.index.Indexer;
-import org.apache.hadoop.hbase.index.parallel.QuickFailingTaskRunner;
-import org.apache.hadoop.hbase.index.parallel.Task;
-import org.apache.hadoop.hbase.index.parallel.TaskBatch;
-import org.apache.hadoop.hbase.index.parallel.ThreadPoolBuilder;
-import org.apache.hadoop.hbase.index.parallel.ThreadPoolManager;
-
-/**
- * Manage the building of index updates from primary table updates.
- * <p>
- * Internally, parallelizes updates through a thread-pool to a delegate index builder. Underlying
- * {@link IndexBuilder} <b>must be thread safe</b> for each index update.
- */
-public class IndexBuildManager implements Stoppable {
-
-  private static final Log LOG = LogFactory.getLog(IndexBuildManager.class);
-  private final IndexBuilder delegate;
-  private QuickFailingTaskRunner pool;
-  private boolean stopped;
-
-  /**
-   * Set the number of threads with which we can concurrently build index updates. Unused threads
-   * will be released, but setting the number of threads too high could cause frequent swapping and
-   * resource contention on the server - <i>tune with care</i>. However, if you are spending a lot
-   * of time building index updates, it could be worthwhile to spend the time to tune this parameter
-   * as it could lead to dramatic increases in speed.
-   */
-  public static final String NUM_CONCURRENT_INDEX_BUILDER_THREADS_CONF_KEY = "index.builder.threads.max";
-  /** Default to a single thread. This is the safest course of action, but the slowest as well */
-  private static final int DEFAULT_CONCURRENT_INDEX_BUILDER_THREADS = 10;
-  /**
-   * Amount of time to keep idle threads in the pool. After this time (seconds) we expire the
-   * threads and will re-create them as needed, up to the configured max
-   */
-  private static final String INDEX_BUILDER_KEEP_ALIVE_TIME_CONF_KEY =
-      "index.builder.threads.keepalivetime";
-
-  /**
-   * @param env environment in which <tt>this</tt> is running. Used to setup the
-   *          {@link IndexBuilder} and executor
-   * @throws IOException if an {@link IndexBuilder} cannot be correctly steup
-   */
-  public IndexBuildManager(RegionCoprocessorEnvironment env) throws IOException {
-    this(getIndexBuilder(env), new QuickFailingTaskRunner(ThreadPoolManager.getExecutor(
-      getPoolBuilder(env), env)));
-  }
-
-  private static IndexBuilder getIndexBuilder(RegionCoprocessorEnvironment e) throws IOException {
-    Configuration conf = e.getConfiguration();
-    Class<? extends IndexBuilder> builderClass =
-        conf.getClass(Indexer.INDEX_BUILDER_CONF_KEY, null, IndexBuilder.class);
-    try {
-      IndexBuilder builder = builderClass.newInstance();
-      builder.setup(e);
-      return builder;
-    } catch (InstantiationException e1) {
-      throw new IOException("Couldn't instantiate index builder:" + builderClass
-          + ", disabling indexing on table " + e.getRegion().getTableDesc().getNameAsString());
-    } catch (IllegalAccessException e1) {
-      throw new IOException("Couldn't instantiate index builder:" + builderClass
-          + ", disabling indexing on table " + e.getRegion().getTableDesc().getNameAsString());
-    }
-  }
-
-  private static ThreadPoolBuilder getPoolBuilder(RegionCoprocessorEnvironment env) {
-    String serverName = env.getRegionServerServices().getServerName().getServerName();
-    return new ThreadPoolBuilder(serverName + "-index-builder", env.getConfiguration()).
-        setCoreTimeout(INDEX_BUILDER_KEEP_ALIVE_TIME_CONF_KEY).
-        setMaxThread(NUM_CONCURRENT_INDEX_BUILDER_THREADS_CONF_KEY,
-          DEFAULT_CONCURRENT_INDEX_BUILDER_THREADS);
-  }
-
-  public IndexBuildManager(IndexBuilder builder, QuickFailingTaskRunner pool) {
-    this.delegate = builder;
-    this.pool = pool;
-  }
-
-
-  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(
-      MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp,
-      Collection<? extends Mutation> mutations) throws Throwable {
-    // notify the delegate that we have started processing a batch
-    this.delegate.batchStarted(miniBatchOp);
-
-    // parallelize each mutation into its own task
-    // each task is cancelable via two mechanisms: (1) underlying HRegion is closing (which would
-    // fail lookups/scanning) and (2) by stopping this via the #stop method. Interrupts will only be
-    // acknowledged on each thread before doing the actual lookup, but after that depends on the
-    // underlying builder to look for the closed flag.
-    TaskBatch<Collection<Pair<Mutation, byte[]>>> tasks =
-        new TaskBatch<Collection<Pair<Mutation, byte[]>>>(mutations.size());
-    for (final Mutation m : mutations) {
-      tasks.add(new Task<Collection<Pair<Mutation, byte[]>>>() {
-
-        @Override
-        public Collection<Pair<Mutation, byte[]>> call() throws IOException {
-          return delegate.getIndexUpdate(m);
-        }
-
-      });
-    }
-    List<Collection<Pair<Mutation, byte[]>>> allResults = null;
-    try {
-      allResults = pool.submitUninterruptible(tasks);
-    } catch (CancellationException e) {
-      throw e;
-    } catch (ExecutionException e) {
-      LOG.error("Found a failed index update!");
-      throw e.getCause();
-    }
-
-    // we can only get here if we get successes from each of the tasks, so each of these must have a
-    // correct result
-    Collection<Pair<Mutation, byte[]>> results = new ArrayList<Pair<Mutation, byte[]>>();
-    for (Collection<Pair<Mutation, byte[]>> result : allResults) {
-      assert result != null : "Found an unsuccessful result, but didn't propagate a failure earlier";
-      results.addAll(result);
-    }
-
-    return results;
-  }
-
-  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Delete delete) throws IOException {
-    // all we get is a single update, so it would probably just go slower if we needed to queue it
-    // up. It will increase underlying resource contention a little bit, but the mutation case is
-    // far more common, so let's not worry about it for now.
-    // short circuit so we don't waste time.
-    if (!this.delegate.isEnabled(delete)) {
-      return null;
-    }
-
-    return delegate.getIndexUpdate(delete);
-
-  }
-
-  public Collection<Pair<Mutation, byte[]>> getIndexUpdateForFilteredRows(
-      Collection<KeyValue> filtered) throws IOException {
-    // this is run async, so we can take our time here
-    return delegate.getIndexUpdateForFilteredRows(filtered);
-  }
-
-  public void batchCompleted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) {
-    delegate.batchCompleted(miniBatchOp);
-  }
-
-  public void batchStarted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp)
-      throws IOException {
-    delegate.batchStarted(miniBatchOp);
-  }
-
-  public boolean isEnabled(Mutation m) throws IOException {
-    return delegate.isEnabled(m);
-  }
-
-  public byte[] getBatchId(Mutation m) {
-    return delegate.getBatchId(m);
-  }
-
-  @Override
-  public void stop(String why) {
-    if (stopped) {
-      return;
-    }
-    this.stopped = true;
-    this.delegate.stop(why);
-    this.pool.stop(why);
-  }
-
-  @Override
-  public boolean isStopped() {
-    return this.stopped;
-  }
-
-  public IndexBuilder getBuilderForTesting() {
-    return this.delegate;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuilder.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuilder.java
deleted file mode 100644
index f469482..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuilder.java
+++ /dev/null
@@ -1,142 +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.hadoop.hbase.index.builder;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.hadoop.hbase.index.Indexer;
-
-/**
- * Interface to build updates ({@link Mutation}s) to the index tables, based on the primary table
- * updates.
- * <p>
- * Either all the index updates will be applied to all tables or the primary table will kill itself
- * and will attempt to replay the index edits through the WAL replay mechanism.
- */
-public interface IndexBuilder extends Stoppable {
-
-  /** Helper method signature to ensure people don't attempt to extend this class directly */
-  public void extendBaseIndexBuilderInstead();
-
-  /**
-   * This is always called exactly once on install of {@link Indexer}, before any calls
-   * {@link #getIndexUpdate} on
-   * @param env in which the builder is running
-   * @throws IOException on failure to setup the builder
-   */
-  public void setup(RegionCoprocessorEnvironment env) throws IOException;
-
-  /**
-   * Your opportunity to update any/all index tables based on the update of the primary table row.
-   * Its up to your implementation to ensure that timestamps match between the primary and index
-   * tables.
-   * <p>
-   * The mutation is a generic mutation (not a {@link Put} or a {@link Delete}), as it actually
-   * corresponds to a batch update. Its important to note that {@link Put}s always go through the
-   * batch update code path, so a single {@link Put} will come through here and update the primary
-   * table as the only update in the mutation.
-   * <p>
-   * Implementers must ensure that this method is thread-safe - it could (and probably will) be
-   * called concurrently for different mutations, which may or may not be part of the same batch.
-   * @param mutation update to the primary table to be indexed.
-   * @return a Map of the mutations to make -> target index table name
-   * @throws IOException on failure
-   */
-  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Mutation mutation) throws IOException;
-
-  /**
-   * The counter-part to {@link #getIndexUpdate(Mutation)} - your opportunity to update any/all
-   * index tables based on the delete of the primary table row. This is only called for cases where
-   * the client sends a single delete ({@link HTable#delete}). We separate this method from
-   * {@link #getIndexUpdate(Mutation)} only for the ease of implementation as the delete path has
-   * subtly different semantics for updating the families/timestamps from the generic batch path.
-   * <p>
-   * Its up to your implementation to ensure that timestamps match between the primary and index
-   * tables.
-   * <p>
-   * Implementers must ensure that this method is thread-safe - it could (and probably will) be
-   * called concurrently for different mutations, which may or may not be part of the same batch.
-   * @param delete {@link Delete} to the primary table that may be indexed
-   * @return a {@link Map} of the mutations to make -> target index table name
-   * @throws IOException on failure
-   */
-  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Delete delete) throws IOException;
-
-  /**
-   * Build an index update to cleanup the index when we remove {@link KeyValue}s via the normal
-   * flush or compaction mechanisms.
-   * @param filtered {@link KeyValue}s that previously existed, but won't be included in further
-   *          output from HBase.
-   * @return a {@link Map} of the mutations to make -> target index table name
-   * @throws IOException on failure
-   */
-  public Collection<Pair<Mutation, byte[]>> getIndexUpdateForFilteredRows(
-      Collection<KeyValue> filtered)
-      throws IOException;
-
-  /**
-   * Notification that a batch of updates has successfully been written.
-   * @param miniBatchOp the full batch operation that was written
-   */
-  public void batchCompleted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp);
-
-  /**
-   * Notification that a batch has been started.
-   * <p>
-   * Unfortunately, the way HBase has the coprocessor hooks setup, this is actually called
-   * <i>after</i> the {@link #getIndexUpdate} methods. Therefore, you will likely need an attribute
-   * on your {@link Put}/{@link Delete} to indicate it is a batch operation.
-   * @param miniBatchOp the full batch operation to be written
- * @throws IOException 
-   */
-  public void batchStarted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException;
-
-  /**
-   * This allows the codec to dynamically change whether or not indexing should take place for a
-   * table. If it doesn't take place, we can save a lot of time on the regular Put patch. By making
-   * it dynamic, we can save offlining and then onlining a table just to turn indexing on.
-   * <p>
-   * We can also be smart about even indexing a given update here too - if the update doesn't
-   * contain any columns that we care about indexing, we can save the effort of analyzing the put
-   * and further.
-   * @param m mutation that should be indexed.
-   * @return <tt>true</tt> if indexing is enabled for the given table. This should be on a per-table
-   *         basis, as each codec is instantiated per-region.
- * @throws IOException 
-   */
-  public boolean isEnabled(Mutation m) throws IOException;
-
-  /**
-   * @param m mutation that has been received by the indexer and is waiting to be indexed
-   * @return the ID of batch to which the Mutation belongs, or <tt>null</tt> if the mutation is not
-   *         part of a batch.
-   */
-  public byte[] getBatchId(Mutation m);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildingFailureException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildingFailureException.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildingFailureException.java
deleted file mode 100644
index 3d0ef14..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/builder/IndexBuildingFailureException.java
+++ /dev/null
@@ -1,48 +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.hadoop.hbase.index.builder;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-
-/**
- * Unexpected failure while building index updates that wasn't caused by an {@link IOException}.
- * This should be used if there is some basic issue with indexing - and no matter of retries will
- * fix it.
- */
-@SuppressWarnings("serial")
-public class IndexBuildingFailureException extends DoNotRetryIOException {
-
-  /**
-   * Constructor for over the wire propagation. Generally, shouldn't be used since index failure
-   * should have an underlying cause to propagate.
-   * @param msg reason for the failure
-   */
-  public IndexBuildingFailureException(String msg) {
-    super(msg);
-  }
-
-  /**
-   * @param msg reason
-   * @param cause underlying cause for the failure
-   */
-  public IndexBuildingFailureException(String msg, Throwable cause) {
-    super(msg, cause);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/Batch.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/Batch.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/Batch.java
deleted file mode 100644
index 088203e..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/Batch.java
+++ /dev/null
@@ -1,60 +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.hadoop.hbase.index.covered;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.hbase.KeyValue;
-
-/**
- * A collection of {@link KeyValue KeyValues} to the primary table
- */
-public class Batch {
-
-  private static final long pointDeleteCode = KeyValue.Type.Delete.getCode();
-  private final long timestamp;
-  private List<KeyValue> batch = new ArrayList<KeyValue>();
-  private boolean allPointDeletes = true;
-
-  /**
-   * @param ts
-   */
-  public Batch(long ts) {
-    this.timestamp = ts;
-  }
-
-  public void add(KeyValue kv){
-    if (pointDeleteCode != kv.getType()) {
-      allPointDeletes = false;
-    }
-    batch.add(kv);
-  }
-
-  public boolean isAllPointDeletes() {
-    return allPointDeletes;
-  }
-
-  public long getTimestamp() {
-    return this.timestamp;
-  }
-
-  public List<KeyValue> getKvs() {
-    return this.batch;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumns.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumns.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumns.java
deleted file mode 100644
index dffbe4e..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumns.java
+++ /dev/null
@@ -1,49 +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.hadoop.hbase.index.covered;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-
-/**
- * Manage a set of {@link ColumnReference}s for the {@link LocalTableState}.
- */
-public class CoveredColumns {
-
-  Set<ColumnReference> columns = new HashSet<ColumnReference>();
-
-  public Collection<? extends ColumnReference> findNonCoveredColumns(
-      Collection<? extends ColumnReference> columns2) {
-    List<ColumnReference> uncovered = new ArrayList<ColumnReference>();
-    for (ColumnReference column : columns2) {
-      if (!columns.contains(column)) {
-        uncovered.add(column);
-      }
-    }
-    return uncovered;
-  }
-
-  public void addColumn(ColumnReference column) {
-    this.columns.add(column);
-  }
-}


[09/15] Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexUpdate.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexUpdate.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexUpdate.java
new file mode 100644
index 0000000..e3132d6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexUpdate.java
@@ -0,0 +1,77 @@
+/*
+ * 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.phoenix.hbase.index.covered;
+
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.hbase.index.covered.update.ColumnTracker;
+
+/**
+ * Update to make to the index table.
+ */
+public class IndexUpdate {
+  Mutation update;
+  byte[] tableName;
+  ColumnTracker columns;
+
+  IndexUpdate(ColumnTracker tracker) {
+    this.columns = tracker;
+  }
+
+  public void setUpdate(Mutation p) {
+    this.update = p;
+  }
+
+  public void setTable(byte[] tableName) {
+    this.tableName = tableName;
+  }
+
+  public Mutation getUpdate() {
+    return update;
+  }
+
+  public byte[] getTableName() {
+    return tableName;
+  }
+
+  public ColumnTracker getIndexedColumns() {
+    return columns;
+  }
+
+  @Override
+  public String toString() {
+    return "IndexUpdate: \n\ttable - " + Bytes.toString(tableName) + "\n\tupdate: " + update
+        + "\n\tcolumns: " + columns;
+  }
+
+  public static IndexUpdate createIndexUpdateForTesting(ColumnTracker tracker, byte[] table, Put p) {
+    IndexUpdate update = new IndexUpdate(tracker);
+    update.setTable(table);
+    update.setUpdate(p);
+    return update;
+  }
+
+  /**
+   * @return <tt>true</tt> if the necessary state for a valid index update has been set.
+   */
+  public boolean isValid() {
+    return this.tableName != null && this.update != null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/KeyValueStore.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/KeyValueStore.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/KeyValueStore.java
new file mode 100644
index 0000000..39f9062
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/KeyValueStore.java
@@ -0,0 +1,33 @@
+/*
+ * 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.phoenix.hbase.index.covered;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+
+/**
+ * Store a collection of KeyValues in memory.
+ */
+public interface KeyValueStore {
+
+  public void add(KeyValue kv, boolean overwrite);
+
+  public KeyValueScanner getScanner();
+
+  public void rollback(KeyValue kv);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/LocalTableState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/LocalTableState.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/LocalTableState.java
new file mode 100644
index 0000000..cb21a00
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/LocalTableState.java
@@ -0,0 +1,242 @@
+/*
+ * 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.phoenix.hbase.index.covered;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.phoenix.hbase.index.covered.data.IndexMemStore;
+import org.apache.phoenix.hbase.index.covered.data.LocalHBaseState;
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.hbase.index.covered.update.ColumnTracker;
+import org.apache.phoenix.hbase.index.covered.update.IndexedColumnGroup;
+import org.apache.phoenix.hbase.index.scanner.Scanner;
+import org.apache.phoenix.hbase.index.scanner.ScannerBuilder;
+
+/**
+ * Manage the state of the HRegion's view of the table, for the single row.
+ * <p>
+ * Currently, this is a single-use object - you need to create a new one for each row that you need
+ * to manage. In the future, we could make this object reusable, but for the moment its easier to
+ * manage as a throw-away object.
+ * <p>
+ * This class is <b>not</b> thread-safe - it requires external synchronization is access
+ * concurrently.
+ */
+public class LocalTableState implements TableState {
+
+  private long ts;
+  private RegionCoprocessorEnvironment env;
+  private KeyValueStore memstore;
+  private LocalHBaseState table;
+  private Mutation update;
+  private Set<ColumnTracker> trackedColumns = new HashSet<ColumnTracker>();
+  private ScannerBuilder scannerBuilder;
+  private List<KeyValue> kvs = new ArrayList<KeyValue>();
+  private List<? extends IndexedColumnGroup> hints;
+  private CoveredColumns columnSet;
+
+  public LocalTableState(RegionCoprocessorEnvironment environment, LocalHBaseState table, Mutation update) {
+    this.env = environment;
+    this.table = table;
+    this.update = update;
+    this.memstore = new IndexMemStore();
+    this.scannerBuilder = new ScannerBuilder(memstore, update);
+    this.columnSet = new CoveredColumns();
+  }
+
+  public void addPendingUpdates(KeyValue... kvs) {
+    if (kvs == null) return;
+    addPendingUpdates(Arrays.asList(kvs));
+  }
+
+  public void addPendingUpdates(List<KeyValue> kvs) {
+    if(kvs == null) return;
+    setPendingUpdates(kvs);
+    addUpdate(kvs);
+  }
+
+  private void addUpdate(List<KeyValue> list) {
+    addUpdate(list, true);
+  }
+
+  private void addUpdate(List<KeyValue> list, boolean overwrite) {
+    if (list == null) return;
+    for (KeyValue kv : list) {
+      this.memstore.add(kv, overwrite);
+    }
+  }
+
+  @Override
+  public RegionCoprocessorEnvironment getEnvironment() {
+    return this.env;
+  }
+
+  @Override
+  public long getCurrentTimestamp() {
+    return this.ts;
+  }
+
+  @Override
+  public void setCurrentTimestamp(long timestamp) {
+    this.ts = timestamp;
+  }
+
+  public void resetTrackedColumns() {
+    this.trackedColumns.clear();
+  }
+
+  public Set<ColumnTracker> getTrackedColumns() {
+    return this.trackedColumns;
+  }
+
+  @Override
+  public Pair<Scanner, IndexUpdate> getIndexedColumnsTableState(
+      Collection<? extends ColumnReference> indexedColumns) throws IOException {
+    ensureLocalStateInitialized(indexedColumns);
+    // filter out things with a newer timestamp and track the column references to which it applies
+    ColumnTracker tracker = new ColumnTracker(indexedColumns);
+    synchronized (this.trackedColumns) {
+      // we haven't seen this set of columns before, so we need to create a new tracker
+      if (!this.trackedColumns.contains(tracker)) {
+        this.trackedColumns.add(tracker);
+      }
+    }
+
+    Scanner scanner =
+        this.scannerBuilder.buildIndexedColumnScanner(indexedColumns, tracker, ts);
+
+    return new Pair<Scanner, IndexUpdate>(scanner, new IndexUpdate(tracker));
+  }
+
+  /**
+   * Initialize the managed local state. Generally, this will only be called by
+   * {@link #getNonIndexedColumnsTableState(List)}, which is unlikely to be called concurrently from the outside.
+   * Even then, there is still fairly low contention as each new Put/Delete will have its own table
+   * state.
+   */
+  private synchronized void ensureLocalStateInitialized(
+      Collection<? extends ColumnReference> columns) throws IOException {
+    // check to see if we haven't initialized any columns yet
+    Collection<? extends ColumnReference> toCover = this.columnSet.findNonCoveredColumns(columns);
+    // we have all the columns loaded, so we are good to go.
+    if (toCover.isEmpty()) {
+      return;
+    }
+
+    // add the current state of the row
+    this.addUpdate(this.table.getCurrentRowState(update, toCover).list(), false);
+
+    // add the covered columns to the set
+    for (ColumnReference ref : toCover) {
+      this.columnSet.addColumn(ref);
+    }
+  }
+
+  @Override
+  public Map<String, byte[]> getUpdateAttributes() {
+    return this.update.getAttributesMap();
+  }
+
+  @Override
+  public byte[] getCurrentRowKey() {
+    return this.update.getRow();
+  }
+
+  public Result getCurrentRowState() {
+    KeyValueScanner scanner = this.memstore.getScanner();
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    while (scanner.peek() != null) {
+      try {
+        kvs.add(scanner.next());
+      } catch (IOException e) {
+        // this should never happen - something has gone terribly arwy if it has
+        throw new RuntimeException("Local MemStore threw IOException!");
+      }
+    }
+    return new Result(kvs);
+  }
+
+  /**
+   * Helper to add a {@link Mutation} to the values stored for the current row
+   * @param pendingUpdate update to apply
+   */
+  public void addUpdateForTesting(Mutation pendingUpdate) {
+    for (Map.Entry<byte[], List<KeyValue>> e : pendingUpdate.getFamilyMap().entrySet()) {
+      List<KeyValue> edits = e.getValue();
+      addUpdate(edits);
+    }
+  }
+
+  /**
+   * @param hints
+   */
+  public void setHints(List<? extends IndexedColumnGroup> hints) {
+    this.hints = hints;
+  }
+
+  @Override
+  public List<? extends IndexedColumnGroup> getIndexColumnHints() {
+    return this.hints;
+  }
+
+  @Override
+  public Collection<KeyValue> getPendingUpdate() {
+    return this.kvs;
+  }
+
+  /**
+   * Set the {@link KeyValue}s in the update for which we are currently building an index update,
+   * but don't actually apply them.
+   * @param update pending {@link KeyValue}s
+   */
+  public void setPendingUpdates(Collection<KeyValue> update) {
+    this.kvs.clear();
+    this.kvs.addAll(update);
+  }
+
+  /**
+   * Apply the {@link KeyValue}s set in {@link #setPendingUpdates(Collection)}.
+   */
+  public void applyPendingUpdates() {
+    this.addUpdate(kvs);
+  }
+
+  /**
+   * Rollback all the given values from the underlying state.
+   * @param values
+   */
+  public void rollback(Collection<KeyValue> values) {
+    for (KeyValue kv : values) {
+      this.memstore.rollback(kv);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/TableState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/TableState.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/TableState.java
new file mode 100644
index 0000000..4c4d0b0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/TableState.java
@@ -0,0 +1,116 @@
+/*
+ * 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.phoenix.hbase.index.covered;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.hbase.index.covered.update.IndexedColumnGroup;
+import org.apache.phoenix.hbase.index.scanner.Scanner;
+
+/**
+ * Interface for the current state of the table. This is generally going to be as of a timestamp - a
+ * view on the current state of the HBase table - so you don't have to worry about exposing too much
+ * information.
+ */
+public interface TableState {
+
+  // use this to get batch ids/ptable stuff
+  /**
+   * WARNING: messing with this can affect the indexing plumbing. Use with caution :)
+   * @return get the current environment in which this table lives.
+   */
+  public RegionCoprocessorEnvironment getEnvironment();
+
+  /**
+   * @return the current timestamp up-to-which we are releasing table state.
+   */
+  public long getCurrentTimestamp();
+
+  /**
+   * Set the current timestamp up to which the table should allow access to the underlying table.
+   * This overrides the timestamp view provided by the indexer - use with care!
+   * @param timestamp timestamp up to which the table should allow access.
+   */
+  public void setCurrentTimestamp(long timestamp);
+
+  /**
+   * @return the attributes attached to the current update (e.g. {@link Mutation}).
+   */
+  public Map<String, byte[]> getUpdateAttributes();
+
+  /**
+   * Get a scanner on the columns that are needed by the index.
+   * <p>
+   * The returned scanner is already pre-seeked to the first {@link KeyValue} that matches the given
+   * columns with a timestamp earlier than the timestamp to which the table is currently set (the
+   * current state of the table for which we need to build an update).
+   * <p>
+   * If none of the passed columns matches any of the columns in the pending update (as determined
+   * by {@link ColumnReference#matchesFamily(byte[])} and
+   * {@link ColumnReference#matchesQualifier(byte[])}, then an empty scanner will be returned. This
+   * is because it doesn't make sense to build index updates when there is no change in the table
+   * state for any of the columns you are indexing.
+   * <p>
+   * <i>NOTE:</i> This method should <b>not</b> be used during
+   * {@link IndexCodec#getIndexDeletes(TableState)} as the pending update will not yet have been
+   * applied - you are merely attempting to cleanup the current state and therefore do <i>not</i>
+   * need to track the indexed columns.
+   * <p>
+   * As a side-effect, we update a timestamp for the next-most-recent timestamp for the columns you
+   * request - you will never see a column with the timestamp we are tracking, but the next oldest
+   * timestamp for that column.
+   * @param indexedColumns the columns to that will be indexed
+   * @return an iterator over the columns and the {@link IndexUpdate} that should be passed back to
+   *         the builder. Even if no update is necessary for the requested columns, you still need
+   *         to return the {@link IndexUpdate}, just don't set the update for the
+   *         {@link IndexUpdate}.
+   * @throws IOException
+   */
+  Pair<Scanner, IndexUpdate> getIndexedColumnsTableState(
+      Collection<? extends ColumnReference> indexedColumns) throws IOException;
+
+  /**
+   * @return the row key for the current row for which we are building an index update.
+   */
+  byte[] getCurrentRowKey();
+
+  /**
+   * Get the 'hint' for the columns that were indexed last time for the same set of keyvalues.
+   * Generally, this will only be used when fixing up a 'back in time' put or delete as we need to
+   * fix up all the indexes that reference the changed columns.
+   * @return the hint the index columns that were queried on the last iteration for the changed
+   *         column
+   */
+  List<? extends IndexedColumnGroup> getIndexColumnHints();
+
+  /**
+   * Can be used to help the codec to determine which columns it should attempt to index.
+   * @return the keyvalues in the pending update to the table.
+   */
+  Collection<KeyValue> getPendingUpdate();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/IndexMemStore.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/IndexMemStore.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/IndexMemStore.java
new file mode 100644
index 0000000..3d114c5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/IndexMemStore.java
@@ -0,0 +1,331 @@
+/*
+ * 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.phoenix.hbase.index.covered.data;
+
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.SortedSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.KeyComparator;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.regionserver.IndexKeyValueSkipListSet;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+import org.apache.hadoop.hbase.regionserver.MemStore;
+import org.apache.hadoop.hbase.regionserver.NonLazyKeyValueScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.hbase.index.covered.KeyValueStore;
+import org.apache.phoenix.hbase.index.covered.LocalTableState;
+
+/**
+ * Like the HBase {@link MemStore}, but without all that extra work around maintaining snapshots and
+ * sizing (for right now). We still support the concurrent access (in case indexes are built in
+ * parallel).
+ * <p>
+ * 
+ We basically wrap a KeyValueSkipListSet, just like a regular MemStore, except we are:
+ * <ol>
+ *  <li>not dealing with
+ *    <ul>
+ *      <li>space considerations</li>
+ *      <li>a snapshot set</li>
+ *    </ul>
+ *  </li>
+ *  <li>ignoring memstore timestamps in favor of deciding when we want to overwrite keys based on how
+ *    we obtain them</li>
+ *   <li>ignoring time range updates (so 
+ *    {@link KeyValueScanner#shouldUseScanner(Scan, SortedSet, long)} isn't supported from 
+ *    {@link #getScanner()}).</li>
+ * </ol>
+ * <p>
+ * We can ignore the memstore timestamps because we know that anything we get from the local region
+ * is going to be MVCC visible - so it should just go in. However, we also want overwrite any
+ * existing state with our pending write that we are indexing, so that needs to clobber the KVs we
+ * get from the HRegion. This got really messy with a regular memstore as each KV from the MemStore
+ * frequently has a higher MemStoreTS, but we can't just up the pending KVs' MemStoreTs because a
+ * memstore relies on the MVCC readpoint, which generally is less than {@link Long#MAX_VALUE}.
+ * <p>
+ * By realizing that we don't need the snapshot or space requirements, we can go much faster than
+ * the previous implementation. Further, by being smart about how we manage the KVs, we can drop the
+ * extra object creation we were doing to wrap the pending KVs (which we did previously to ensure
+ * they sorted before the ones we got from the HRegion). We overwrite {@link KeyValue}s when we add
+ * them from external sources {@link #add(KeyValue, boolean)}, but then don't overwrite existing
+ * keyvalues when read them from the underlying table (because pending keyvalues should always
+ * overwrite current ones) - this logic is all contained in LocalTableState.
+ * @see LocalTableState
+ */
+public class IndexMemStore implements KeyValueStore {
+
+  private static final Log LOG = LogFactory.getLog(IndexMemStore.class);
+  private IndexKeyValueSkipListSet kvset;
+  private Comparator<KeyValue> comparator;
+
+  /**
+   * Compare two {@link KeyValue}s based only on their row keys. Similar to the standard
+   * {@link KeyValue#COMPARATOR}, but doesn't take into consideration the memstore timestamps. We
+   * instead manage which KeyValue to retain based on how its loaded here
+   */
+  public static final Comparator<KeyValue> COMPARATOR = new Comparator<KeyValue>() {
+
+    private final KeyComparator rawcomparator = new KeyComparator();
+
+    @Override
+    public int compare(final KeyValue left, final KeyValue right) {
+      return rawcomparator.compare(left.getBuffer(), left.getOffset() + KeyValue.ROW_OFFSET,
+        left.getKeyLength(), right.getBuffer(), right.getOffset() + KeyValue.ROW_OFFSET,
+        right.getKeyLength());
+    }
+  };
+
+  public IndexMemStore() {
+    this(COMPARATOR);
+  }
+
+  /**
+   * Create a store with the given comparator. This comparator is used to determine both sort order
+   * <b>as well as equality of {@link KeyValue}s</b>.
+   * <p>
+   * Exposed for subclassing/testing.
+   * @param comparator to use
+   */
+  IndexMemStore(Comparator<KeyValue> comparator) {
+    this.comparator = comparator;
+    this.kvset = IndexKeyValueSkipListSet.create(comparator);
+  }
+
+  @Override
+  public void add(KeyValue kv, boolean overwrite) {
+    if (LOG.isDebugEnabled()) {
+      LOG.info("Inserting: " + toString(kv));
+    }
+    // if overwriting, we will always update
+    if (!overwrite) {
+      // null if there was no previous value, so we added the kv
+      kvset.putIfAbsent(kv);
+    } else {
+      kvset.add(kv);
+    }
+
+    if (LOG.isTraceEnabled()) {
+      dump();
+    }
+  }
+
+  private void dump() {
+    LOG.trace("Current kv state:\n");
+    for (KeyValue kv : this.kvset) {
+      LOG.trace("KV: " + toString(kv));
+    }
+    LOG.trace("========== END MemStore Dump ==================\n");
+  }
+
+  private String toString(KeyValue kv) {
+    return kv.toString() + "/value=" + Bytes.toString(kv.getValue());
+  }
+
+  @Override
+  public void rollback(KeyValue kv) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Rolling back: " + toString(kv));
+    }
+    // If the key is in the store, delete it
+    this.kvset.remove(kv);
+    if (LOG.isTraceEnabled()) {
+      dump();
+    }
+  }
+
+  @Override
+  public KeyValueScanner getScanner() {
+    return new MemStoreScanner();
+  }
+
+  /*
+   * MemStoreScanner implements the KeyValueScanner. It lets the caller scan the contents of a
+   * memstore -- both current map and snapshot. This behaves as if it were a real scanner but does
+   * not maintain position.
+   */
+  // This class is adapted from org.apache.hadoop.hbase.MemStore.MemStoreScanner, HBase 0.94.12
+  // It does basically the same thing as the MemStoreScanner, but it only keeps track of a single
+  // set, rather than a primary and a secondary set of KeyValues.
+  protected class MemStoreScanner extends NonLazyKeyValueScanner {
+    // Next row information for the set
+    private KeyValue nextRow = null;
+
+    // last iterated KVs for kvset and snapshot (to restore iterator state after reseek)
+    private KeyValue kvsetItRow = null;
+
+    // iterator based scanning.
+    private Iterator<KeyValue> kvsetIt;
+
+    // The kvset at the time of creating this scanner
+    volatile IndexKeyValueSkipListSet kvsetAtCreation;
+
+    MemStoreScanner() {
+      super();
+      kvsetAtCreation = kvset;
+    }
+
+    private KeyValue getNext(Iterator<KeyValue> it) {
+      // in the original implementation we cared about the current thread's readpoint from MVCC.
+      // However, we don't need to worry here because everything the index can see, is also visible
+      // to the client (or is the pending primary table update, so it will be once the index is
+      // written, so it might as well be).
+      KeyValue v = null;
+      try {
+        while (it.hasNext()) {
+          v = it.next();
+          return v;
+        }
+
+        return null;
+      } finally {
+        if (v != null) {
+          kvsetItRow = v;
+        }
+      }
+    }
+
+    /**
+     * Set the scanner at the seek key. Must be called only once: there is no thread safety between
+     * the scanner and the memStore.
+     * @param key seek value
+     * @return false if the key is null or if there is no data
+     */
+    @Override
+    public synchronized boolean seek(KeyValue key) {
+      if (key == null) {
+        close();
+        return false;
+      }
+
+      // kvset and snapshot will never be null.
+      // if tailSet can't find anything, SortedSet is empty (not null).
+      kvsetIt = kvsetAtCreation.tailSet(key).iterator();
+      kvsetItRow = null;
+
+      return seekInSubLists(key);
+    }
+
+    /**
+     * (Re)initialize the iterators after a seek or a reseek.
+     */
+    private synchronized boolean seekInSubLists(KeyValue key) {
+      nextRow = getNext(kvsetIt);
+      return nextRow != null;
+    }
+
+    /**
+     * Move forward on the sub-lists set previously by seek.
+     * @param key seek value (should be non-null)
+     * @return true if there is at least one KV to read, false otherwise
+     */
+    @Override
+    public synchronized boolean reseek(KeyValue key) {
+      /*
+       * See HBASE-4195 & HBASE-3855 & HBASE-6591 for the background on this implementation. This
+       * code is executed concurrently with flush and puts, without locks. Two points must be known
+       * when working on this code: 1) It's not possible to use the 'kvTail' and 'snapshot'
+       * variables, as they are modified during a flush. 2) The ideal implementation for performance
+       * would use the sub skip list implicitly pointed by the iterators 'kvsetIt' and 'snapshotIt'.
+       * Unfortunately the Java API does not offer a method to get it. So we remember the last keys
+       * we iterated to and restore the reseeked set to at least that point.
+       */
+
+      kvsetIt = kvsetAtCreation.tailSet(getHighest(key, kvsetItRow)).iterator();
+      return seekInSubLists(key);
+    }
+
+    /*
+     * Returns the higher of the two key values, or null if they are both null. This uses
+     * comparator.compare() to compare the KeyValue using the memstore comparator.
+     */
+    private KeyValue getHighest(KeyValue first, KeyValue second) {
+      if (first == null && second == null) {
+        return null;
+      }
+      if (first != null && second != null) {
+        int compare = comparator.compare(first, second);
+        return (compare > 0 ? first : second);
+      }
+      return (first != null ? first : second);
+    }
+
+    @Override
+    public synchronized KeyValue peek() {
+      // DebugPrint.println(" MS@" + hashCode() + " peek = " + getLowest());
+      return nextRow;
+    }
+
+    @Override
+    public synchronized KeyValue next() {
+      if (nextRow == null) {
+        return null;
+      }
+
+      final KeyValue ret = nextRow;
+
+      // Advance the iterators
+      nextRow = getNext(kvsetIt);
+
+      return ret;
+    }
+
+    @Override
+    public synchronized void close() {
+      this.nextRow = null;
+      this.kvsetIt = null;
+      this.kvsetItRow = null;
+    }
+
+    /**
+     * MemStoreScanner returns max value as sequence id because it will always have the latest data
+     * among all files.
+     */
+    @Override
+    public long getSequenceID() {
+      return Long.MAX_VALUE;
+    }
+
+    @Override
+    public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns, long oldestUnexpiredTS) {
+      throw new UnsupportedOperationException(this.getClass().getName()
+          + " doesn't support checking to see if it should use a scanner!");
+    }
+
+    /*
+    @Override
+    public boolean backwardSeek(KeyValue arg0) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean seekToLastRow() throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean seekToPreviousRow(KeyValue arg0) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+    */
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LazyValueGetter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LazyValueGetter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LazyValueGetter.java
new file mode 100644
index 0000000..43c4028
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LazyValueGetter.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.hbase.index.covered.data;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+import org.apache.phoenix.hbase.index.ValueGetter;
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.hbase.index.scanner.Scanner;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * {@link ValueGetter} that uses lazy initialization to get the value for the given
+ * {@link ColumnReference}. Once stored, the mapping for that reference is retained.
+ */
+public class LazyValueGetter implements ValueGetter {
+
+  private Scanner scan;
+  private volatile Map<ColumnReference, ImmutableBytesPtr> values;
+  private byte[] row;
+  
+  /**
+   * Back the getter with a {@link Scanner} to actually access the local data.
+   * @param scan backing scanner
+   * @param currentRow row key for the row to seek in the scanner
+   */
+  public LazyValueGetter(Scanner scan, byte[] currentRow) {
+    this.scan = scan;
+    this.row = currentRow;
+  }
+
+  @Override
+  public ImmutableBytesPtr getLatestValue(ColumnReference ref) throws IOException {
+    // ensure we have a backing map
+    if (values == null) {
+      synchronized (this) {
+        values = Collections.synchronizedMap(new HashMap<ColumnReference, ImmutableBytesPtr>());
+      }
+    }
+
+    // check the value in the map
+    ImmutableBytesPtr value = values.get(ref);
+    if (value == null) {
+      value = get(ref);
+      values.put(ref, value);
+    }
+
+    return value;
+  }
+
+  /**
+   * @param ref
+   * @return the first value on the scanner for the given column
+   */
+  private ImmutableBytesPtr get(ColumnReference ref) throws IOException {
+    KeyValue first = ref.getFirstKeyValueForRow(row);
+    if (!scan.seek(first)) {
+      return null;
+    }
+    // there is a next value - we only care about the current value, so we can just snag that
+    KeyValue next = scan.next();
+    if (ref.matches(next)) {
+      return new ImmutableBytesPtr(next.getBuffer(), next.getValueOffset(), next.getValueLength());
+    }
+    return null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalHBaseState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalHBaseState.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalHBaseState.java
new file mode 100644
index 0000000..6d20c18
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalHBaseState.java
@@ -0,0 +1,47 @@
+/*
+ * 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.phoenix.hbase.index.covered.data;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Result;
+
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+
+/**
+ * Access the current state of the row in the local HBase table, given a mutation
+ */
+public interface LocalHBaseState {
+
+  /**
+   * @param m mutation for which we should get the current table state
+   * @param toCover all the columns the current row state needs to cover; hint the underlying lookup
+   *          to save getting all the columns for the row
+   * @return the full state of the given row. Includes all current versions (even if they are not
+   *         usually visible to the client (unless they are also doing a raw scan)). Never returns a
+   *         <tt>null</tt> {@link Result} - instead, when there is not data for the row, returns a
+   *         {@link Result} with no stored {@link KeyValue}s.
+   * @throws IOException if there is an issue reading the row
+   */
+  public Result getCurrentRowState(Mutation m, Collection<? extends ColumnReference> toCover)
+      throws IOException;
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalTable.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalTable.java
new file mode 100644
index 0000000..f2f247e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalTable.java
@@ -0,0 +1,72 @@
+/*
+ * 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.phoenix.hbase.index.covered.data;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
+
+/**
+ * Wrapper around a lazily instantiated, local HTable.
+ * <p>
+ * Previously, we had used various row and batch caches. However, this ends up being very
+ * complicated when attempting manage updating and invalidating the cache with no real gain as any
+ * row accessed multiple times will likely be in HBase's block cache, invalidating any extra caching
+ * we are doing here. In the end, its simpler and about as efficient to just get the current state
+ * of the row from HBase and let HBase manage caching the row from disk on its own.
+ */
+public class LocalTable implements LocalHBaseState {
+
+  private RegionCoprocessorEnvironment env;
+
+  public LocalTable(RegionCoprocessorEnvironment env) {
+    this.env = env;
+  }
+
+  @Override
+  public Result getCurrentRowState(Mutation m, Collection<? extends ColumnReference> columns)
+      throws IOException {
+    byte[] row = m.getRow();
+    // need to use a scan here so we can get raw state, which Get doesn't provide.
+    Scan s = IndexManagementUtil.newLocalStateScan(Collections.singletonList(columns));
+    s.setStartRow(row);
+    s.setStopRow(row);
+    HRegion region = this.env.getRegion();
+    RegionScanner scanner = region.getScanner(s);
+    List<KeyValue> kvs = new ArrayList<KeyValue>(1);
+    boolean more = scanner.next(kvs);
+    assert !more : "Got more than one result when scanning" + " a single row in the primary table!";
+
+    Result r = new Result(kvs);
+    scanner.close();
+    return r;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/ColumnGroup.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/ColumnGroup.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/ColumnGroup.java
new file mode 100644
index 0000000..ba2b092
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/ColumnGroup.java
@@ -0,0 +1,112 @@
+/*
+ * 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.phoenix.hbase.index.covered.example;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+/**
+ * A collection of {@link CoveredColumn}s that should be included in a covered index.
+ */
+public class ColumnGroup implements Iterable<CoveredColumn> {
+
+  private List<CoveredColumn> columns = new ArrayList<CoveredColumn>();
+  private String table;
+
+  public ColumnGroup(String tableName) {
+    this.table = tableName;
+  }
+
+  public void add(CoveredColumn column) {
+    this.columns.add(column);
+  }
+
+  public String getTable() {
+    return table;
+  }
+
+  /**
+   * Check to see if any {@link CoveredColumn} in <tt>this</tt> matches the given family
+   * @param family to check
+   * @return <tt>true</tt> if any column covers the family
+   */
+  public boolean matches(String family) {
+    for (CoveredColumn column : columns) {
+      if (column.matchesFamily(family)) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /**
+   * Check to see if any column matches the family/qualifier pair
+   * @param family family to match against
+   * @param qualifier qualifier to match, can be <tt>null</tt>, in which case we match all
+   *          qualifiers
+   * @return <tt>true</tt> if any column matches, <tt>false</tt> otherwise
+   */
+  public boolean matches(byte[] family, byte[] qualifier) {
+    // families are always printable characters
+    String fam = Bytes.toString(family);
+    for (CoveredColumn column : columns) {
+      if (column.matchesFamily(fam)) {
+        // check the qualifier
+          if (column.matchesQualifier(qualifier)) {
+            return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * @return the number of columns in the group
+   */
+  public int size() {
+    return this.columns.size();
+  }
+
+  @Override
+  public Iterator<CoveredColumn> iterator() {
+    return columns.iterator();
+  }
+
+  /**
+   * @param index index of the column to get
+   * @return the specified column
+   */
+  public CoveredColumn getColumnForTesting(int index) {
+    return this.columns.get(index);
+  }
+
+  @Override
+  public String toString() {
+    return "ColumnGroup - table: " + table + ", columns: " + columns;
+  }
+
+  public List<CoveredColumn> getColumns() {
+    return this.columns;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumn.java
new file mode 100644
index 0000000..5c6989f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumn.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.hbase.index.covered.example;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+
+/**
+ * A single Column (either a Column Family or a full Family:Qualifier pair) in a {@link ColumnGroup}
+ * . If no column qualifier is specified (null), matches all known qualifiers of the family.
+ */
+public class CoveredColumn extends ColumnReference {
+
+  public static final String SEPARATOR = ":";
+  String familyString;
+  private final int hashCode;
+
+  public CoveredColumn(byte[] family, byte[] qualifier){
+    this(Bytes.toString(family), qualifier);
+  }
+
+  public CoveredColumn(String family, byte[] qualifier) {
+    super(Bytes.toBytes(family), qualifier == null ? ColumnReference.ALL_QUALIFIERS : qualifier);
+    this.familyString = family;
+    this.hashCode = calcHashCode(family, qualifier);
+  }
+
+  public static CoveredColumn parse(String spec) {
+    int sep = spec.indexOf(SEPARATOR);
+    if (sep < 0) {
+      throw new IllegalArgumentException(spec + " is not a valid specifier!");
+    }
+    String family = spec.substring(0, sep);
+    String qual = spec.substring(sep + 1);
+    byte[] column = qual.length() == 0 ? null : Bytes.toBytes(qual);
+    return new CoveredColumn(family, column);
+  }
+
+  public String serialize() {
+    return CoveredColumn.serialize(familyString, qualifier);
+  }
+
+  public static String serialize(String first, byte[] second) {
+    String nextValue = first + CoveredColumn.SEPARATOR;
+    if (second != null) {
+      nextValue += Bytes.toString(second);
+    }
+    return nextValue;
+  }
+
+  /**
+   * @param family2 to check
+   * @return <tt>true</tt> if the passed family matches the family this column covers
+   */
+  public boolean matchesFamily(String family2) {
+    return this.familyString.equals(family2);
+  }
+
+  @Override
+  public int hashCode() {
+    return hashCode;
+  }
+
+  private static int calcHashCode(String familyString, byte[] qualifier) {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + familyString.hashCode();
+    if (qualifier != null) {
+      result = prime * result + Bytes.hashCode(qualifier);
+    }
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) return true;
+    if (!super.equals(obj)) return false;
+    if (getClass() != obj.getClass()) return false;
+    CoveredColumn other = (CoveredColumn) obj;
+    if (hashCode != other.hashCode) return false;
+    if (!familyString.equals(other.familyString)) return false;
+    return Bytes.equals(qualifier, other.qualifier);
+  }
+
+  @Override
+  public String toString() {
+    String qualString = qualifier == null ? "null" : Bytes.toString(qualifier);
+    return "CoveredColumn:[" + familyString + ":" + qualString + "]";
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexCodec.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexCodec.java
new file mode 100644
index 0000000..691ab08
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexCodec.java
@@ -0,0 +1,367 @@
+/**
+ * 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.phoenix.hbase.index.covered.example;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map.Entry;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.hbase.index.covered.IndexUpdate;
+import org.apache.phoenix.hbase.index.covered.TableState;
+import org.apache.phoenix.hbase.index.scanner.Scanner;
+import org.apache.phoenix.index.BaseIndexCodec;
+
+/**
+ *
+ */
+public class CoveredColumnIndexCodec extends BaseIndexCodec {
+
+  private static final byte[] EMPTY_BYTES = new byte[0];
+  public static final byte[] INDEX_ROW_COLUMN_FAMILY = Bytes.toBytes("INDEXED_COLUMNS");
+
+  private List<ColumnGroup> groups;
+
+  /**
+   * @param groups to initialize the codec with
+   * @return an instance that is initialized with the given {@link ColumnGroup}s, for testing
+   *         purposes
+   */
+  public static CoveredColumnIndexCodec getCodecForTesting(List<ColumnGroup> groups) {
+    CoveredColumnIndexCodec codec = new CoveredColumnIndexCodec();
+    codec.groups = Lists.newArrayList(groups);
+    return codec;
+  }
+
+  @Override
+  public void initialize(RegionCoprocessorEnvironment env) {
+    groups = CoveredColumnIndexSpecifierBuilder.getColumns(env.getConfiguration());
+  }
+
+  @Override
+  public Iterable<IndexUpdate> getIndexUpserts(TableState state) {
+    List<IndexUpdate> updates = new ArrayList<IndexUpdate>();
+    for (ColumnGroup group : groups) {
+      IndexUpdate update = getIndexUpdateForGroup(group, state);
+      updates.add(update);
+    }
+    return updates;
+  }
+
+  /**
+   * @param group
+   * @param state
+   * @return the update that should be made to the table
+   */
+  private IndexUpdate getIndexUpdateForGroup(ColumnGroup group, TableState state) {
+    List<CoveredColumn> refs = group.getColumns();
+    try {
+      Pair<Scanner, IndexUpdate> stateInfo = state.getIndexedColumnsTableState(refs);
+      Scanner kvs = stateInfo.getFirst();
+      Pair<Integer, List<ColumnEntry>> columns =
+          getNextEntries(refs, kvs, state.getCurrentRowKey());
+      // make sure we close the scanner
+      kvs.close();
+      if (columns.getFirst().intValue() == 0) {
+        return stateInfo.getSecond();
+      }
+      // have all the column entries, so just turn it into a Delete for the row
+      // convert the entries to the needed values
+      byte[] rowKey =
+          composeRowKey(state.getCurrentRowKey(), columns.getFirst(), columns.getSecond());
+      Put p = new Put(rowKey, state.getCurrentTimestamp());
+      // add the columns to the put
+      addColumnsToPut(p, columns.getSecond());
+
+      // update the index info
+      IndexUpdate update = stateInfo.getSecond();
+      update.setTable(Bytes.toBytes(group.getTable()));
+      update.setUpdate(p);
+      return update;
+    } catch (IOException e) {
+      throw new RuntimeException("Unexpected exception when getting state for columns: " + refs);
+    }
+  }
+
+  private static void addColumnsToPut(Put indexInsert, List<ColumnEntry> columns) {
+    // add each of the corresponding families to the put
+    int count = 0;
+    for (ColumnEntry column : columns) {
+      indexInsert.add(INDEX_ROW_COLUMN_FAMILY,
+        ArrayUtils.addAll(Bytes.toBytes(count++), toIndexQualifier(column.ref)), null);
+    }
+  }
+
+  private static byte[] toIndexQualifier(CoveredColumn column) {
+    return ArrayUtils.addAll(Bytes.toBytes(column.familyString + CoveredColumn.SEPARATOR),
+      column.getQualifier());
+  }
+
+  @Override
+  public Iterable<IndexUpdate> getIndexDeletes(TableState state) {
+    List<IndexUpdate> deletes = new ArrayList<IndexUpdate>();
+    for (ColumnGroup group : groups) {
+      deletes.add(getDeleteForGroup(group, state));
+    }
+    return deletes;
+  }
+
+
+  /**
+   * Get all the deletes necessary for a group of columns - logically, the cleanup the index table
+   * for a given index.
+   * @param group index information
+   * @return the cleanup for the given index, or <tt>null</tt> if no cleanup is necessary
+   */
+  private IndexUpdate getDeleteForGroup(ColumnGroup group, TableState state) {
+    List<CoveredColumn> refs = group.getColumns();
+    try {
+      Pair<Scanner, IndexUpdate> kvs = state.getIndexedColumnsTableState(refs);
+      Pair<Integer, List<ColumnEntry>> columns =
+          getNextEntries(refs, kvs.getFirst(), state.getCurrentRowKey());
+      // make sure we close the scanner reference
+      kvs.getFirst().close();
+      // no change, just return the passed update
+      if (columns.getFirst() == 0) {
+        return kvs.getSecond();
+      }
+      // have all the column entries, so just turn it into a Delete for the row
+      // convert the entries to the needed values
+      byte[] rowKey =
+          composeRowKey(state.getCurrentRowKey(), columns.getFirst(), columns.getSecond());
+      Delete d = new Delete(rowKey);
+      d.setTimestamp(state.getCurrentTimestamp());
+      IndexUpdate update = kvs.getSecond();
+      update.setUpdate(d);
+      update.setTable(Bytes.toBytes(group.getTable()));
+      return update;
+    } catch (IOException e) {
+      throw new RuntimeException("Unexpected exception when getting state for columns: " + refs);
+    }
+  }
+
+  /**
+   * Get the next batch of primary table values for the given columns
+   * @param refs columns to match against
+   * @param state
+   * @return the total length of all values found and the entries to add for the index
+   */
+  private Pair<Integer, List<ColumnEntry>> getNextEntries(List<CoveredColumn> refs, Scanner kvs,
+      byte[] currentRow) throws IOException {
+    int totalValueLength = 0;
+    List<ColumnEntry> entries = new ArrayList<ColumnEntry>(refs.size());
+
+    // pull out the latest state for each column reference, in order
+    for (CoveredColumn ref : refs) {
+      KeyValue first = ref.getFirstKeyValueForRow(currentRow);
+      if (!kvs.seek(first)) {
+        // no more keys, so add a null value
+        entries.add(new ColumnEntry(null, ref));
+        continue;
+      }
+      // there is a next value - we only care about the current value, so we can just snag that
+      KeyValue next = kvs.next();
+      if (ref.matchesFamily(next.getFamily()) && ref.matchesQualifier(next.getQualifier())) {
+        byte[] v = next.getValue();
+        totalValueLength += v.length;
+        entries.add(new ColumnEntry(v, ref));
+      } else {
+        // this first one didn't match at all, so we have to put in a null entry
+        entries.add(new ColumnEntry(null, ref));
+        continue;
+      }
+      // here's where is gets a little tricky - we either need to decide if we should continue
+      // adding entries (matches all qualifiers) or if we are done (matches a single qualifier)
+      if (!ref.allColumns()) {
+        continue;
+      }
+      // matches all columns, so we need to iterate until we hit the next column with the same
+      // family as the current key
+      byte[] lastQual = next.getQualifier();
+      byte[] nextQual = null;
+      while ((next = kvs.next()) != null) {
+        // different family, done with this column
+        if (!ref.matchesFamily(next.getFamily())) {
+          break;
+        }
+        nextQual = next.getQualifier();
+        // we are still on the same qualifier - skip it, since we already added a column for it
+        if (Arrays.equals(lastQual, nextQual)) {
+          continue;
+        }
+        // this must match the qualifier since its an all-qualifiers specifier, so we add it
+        byte[] v = next.getValue();
+        totalValueLength += v.length;
+        entries.add(new ColumnEntry(v, ref));
+        // update the last qualifier to check against
+        lastQual = nextQual;
+      }
+    }
+    return new Pair<Integer, List<ColumnEntry>>(totalValueLength, entries);
+  }
+
+  static class ColumnEntry {
+    byte[] value = EMPTY_BYTES;
+    CoveredColumn ref;
+
+    public ColumnEntry(byte[] value, CoveredColumn ref) {
+      this.value = value == null ? EMPTY_BYTES : value;
+      this.ref = ref;
+    }
+  }
+
+  /**
+   * Compose the final index row key.
+   * <p>
+   * This is faster than adding each value independently as we can just build a single a array and
+   * copy everything over once.
+   * @param pk primary key of the original row
+   * @param length total number of bytes of all the values that should be added
+   * @param values to use when building the key
+   * @return
+   */
+  static byte[] composeRowKey(byte[] pk, int length, List<ColumnEntry> values) {
+    // now build up expected row key, each of the values, in order, followed by the PK and then some
+    // info about lengths so we can deserialize each value
+    byte[] output = new byte[length + pk.length];
+    int pos = 0;
+    int[] lengths = new int[values.size()];
+    int i = 0;
+    for (ColumnEntry entry : values) {
+      byte[] v = entry.value;
+      // skip doing the copy attempt, if we don't need to
+      if (v.length != 0) {
+        System.arraycopy(v, 0, output, pos, v.length);
+        pos += v.length;
+      }
+      lengths[i++] = v.length;
+    }
+
+    // add the primary key to the end of the row key
+    System.arraycopy(pk, 0, output, pos, pk.length);
+
+    // add the lengths as suffixes so we can deserialize the elements again
+    for (int l : lengths) {
+      output = ArrayUtils.addAll(output, Bytes.toBytes(l));
+    }
+
+    // and the last integer is the number of values
+    return ArrayUtils.addAll(output, Bytes.toBytes(values.size()));
+  }
+
+  /**
+   * Essentially a short-cut from building a {@link Put}.
+   * @param pk row key
+   * @param timestamp timestamp of all the keyvalues
+   * @param values expected value--column pair
+   * @return a keyvalues that the index contains for a given row at a timestamp with the given value
+   *         -- column pairs.
+   */
+  public static List<KeyValue> getIndexKeyValueForTesting(byte[] pk, long timestamp,
+      List<Pair<byte[], CoveredColumn>> values) {
+  
+    int length = 0;
+    List<ColumnEntry> expected = new ArrayList<ColumnEntry>(values.size());
+    for (Pair<byte[], CoveredColumn> value : values) {
+      ColumnEntry entry = new ColumnEntry(value.getFirst(), value.getSecond());
+      length += value.getFirst().length;
+      expected.add(entry);
+    }
+  
+    byte[] rowKey = CoveredColumnIndexCodec.composeRowKey(pk, length, expected);
+    Put p = new Put(rowKey, timestamp);
+    CoveredColumnIndexCodec.addColumnsToPut(p, expected);
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    for (Entry<byte[], List<KeyValue>> entry : p.getFamilyMap().entrySet()) {
+      kvs.addAll(entry.getValue());
+    }
+  
+    return kvs;
+  }
+
+  public static List<byte[]> getValues(byte[] bytes) {
+    // get the total number of keys in the bytes
+    int keyCount = CoveredColumnIndexCodec.getPreviousInteger(bytes, bytes.length);
+    List<byte[]> keys = new ArrayList<byte[]>(keyCount);
+    int[] lengths = new int[keyCount];
+    int lengthPos = keyCount - 1;
+    int pos = bytes.length - Bytes.SIZEOF_INT;
+    // figure out the length of each key
+    for (int i = 0; i < keyCount; i++) {
+      lengths[lengthPos--] = CoveredColumnIndexCodec.getPreviousInteger(bytes, pos);
+      pos -= Bytes.SIZEOF_INT;
+    }
+
+    int current = 0;
+    for (int length : lengths) {
+      byte[] key = Arrays.copyOfRange(bytes, current, current + length);
+      keys.add(key);
+      current += length;
+    }
+
+    return keys;
+  }
+
+  /**
+   * Read an integer from the preceding {@value Bytes#SIZEOF_INT} bytes
+   * @param bytes array to read from
+   * @param start start point, backwards from which to read. For example, if specifying "25", we
+   *          would try to read an integer from 21 -> 25
+   * @return an integer from the proceeding {@value Bytes#SIZEOF_INT} bytes, if it exists.
+   */
+  private static int getPreviousInteger(byte[] bytes, int start) {
+    return Bytes.toInt(bytes, start - Bytes.SIZEOF_INT);
+  }
+
+  /**
+   * Check to see if an row key just contains a list of null values.
+   * @param bytes row key to examine
+   * @return <tt>true</tt> if all the values are zero-length, <tt>false</tt> otherwise
+   */
+  public static boolean checkRowKeyForAllNulls(byte[] bytes) {
+    int keyCount = CoveredColumnIndexCodec.getPreviousInteger(bytes, bytes.length);
+    int pos = bytes.length - Bytes.SIZEOF_INT;
+    for (int i = 0; i < keyCount; i++) {
+      int next = CoveredColumnIndexCodec.getPreviousInteger(bytes, pos);
+      if (next > 0) {
+        return false;
+      }
+      pos -= Bytes.SIZEOF_INT;
+    }
+
+    return true;
+  }
+
+  @Override
+  public boolean isEnabled(Mutation m) {
+    // this could be a bit smarter, looking at the groups for the mutation, but we leave it at this
+    // simple check for the moment.
+    return groups.size() > 0;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java
new file mode 100644
index 0000000..9fcd5f3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.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.phoenix.hbase.index.covered.example;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HTableDescriptor;
+
+import org.apache.phoenix.hbase.index.Indexer;
+import org.apache.phoenix.hbase.index.covered.CoveredColumnsIndexBuilder;
+import org.apache.phoenix.hbase.index.covered.IndexCodec;
+
+/**
+ * Helper to build the configuration for the {@link CoveredColumnIndexer}.
+ * <p>
+ * This class is NOT thread-safe; all concurrent access must be managed externally.
+ */
+public class CoveredColumnIndexSpecifierBuilder {
+
+  private static final String INDEX_TO_TABLE_CONF_PREFX = "hbase.index.covered.";
+  // number of index 'groups'. Each group represents a set of 'joined' columns. The data stored with
+  // each joined column are either just the columns in the group or all the most recent data in the
+  // row (a fully covered index).
+  private static final String COUNT = ".count";
+  private static final String INDEX_GROUPS_COUNT_KEY = INDEX_TO_TABLE_CONF_PREFX + ".groups" + COUNT;
+  private static final String INDEX_GROUP_PREFIX = INDEX_TO_TABLE_CONF_PREFX + "group.";
+  private static final String INDEX_GROUP_COVERAGE_SUFFIX = ".columns";
+  private static final String TABLE_SUFFIX = ".table";
+
+  // right now, we don't support this should be easy enough to add later
+  // private static final String INDEX_GROUP_FULLY_COVERED = ".covered";
+
+  List<ColumnGroup> groups = new ArrayList<ColumnGroup>();
+  private Map<String, String> specs = new HashMap<String, String>();
+
+  /**
+   * Add a group of columns to index
+   * @param columns Pairs of cf:cq (full specification of a column) to index
+   * @return the index of the group. This can be used to remove or modify the group via
+   *         {@link #remove(int)} or {@link #get(int)}, any time before building
+   */
+  public int addIndexGroup(ColumnGroup columns) {
+    if (columns == null || columns.size() == 0) {
+      throw new IllegalArgumentException("Must specify some columns to index!");
+    }
+    int size = this.groups.size();
+    this.groups.add(columns);
+    return size;
+  }
+
+  public void remove(int i) {
+    this.groups.remove(i);
+  }
+
+  public ColumnGroup get(int i) {
+    return this.groups.get(i);
+  }
+
+  /**
+   * Clear the stored {@link ColumnGroup}s for resuse.
+   */
+  public void reset() {
+    this.groups.clear();
+  }
+
+  Map<String, String> convertToMap() {
+    int total = this.groups.size();
+    // hbase.index.covered.groups = i
+    specs.put(INDEX_GROUPS_COUNT_KEY, Integer.toString(total));
+
+    int i = 0;
+    for (ColumnGroup group : groups) {
+      addIndexGroupToSpecs(specs, group, i++);
+    }
+
+    return specs;
+  }
+
+  /**
+   * @param specs
+   * @param columns
+   * @param index
+   */
+  private void addIndexGroupToSpecs(Map<String, String> specs, ColumnGroup columns, int index) {
+    // hbase.index.covered.group.<i>
+    String prefix = INDEX_GROUP_PREFIX + Integer.toString(index);
+
+    // set the table to which the group writes
+    // hbase.index.covered.group.<i>.table
+    specs.put(prefix + TABLE_SUFFIX, columns.getTable());
+    
+    // a different key for each column in the group
+    // hbase.index.covered.group.<i>.columns
+    String columnPrefix = prefix + INDEX_GROUP_COVERAGE_SUFFIX;
+    // hbase.index.covered.group.<i>.columns.count = <j>
+    String columnsSizeKey = columnPrefix + COUNT;
+    specs.put(columnsSizeKey, Integer.toString(columns.size()));
+    
+    // add each column in the group
+    int i=0; 
+    for (CoveredColumn column : columns) {
+      // hbase.index.covered.group.<i>.columns.<j>
+      String nextKey = columnPrefix + "." + Integer.toString(i);
+      String nextValue = column.serialize();
+      specs.put(nextKey, nextValue);
+      i++;
+    }
+  }
+
+  public void build(HTableDescriptor desc) throws IOException {
+    build(desc, CoveredColumnIndexCodec.class);
+  }
+
+  void build(HTableDescriptor desc, Class<? extends IndexCodec> clazz) throws IOException {
+    // add the codec for the index to the map of options
+    Map<String, String> opts = this.convertToMap();
+    opts.put(CoveredColumnsIndexBuilder.CODEC_CLASS_NAME_KEY, clazz.getName());
+    Indexer.enableIndexing(desc, CoveredColumnIndexer.class, opts);
+  }
+
+  static List<ColumnGroup> getColumns(Configuration conf) {
+    int count= conf.getInt(INDEX_GROUPS_COUNT_KEY, 0);
+    if (count ==0) {
+      return Collections.emptyList();
+    }
+
+    // parse out all the column groups we should index
+    List<ColumnGroup> columns = new ArrayList<ColumnGroup>(count);
+    for (int i = 0; i < count; i++) {
+      // parse out each group
+      String prefix = INDEX_GROUP_PREFIX + i;
+
+      // hbase.index.covered.group.<i>.table
+      String table = conf.get(prefix + TABLE_SUFFIX);
+      ColumnGroup group = new ColumnGroup(table);
+
+      // parse out each column in the group
+      // hbase.index.covered.group.<i>.columns
+      String columnPrefix = prefix + INDEX_GROUP_COVERAGE_SUFFIX;
+      // hbase.index.covered.group.<i>.columns.count = j
+      String columnsSizeKey = columnPrefix + COUNT;
+      int columnCount = conf.getInt(columnsSizeKey, 0);
+      for(int j=0; j< columnCount; j++){
+        String columnKey = columnPrefix + "." + j;
+        CoveredColumn column = CoveredColumn.parse(conf.get(columnKey));
+        group.add(column);
+      }
+
+      // add the group
+      columns.add(group);
+    }
+    return columns;
+  }
+
+  /**
+   * @param key
+   * @param value
+   */
+  public void addArbitraryConfigForTesting(String key, String value) {
+    this.specs.put(key, value);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexer.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexer.java
new file mode 100644
index 0000000..c5d7119
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexer.java
@@ -0,0 +1,164 @@
+/*
+ * 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.phoenix.hbase.index.covered.example;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.phoenix.hbase.index.covered.Batch;
+import org.apache.phoenix.hbase.index.covered.CoveredColumnsIndexBuilder;
+import org.apache.phoenix.hbase.index.covered.LocalTableState;
+import org.apache.phoenix.hbase.index.covered.update.IndexUpdateManager;
+
+/**
+ * Index maintainer that maintains multiple indexes based on '{@link ColumnGroup}s'. Each group is a
+ * fully covered within itself and stores the fully 'pre-joined' version of that values for that
+ * group of columns.
+ * <p>
+ * <h2>Index Layout</h2> The row key for a given index entry is the current state of the all the
+ * values of the columns in a column group, followed by the primary key (row key) of the original
+ * row, and then the length of each value and then finally the total number of values. This is then
+ * enough information to completely rebuild the latest value of row for each column in the group.
+ * <p>
+ * The family is always {@link CoveredColumnIndexCodec#INDEX_ROW_COLUMN_FAMILY}
+ * <p>
+ * The qualifier is prepended with the integer index (serialized with {@link Bytes#toBytes(int)}) of
+ * the column in the group. This index corresponds the index of the value for the group in the row
+ * key.
+ * 
+ * <pre>
+ *         ROW                            ||   FAMILY     ||    QUALIFIER     ||   VALUE
+ * (v1)(v2)...(vN)(pk)(L1)(L2)...(Ln)(#V) || INDEX_FAMILY ||     1Cf1:Cq1     ||  null
+ * (v1)(v2)...(vN)(pk)(L1)(L2)...(Ln)(#V) || INDEX_FAMILY ||     2Cf2:Cq2     ||  null
+ * ...
+ * (v1)(v2)...(vN)(pk)(L1)(L2)...(Ln)(#V) || INDEX_FAMILY ||     NCfN:CqN     ||  null
+ * </pre>
+ * 
+ * <h2>Index Maintenance</h2>
+ * <p>
+ * When making an insertion into the table, we also attempt to cleanup the index. This means that we
+ * need to remove the previous entry from the index. Generally, this is completed by inserting a
+ * delete at the previous value of the previous row.
+ * <p>
+ * The main caveat here is when dealing with custom timestamps. If there is no special timestamp
+ * specified, we can just insert the proper {@link Delete} at the current timestamp and move on.
+ * However, when the client specifies a timestamp, we could see updates out of order. In that case,
+ * we can do an insert using the specified timestamp, but a delete is different...
+ * <p>
+ * Taking the simple case, assume we do a single column in a group. Then if we get an out of order
+ * update, we need to check the current state of that column in the current row. If the current row
+ * is older, we can issue a delete as normal. If the current row is newer, however, we then have to
+ * issue a delete for the index update at the time of the current row. This ensures that the index
+ * update made for the 'future' time still covers the existing row.
+ * <p>
+ * <b>ASSUMPTION:</b> all key-values in a single {@link Delete}/{@link Put} have the same timestamp.
+ * This dramatically simplifies the logic needed to manage updating the index for out-of-order
+ * {@link Put}s as we don't need to manage multiple levels of timestamps across multiple columns.
+ * <p>
+ * We can extend this to multiple columns by picking the latest update of any column in group as the
+ * delete point.
+ * <p>
+ * <b>NOTE:</b> this means that we need to do a lookup (point {@link Get}) of the entire row
+ * <i>every time there is a write to the table</i>.
+ */
+public class CoveredColumnIndexer extends CoveredColumnsIndexBuilder {
+
+  /**
+   * Create the specified index table with the necessary columns
+   * @param admin {@link HBaseAdmin} to use when creating the table
+   * @param indexTable name of the index table.
+   * @throws IOException
+   */
+  public static void createIndexTable(HBaseAdmin admin, String indexTable) throws IOException {
+    createIndexTable(admin, new HTableDescriptor(indexTable));
+  }
+
+  /**
+   * @param admin to create the table
+   * @param index descriptor to update before creating table
+   */
+  public static void createIndexTable(HBaseAdmin admin, HTableDescriptor index) throws IOException {
+    HColumnDescriptor col =
+        new HColumnDescriptor(CoveredColumnIndexCodec.INDEX_ROW_COLUMN_FAMILY);
+    // ensure that we can 'see past' delete markers when doing scans
+    col.setKeepDeletedCells(true);
+    index.addFamily(col);
+    admin.createTable(index);
+  }
+
+  @Override
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdateForFilteredRows(
+      Collection<KeyValue> filtered) throws IOException {
+
+    // stores all the return values
+    IndexUpdateManager updateMap = new IndexUpdateManager();
+    // batch the updates by row to make life easier and ordered
+    Collection<Batch> batches = batchByRow(filtered);
+
+    for (Batch batch : batches) {
+      Put p = new Put(batch.getKvs().iterator().next().getRow());
+      for (KeyValue kv : batch.getKvs()) {
+        // we only need to cleanup Put entries
+        byte type = kv.getType();
+        Type t = KeyValue.Type.codeToType(type);
+        if (!t.equals(Type.Put)) {
+          continue;
+        }
+
+        // add the kv independently
+        p.add(kv);
+      }
+
+      // do the usual thing as for deletes
+      Collection<Batch> timeBatch = createTimestampBatchesFromMutation(p);
+      LocalTableState state = new LocalTableState(env, localTable, p);
+      for (Batch entry : timeBatch) {
+        //just set the timestamp on the table - it already has all the future state
+        state.setCurrentTimestamp(entry.getTimestamp());
+        this.addDeleteUpdatesToMap(updateMap, state, entry.getTimestamp());
+      }
+    }
+    return updateMap.toMap();
+  }
+
+
+  /**
+   * @param filtered
+   * @return
+   */
+  private Collection<Batch>  batchByRow(Collection<KeyValue> filtered) {
+    Map<Long, Batch> batches = new HashMap<Long, Batch>();
+    createTimestampBatchesFromKeyValues(filtered, batches);
+    return batches.values();
+  }
+}
\ No newline at end of file


[12/15] Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/BaseTaskRunner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/BaseTaskRunner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/BaseTaskRunner.java
deleted file mode 100644
index 1f8e45b..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/BaseTaskRunner.java
+++ /dev/null
@@ -1,129 +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.hadoop.hbase.index.parallel;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.Abortable;
-
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-
-/**
- * {@link TaskRunner} that just manages the underlying thread pool. On called to
- * {@link #stop(String)}, the thread pool is shutdown immediately - all pending tasks are cancelled
- * and running tasks receive and interrupt.
- * <p>
- * If we find a failure the failure is propagated to the {@link TaskBatch} so any {@link Task} that
- * is interested can kill itself as well.
- */
-public abstract class BaseTaskRunner implements TaskRunner {
-
-  private static final Log LOG = LogFactory.getLog(BaseTaskRunner.class);
-  protected ListeningExecutorService writerPool;
-  private boolean stopped;
-
-  public BaseTaskRunner(ExecutorService service) {
-    this.writerPool = MoreExecutors.listeningDecorator(service);
-  }
-
-  @Override
-  public <R> List<R> submit(TaskBatch<R> tasks) throws CancellationException, ExecutionException,
-      InterruptedException {
-    // submit each task to the pool and queue it up to be watched
-    List<ListenableFuture<R>> futures = new ArrayList<ListenableFuture<R>>(tasks.size());
-    for (Task<R> task : tasks.getTasks()) {
-      futures.add(this.writerPool.submit(task));
-    }
-    try {
-      // This logic is actually much more synchronized than the previous logic. Now we rely on a
-      // synchronization around the status to tell us when we are done. While this does have the
-      // advantage of being (1) less code, and (2) supported as part of a library, it is just that
-      // little bit slower. If push comes to shove, we can revert back to the previous
-      // implementation, but for right now, this works just fine.
-      return submitTasks(futures).get();
-    } catch (CancellationException e) {
-      // propagate the failure back out
-      logAndNotifyAbort(e, tasks);
-      throw e;
-    } catch (ExecutionException e) {
-      // propagate the failure back out
-      logAndNotifyAbort(e, tasks);
-      throw e;
-    }
-  }
-
-  private void logAndNotifyAbort(Exception e, Abortable abort) {
-    String msg = "Found a failed task because: " + e.getMessage();
-    LOG.error(msg, e);
-    abort.abort(msg, e.getCause());
-  }
-
-  /**
-   * Build a ListenableFuture for the tasks. Implementing classes can determine return behaviors on
-   * the given tasks
-   * @param futures to wait on
-   * @return a single {@link ListenableFuture} that completes based on the passes tasks.
-   */
-  protected abstract <R> ListenableFuture<List<R>> submitTasks(List<ListenableFuture<R>> futures);
-
-  @Override
-  public <R> List<R> submitUninterruptible(TaskBatch<R> tasks) throws EarlyExitFailure,
-      ExecutionException {
-    boolean interrupted = false;
-    try {
-      while (!this.isStopped()) {
-        try {
-          return this.submit(tasks);
-        } catch (InterruptedException e) {
-          interrupted = true;
-        }
-      }
-    } finally {
-      // restore the interrupted status
-      if (interrupted) {
-        Thread.currentThread().interrupt();
-      }
-    }
-
-    // should only get here if we are interrupted while waiting for a result and have been told to
-    // shutdown by an external source
-    throw new EarlyExitFailure("Interrupted and stopped before computation was complete!");
-  }
-
-  @Override
-  public void stop(String why) {
-    if (this.stopped) {
-      return;
-    }
-    LOG.info("Shutting down task runner because " + why);
-    this.writerPool.shutdownNow();
-  }
-
-  @Override
-  public boolean isStopped() {
-    return this.stopped;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/EarlyExitFailure.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/EarlyExitFailure.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/EarlyExitFailure.java
deleted file mode 100644
index 4e89034..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/EarlyExitFailure.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.hadoop.hbase.index.parallel;
-
-import java.io.IOException;
-
-/**
- * Exception denoting a need to early-exit a task (or group of tasks) due to external notification
- */
-@SuppressWarnings("serial")
-public class EarlyExitFailure extends IOException {
-
-  /**
-   * @param msg reason for the early exit
-   */
-  public EarlyExitFailure(String msg) {
-    super(msg);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/QuickFailingTaskRunner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/QuickFailingTaskRunner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/QuickFailingTaskRunner.java
deleted file mode 100644
index 732750f..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/QuickFailingTaskRunner.java
+++ /dev/null
@@ -1,49 +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.hadoop.hbase.index.parallel;
-
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-
-/**
- * {@link TaskRunner} that attempts to run all tasks passed, but quits early if any {@link Task}
- * fails, not waiting for the remaining {@link Task}s to complete.
- */
-public class QuickFailingTaskRunner extends BaseTaskRunner {
-
-  static final Log LOG = LogFactory.getLog(QuickFailingTaskRunner.class);
-
-  /**
-   * @param service thread pool to which {@link Task}s are submitted. This service is then 'owned'
-   *          by <tt>this</tt> and will be shutdown on calls to {@link #stop(String)}.
-   */
-  public QuickFailingTaskRunner(ExecutorService service) {
-    super(service);
-  }
-
-  @Override
-  protected <R> ListenableFuture<List<R>> submitTasks(List<ListenableFuture<R>> futures) {
-    return Futures.allAsList(futures);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/Task.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/Task.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/Task.java
deleted file mode 100644
index 3496c6b..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/Task.java
+++ /dev/null
@@ -1,40 +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.hadoop.hbase.index.parallel;
-
-import java.util.concurrent.Callable;
-
-import org.apache.hadoop.hbase.Abortable;
-
-/**
- * Like a {@link Callable}, but supports an internal {@link Abortable} that can be checked
- * periodically to determine if the batch should abort
- * @param <V> expected result of the task
- */
-public abstract class Task<V> implements Callable<V> {
-
-  private Abortable batch;
-
-  void setBatchMonitor(Abortable abort) {
-    this.batch = abort;
-  }
-
-  protected boolean isBatchFailed() {
-    return this.batch.isAborted();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskBatch.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskBatch.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskBatch.java
deleted file mode 100644
index 68e359c..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskBatch.java
+++ /dev/null
@@ -1,74 +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.hadoop.hbase.index.parallel;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.Abortable;
-
-/**
- * A group of {@link Task}s. The tasks are all bound together using the same {@link Abortable} (
- * <tt>this</tt>) to ensure that all tasks are aware when any of the other tasks fails.
- * @param <V> expected result type from all the tasks
- */
-public class TaskBatch<V> implements Abortable {
-  private static final Log LOG = LogFactory.getLog(TaskBatch.class);
-  private AtomicBoolean aborted = new AtomicBoolean();
-  private List<Task<V>> tasks;
-
-  /**
-   * @param size expected number of tasks
-   */
-  public TaskBatch(int size) {
-    this.tasks = new ArrayList<Task<V>>(size);
-  }
-
-  public void add(Task<V> task) {
-    this.tasks.add(task);
-    task.setBatchMonitor(this);
-  }
-
-  public Collection<Task<V>> getTasks() {
-    return this.tasks;
-  }
-
-  @Override
-  public void abort(String why, Throwable e) {
-    if (this.aborted.getAndSet(true)) {
-      return;
-    }
-    LOG.info("Aborting batch of tasks because " + why);
-  }
-
-  @Override
-  public boolean isAborted() {
-    return this.aborted.get();
-  }
-
-  /**
-   * @return the number of tasks assigned to this batch
-   */
-  public int size() {
-    return this.tasks.size();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskRunner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskRunner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskRunner.java
deleted file mode 100644
index dd63753..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/TaskRunner.java
+++ /dev/null
@@ -1,60 +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.hadoop.hbase.index.parallel;
-
-import java.util.List;
-import java.util.concurrent.ExecutionException;
-
-import org.apache.hadoop.hbase.Stoppable;
-
-/**
- *
- */
-public interface TaskRunner extends Stoppable {
-
-  /**
-   * Submit the given tasks to the pool and wait for them to complete. fail.
-   * <p>
-   * Non-interruptible method. To stop any running tasks call {@link #stop(String)} - this will
-   * shutdown the thread pool, causing any pending tasks to be failed early (whose failure will be
-   * ignored) and interrupt any running tasks. It is up to the passed tasks to respect the interrupt
-   * notification
-   * @param tasks to run
-   * @return the result from each task
-   * @throws ExecutionException if any of the tasks fails. Wraps the underyling failure, which can
-   *           be retrieved via {@link ExecutionException#getCause()}.
-   * @throws InterruptedException if the current thread is interrupted while waiting for the batch
-   *           to complete
-   */
-  public <R> List<R> submit(TaskBatch<R> tasks) throws
-      ExecutionException, InterruptedException;
-
-  /**
-   * Similar to {@link #submit(TaskBatch)}, but is not interruptible. If an interrupt is found while
-   * waiting for results, we ignore it and only stop is {@link #stop(String)} has been called. On
-   * return from the method, the interrupt status of the thread is restored.
-   * @param tasks to run
-   * @return the result from each task
-   * @throws EarlyExitFailure if there are still tasks to submit to the pool, but there is a stop
-   *           notification
-   * @throws ExecutionException if any of the tasks fails. Wraps the underyling failure, which can
-   *           be retrieved via {@link ExecutionException#getCause()}.
-   */
-  public <R> List<R> submitUninterruptible(TaskBatch<R> tasks) throws EarlyExitFailure,
-      ExecutionException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolBuilder.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolBuilder.java
deleted file mode 100644
index 10783f2..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolBuilder.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.index.parallel;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.util.Pair;
-
-/**
- * Helper utility to make a thread pool from a configuration based on reasonable defaults and passed
- * configuration keys.
- */
-public class ThreadPoolBuilder {
-
-  private static final Log LOG = LogFactory.getLog(ThreadPoolBuilder.class);
-  private static final long DEFAULT_TIMEOUT = 60;
-  private static final int DEFAULT_MAX_THREADS = 1;// is there a better default?
-  private Pair<String, Long> timeout;
-  private Pair<String, Integer> maxThreads;
-  private String name;
-  private Configuration conf;
-
-  public ThreadPoolBuilder(String poolName, Configuration conf) {
-    this.name = poolName;
-    this.conf = conf;
-  }
-
-  public ThreadPoolBuilder setCoreTimeout(String confkey, long defaultTime) {
-    if (defaultTime <= 0) {
-      defaultTime = DEFAULT_TIMEOUT;
-    }
-    this.timeout = new Pair<String, Long>(confkey, defaultTime);
-    return this;
-  }
-
-  public ThreadPoolBuilder setCoreTimeout(String confKey) {
-    return this.setCoreTimeout(confKey, DEFAULT_TIMEOUT);
-  }
-
-  public ThreadPoolBuilder setMaxThread(String confkey, int defaultThreads) {
-    if (defaultThreads <= 0) {
-      defaultThreads = DEFAULT_MAX_THREADS;
-    }
-    this.maxThreads = new Pair<String, Integer>(confkey, defaultThreads);
-    return this;
-  }
-
-  String getName() {
-   return this.name;
-  }
-
-  int getMaxThreads() {
-    int maxThreads = DEFAULT_MAX_THREADS;
-    if (this.maxThreads != null) {
-      String key = this.maxThreads.getFirst();
-      maxThreads =
-          key == null ? this.maxThreads.getSecond() : conf.getInt(key, this.maxThreads.getSecond());
-    }
-    LOG.trace("Creating pool builder with max " + maxThreads + " threads ");
-    return maxThreads;
-  }
-
-  long getKeepAliveTime() {
-    long timeout =DEFAULT_TIMEOUT;
-    if (this.timeout != null) {
-      String key = this.timeout.getFirst();
-      timeout =
-          key == null ? this.timeout.getSecond() : conf.getLong(key, this.timeout.getSecond());
-    }
-
-    LOG.trace("Creating pool builder with core thread timeout of " + timeout + " seconds ");
-    return timeout;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolManager.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolManager.java
deleted file mode 100644
index dba4789..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/ThreadPoolManager.java
+++ /dev/null
@@ -1,146 +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.hadoop.hbase.index.parallel;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.util.Threads;
-
-/**
- * Manage access to thread pools
- */
-public class ThreadPoolManager {
-
-  private static final Log LOG = LogFactory.getLog(ThreadPoolManager.class);
-
-  /**
-   * Get an executor for the given name, based on the passed {@link Configuration}. If a thread pool
-   * already exists with that name, it will be returned.
-   * @param builder
-   * @param env
-   * @return a {@link ThreadPoolExecutor} for the given name. Thread pool that only shuts down when
-   *         there are no more explicit references to it. You do not need to shutdown the threadpool
-   *         on your own - it is managed for you. When you are done, you merely need to release your
-   *         reference. If you do attempt to shutdown the pool, you should be careful to call
-   *         {@link ThreadPoolExecutor#shutdown()} XOR {@link ThreadPoolExecutor#shutdownNow()} - extra calls to either can lead to
-   *         early shutdown of the pool.
-   */
-  public static synchronized ThreadPoolExecutor getExecutor(ThreadPoolBuilder builder,
-      RegionCoprocessorEnvironment env) {
-    return getExecutor(builder, env.getSharedData());
-  }
-
-  static synchronized ThreadPoolExecutor getExecutor(ThreadPoolBuilder builder,
-      Map<String, Object> poolCache) {
-    ThreadPoolExecutor pool = (ThreadPoolExecutor) poolCache.get(builder.getName());
-    if (pool == null || pool.isTerminating() || pool.isShutdown()) {
-      pool = getDefaultExecutor(builder);
-      LOG.info("Creating new pool for " + builder.getName());
-      poolCache.put(builder.getName(), pool);
-    }
-    ((ShutdownOnUnusedThreadPoolExecutor) pool).addReference();
-
-    return pool;
-  }
-
-  /**
-   * @param conf
-   * @return
-   */
-  private static ShutdownOnUnusedThreadPoolExecutor getDefaultExecutor(ThreadPoolBuilder builder) {
-    int maxThreads = builder.getMaxThreads();
-    long keepAliveTime = builder.getKeepAliveTime();
-
-    // we prefer starting a new thread to queuing (the opposite of the usual ThreadPoolExecutor)
-    // since we are probably writing to a bunch of index tables in this case. Any pending requests
-    // are then queued up in an infinite (Integer.MAX_VALUE) queue. However, we allow core threads
-    // to timeout, to we tune up/down for bursty situations. We could be a bit smarter and more
-    // closely manage the core-thread pool size to handle the bursty traffic (so we can always keep
-    // some core threads on hand, rather than starting from scratch each time), but that would take
-    // even more time. If we shutdown the pool, but are still putting new tasks, we can just do the
-    // usual policy and throw a RejectedExecutionException because we are shutting down anyways and
-    // the worst thing is that this gets unloaded.
-    ShutdownOnUnusedThreadPoolExecutor pool =
-        new ShutdownOnUnusedThreadPoolExecutor(maxThreads, maxThreads, keepAliveTime,
-            TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(),
-            Threads.newDaemonThreadFactory(builder.getName() + "-"), builder.getName());
-    pool.allowCoreThreadTimeOut(true);
-    return pool;
-  }
-
-  /**
-   * Thread pool that only shuts down when there are no more explicit references to it. A reference
-   * is when obtained and released on calls to {@link #shutdown()} or {@link #shutdownNow()}.
-   * Therefore, users should be careful to call {@link #shutdown()} XOR {@link #shutdownNow()} -
-   * extra calls to either can lead to early shutdown of the pool.
-   */
-  private static class ShutdownOnUnusedThreadPoolExecutor extends ThreadPoolExecutor {
-
-    private AtomicInteger references;
-    private String poolName;
-
-    public ShutdownOnUnusedThreadPoolExecutor(int coreThreads, int maxThreads, long keepAliveTime,
-        TimeUnit timeUnit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory,
-        String poolName) {
-      super(coreThreads, maxThreads, keepAliveTime, timeUnit, workQueue, threadFactory);
-      this.references = new AtomicInteger();
-      this.poolName = poolName;
-    }
-
-    public void addReference() {
-      this.references.incrementAndGet();
-    }
-
-    @Override
-    protected void finalize() {
-      // override references counter if we go out of scope - ensures the pool gets cleaned up
-      LOG.info("Shutting down pool '" + poolName + "' because no more references");
-      super.finalize();
-    }
-
-    @Override
-    public void shutdown() {
-      if (references.decrementAndGet() <= 0) {
-        LOG.debug("Shutting down pool " + this.poolName);
-        super.shutdown();
-      }
-    }
-
-    @Override
-    public List<Runnable> shutdownNow() {
-      if (references.decrementAndGet() <= 0) {
-        LOG.debug("Shutting down pool " + this.poolName + " NOW!");
-        return super.shutdownNow();
-      }
-      return Collections.emptyList();
-    }
-
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/WaitForCompletionTaskRunner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/WaitForCompletionTaskRunner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/WaitForCompletionTaskRunner.java
deleted file mode 100644
index 7f94157..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/parallel/WaitForCompletionTaskRunner.java
+++ /dev/null
@@ -1,51 +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.hadoop.hbase.index.parallel;
-
-import java.util.List;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-
-/**
- * A {@link TaskRunner} that ensures that all the tasks have been attempted before we return, even
- * if some of the tasks cause failures.
- * <p>
- * Because we wait until the entire batch is complete to see the failure, checking for failure of
- * the {@link TaskBatch} on the submitted tasks will not help - they will never see the failure of
- * the other tasks. You will need to provide an external mechanism to propagate the error.
- * <p>
- * Does not throw an {@link ExecutionException} if any of the tasks fail.
- */
-public class WaitForCompletionTaskRunner extends BaseTaskRunner {
-  
-  /**
-   * @param service thread pool to which {@link Task}s are submitted. This service is then 'owned'
-   *          by <tt>this</tt> and will be shutdown on calls to {@link #stop(String)}.
-   */
-  public WaitForCompletionTaskRunner(ExecutorService service) {
-    super(service);
-  }
-
-  @Override
-  public <R> ListenableFuture<List<R>> submitTasks(List<ListenableFuture<R>> futures) {
-    return Futures.successfulAsList(futures);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/EmptyScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/EmptyScanner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/EmptyScanner.java
deleted file mode 100644
index 9b57429..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/EmptyScanner.java
+++ /dev/null
@@ -1,50 +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.hadoop.hbase.index.scanner;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.KeyValue;
-
-
-/**
- * {@link Scanner} that has no underlying data
- */
-public class EmptyScanner implements Scanner {
-
-  @Override
-  public KeyValue next() throws IOException {
-    return null;
-  }
-
-  @Override
-  public boolean seek(KeyValue next) throws IOException {
-    return false;
-  }
-
-  @Override
-  public KeyValue peek() throws IOException {
-    return null;
-  }
-
-  @Override
-  public void close() throws IOException {
-    // noop
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/FilteredKeyValueScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/FilteredKeyValueScanner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/FilteredKeyValueScanner.java
deleted file mode 100644
index 7b54790..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/FilteredKeyValueScanner.java
+++ /dev/null
@@ -1,162 +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.hadoop.hbase.index.scanner;
-
-import java.io.IOException;
-import java.util.SortedSet;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
-import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
-
-import org.apache.hadoop.hbase.index.covered.KeyValueStore;
-
-/**
- * Combine a simplified version of the logic in the ScanQueryMatcher and the KeyValueScanner. We can get away with this
- * here because we are only concerned with a single MemStore for the index; we don't need to worry about multiple column
- * families or minimizing seeking through file - we just want to iterate the kvs quickly, in-memory.
- */
-public class FilteredKeyValueScanner implements KeyValueScanner {
-
-    private KeyValueScanner delegate;
-    private Filter filter;
-
-    public FilteredKeyValueScanner(Filter filter, KeyValueStore store) {
-        this(filter, store.getScanner());
-    }
-
-    private FilteredKeyValueScanner(Filter filter, KeyValueScanner delegate) {
-        this.delegate = delegate;
-        this.filter = filter;
-    }
-
-    @Override
-    public KeyValue peek() {
-        return delegate.peek();
-    }
-
-    /**
-     * Same a {@link KeyValueScanner#next()} except that we filter out the next {@link KeyValue} until we find one that
-     * passes the filter.
-     * 
-     * @return the next {@link KeyValue} or <tt>null</tt> if no next {@link KeyValue} is present and passes all the
-     *         filters.
-     */
-    @Override
-    public KeyValue next() throws IOException {
-        seekToNextUnfilteredKeyValue();
-        return delegate.next();
-    }
-
-    @Override
-    public boolean seek(KeyValue key) throws IOException {
-        if (filter.filterAllRemaining()) { return false; }
-        // see if we can seek to the next key
-        if (!delegate.seek(key)) { return false; }
-
-        return seekToNextUnfilteredKeyValue();
-    }
-
-    private boolean seekToNextUnfilteredKeyValue() throws IOException {
-        while (true) {
-            KeyValue peeked = delegate.peek();
-            // no more key values, so we are done
-            if (peeked == null) { return false; }
-
-            // filter the peeked value to see if it should be served
-            ReturnCode code = filter.filterKeyValue(peeked);
-            switch (code) {
-            // included, so we are done
-            case INCLUDE:
-            case INCLUDE_AND_NEXT_COL:
-                return true;
-                // not included, so we need to go to the next row
-            case SKIP:
-            case NEXT_COL:
-            case NEXT_ROW:
-                delegate.next();
-                break;
-            // use a seek hint to find out where we should go
-            case SEEK_NEXT_USING_HINT:
-                delegate.seek(filter.getNextKeyHint(peeked));
-            }
-        }
-    }
-
-    @Override
-    public boolean reseek(KeyValue key) throws IOException {
-        this.delegate.reseek(key);
-        return this.seekToNextUnfilteredKeyValue();
-    }
-
-    @Override
-    public boolean requestSeek(KeyValue kv, boolean forward, boolean useBloom) throws IOException {
-        return this.reseek(kv);
-    }
-
-    @Override
-    public boolean isFileScanner() {
-        return false;
-    }
-
-    @Override
-    public long getSequenceID() {
-        return this.delegate.getSequenceID();
-    }
-
-    @Override
-    public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns, long oldestUnexpiredTS) {
-        throw new UnsupportedOperationException(this.getClass().getName()
-                + " doesn't support checking to see if it should use a scanner!");
-    }
-
-    @Override
-    public boolean realSeekDone() {
-        return this.delegate.realSeekDone();
-    }
-
-    @Override
-    public void enforceSeek() throws IOException {
-        this.delegate.enforceSeek();
-    }
-
-    @Override
-    public void close() {
-        this.delegate.close();
-    }
-
-    /*
-    @Override
-    public boolean backwardSeek(KeyValue arg0) throws IOException {
-        return this.delegate.backwardSeek(arg0);
-    }
-
-    @Override
-    public boolean seekToLastRow() throws IOException {
-        return this.delegate.seekToLastRow();
-    }
-
-    @Override
-    public boolean seekToPreviousRow(KeyValue arg0) throws IOException {
-        return this.delegate.seekToPreviousRow(arg0);
-    }
-    */
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/Scanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/Scanner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/Scanner.java
deleted file mode 100644
index b7b1db1..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/Scanner.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.index.scanner;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.KeyValue;
-
-/**
- * Scan the primary table. This is similar to HBase's scanner, but ensures that you will never see
- * deleted columns/rows
- */
-public interface Scanner extends Closeable {
-
-  /**
-   * @return the next keyvalue in the scanner or <tt>null</tt> if there is no next {@link KeyValue}
-   * @throws IOException if there is an underlying error reading the data
-   */
-  public KeyValue next() throws IOException;
-
-  /**
-   * Seek to immediately before the given {@link KeyValue}. If that exact {@link KeyValue} is
-   * present in <tt>this</tt>, it will be returned by the next call to {@link #next()}. Otherwise,
-   * returns the next {@link KeyValue} after the seeked {@link KeyValue}.
-   * @param next {@link KeyValue} to seek to. Doesn't need to already be present in <tt>this</tt>
-   * @return <tt>true</tt> if there are values left in <tt>this</tt>, <tt>false</tt> otherwise
-   * @throws IOException if there is an error reading the underlying data.
-   */
-  public boolean seek(KeyValue next) throws IOException;
-
-  /**
-   * Read the {@link KeyValue} at the top of <tt>this</tt> without 'popping' it off the top of the
-   * scanner.
-   * @return the next {@link KeyValue} or <tt>null</tt> if there are no more values in <tt>this</tt>
-   * @throws IOException if there is an error reading the underlying data.
-   */
-  public KeyValue peek() throws IOException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/ScannerBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/ScannerBuilder.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/ScannerBuilder.java
deleted file mode 100644
index 2bdc414..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/scanner/ScannerBuilder.java
+++ /dev/null
@@ -1,165 +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.hadoop.hbase.index.scanner;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.filter.BinaryComparator;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.filter.FamilyFilter;
-import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.FilterList;
-import org.apache.hadoop.hbase.filter.QualifierFilter;
-import org.apache.hadoop.hbase.util.Bytes;
-
-import com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.index.covered.KeyValueStore;
-import org.apache.hadoop.hbase.index.covered.filter.ApplyAndFilterDeletesFilter;
-import org.apache.hadoop.hbase.index.covered.filter.ColumnTrackingNextLargestTimestampFilter;
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-import org.apache.hadoop.hbase.index.covered.update.ColumnTracker;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-/**
- *
- */
-public class ScannerBuilder {
-
-  private KeyValueStore memstore;
-  private Mutation update;
-
-
-  public ScannerBuilder(KeyValueStore memstore, Mutation update) {
-    this.memstore = memstore;
-    this.update = update;
-  }
-
-  public Scanner buildIndexedColumnScanner(Collection<? extends ColumnReference> indexedColumns, ColumnTracker tracker, long ts) {
-
-    Filter columnFilters = getColumnFilters(indexedColumns);
-    FilterList filters = new FilterList(Lists.newArrayList(columnFilters));
-
-    // skip to the right TS. This needs to come before the deletes since the deletes will hide any
-    // state that comes before the actual kvs, so we need to capture those TS as they change the row
-    // state.
-    filters.addFilter(new ColumnTrackingNextLargestTimestampFilter(ts, tracker));
-
-    // filter out kvs based on deletes
-    filters.addFilter(new ApplyAndFilterDeletesFilter(getAllFamilies(indexedColumns)));
-
-    // combine the family filters and the rest of the filters as a
-    return getFilteredScanner(filters);
-  }
-
-  /**
-   * @param columns columns to filter
-   * @return filter that will skip any {@link KeyValue} that doesn't match one of the passed columns
-   *         and the
-   */
-  private Filter
-      getColumnFilters(Collection<? extends ColumnReference> columns) {
-    // each column needs to be added as an OR, so we need to separate them out
-    FilterList columnFilters = new FilterList(FilterList.Operator.MUST_PASS_ONE);
-
-    // create a filter that matches each column reference
-    for (ColumnReference ref : columns) {
-      Filter columnFilter =
-          new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(ref.getFamily()));
-      // combine with a match for the qualifier, if the qualifier is a specific qualifier
-      if (!Bytes.equals(ColumnReference.ALL_QUALIFIERS, ref.getQualifier())) {
-        columnFilter =
-            new FilterList(columnFilter, new QualifierFilter(CompareOp.EQUAL, new BinaryComparator(
-                ref.getQualifier())));
-      }
-      columnFilters.addFilter(columnFilter);
-    }
-    return columnFilters;
-  }
-
-  private Set<ImmutableBytesPtr>
-      getAllFamilies(Collection<? extends ColumnReference> columns) {
-    Set<ImmutableBytesPtr> families = new HashSet<ImmutableBytesPtr>();
-    for (ColumnReference ref : columns) {
-      families.add(new ImmutableBytesPtr(ref.getFamily()));
-    }
-    return families;
-  }
-
-  private Scanner getFilteredScanner(Filter filters) {
-    // create a scanner and wrap it as an iterator, meaning you can only go forward
-    final FilteredKeyValueScanner kvScanner = new FilteredKeyValueScanner(filters, memstore);
-    // seek the scanner to initialize it
-    KeyValue start = KeyValue.createFirstOnRow(update.getRow());
-    try {
-      if (!kvScanner.seek(start)) {
-        return new EmptyScanner();
-      }
-    } catch (IOException e) {
-      // This should never happen - everything should explode if so.
-      throw new RuntimeException(
-          "Failed to seek to first key from update on the memstore scanner!", e);
-    }
-
-    // we have some info in the scanner, so wrap it in an iterator and return.
-    return new Scanner() {
-
-      @Override
-      public KeyValue next() {
-        try {
-          return kvScanner.next();
-        } catch (IOException e) {
-          throw new RuntimeException("Error reading kvs from local memstore!");
-        }
-      }
-
-      @Override
-      public boolean seek(KeyValue next) throws IOException {
-        // check to see if the next kv is after the current key, in which case we can use reseek,
-        // which will be more efficient
-        KeyValue peek = kvScanner.peek();
-        // there is another value and its before the requested one - we can do a reseek!
-        if (peek != null) {
-          int compare = KeyValue.COMPARATOR.compare(peek, next);
-          if (compare < 0) {
-            return kvScanner.reseek(next);
-          } else if (compare == 0) {
-            // we are already at the given key!
-            return true;
-          }
-        }
-        return kvScanner.seek(next);
-      }
-
-      @Override
-      public KeyValue peek() throws IOException {
-        return kvScanner.peek();
-      }
-
-      @Override
-      public void close() {
-        kvScanner.close();
-      }
-    };
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CachingHTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CachingHTableFactory.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CachingHTableFactory.java
deleted file mode 100644
index cb22b8a..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CachingHTableFactory.java
+++ /dev/null
@@ -1,110 +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.hadoop.hbase.index.table;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.commons.collections.map.LRUMap;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.util.Bytes;
-
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-/**
- * A simple cache that just uses usual GC mechanisms to cleanup unused {@link HTableInterface}s.
- * When requesting an {@link HTableInterface} via {@link #getTable}, you may get the same table as
- * last time, or it may be a new table.
- * <p>
- * You <b>should not call {@link HTableInterface#close()} </b> that is handled when the table goes
- * out of scope. Along the same lines, you must ensure to not keep a reference to the table for
- * longer than necessary - this leak will ensure that the table never gets closed.
- */
-public class CachingHTableFactory implements HTableFactory {
-
-  /**
-   * LRUMap that closes the {@link HTableInterface} when the table is evicted
-   */
-  @SuppressWarnings("serial")
-  public class HTableInterfaceLRUMap extends LRUMap {
-
-    public HTableInterfaceLRUMap(int cacheSize) {
-      super(cacheSize);
-    }
-
-    @Override
-    protected boolean removeLRU(LinkEntry entry) {
-      HTableInterface table = (HTableInterface) entry.getValue();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Closing connection to table: " + Bytes.toString(table.getTableName())
-            + " because it was evicted from the cache.");
-      }
-      try {
-        table.close();
-      } catch (IOException e) {
-        LOG.info("Failed to correctly close HTable: " + Bytes.toString(table.getTableName())
-            + " ignoring since being removed from queue.");
-      }
-      return true;
-    }
-  }
-
-  public static int getCacheSize(Configuration conf) {
-    return conf.getInt(CACHE_SIZE_KEY, DEFAULT_CACHE_SIZE);
-  }
-
-  private static final Log LOG = LogFactory.getLog(CachingHTableFactory.class);
-  private static final String CACHE_SIZE_KEY = "index.tablefactory.cache.size";
-  private static final int DEFAULT_CACHE_SIZE = 10;
-
-  private HTableFactory delegate;
-
-  @SuppressWarnings("rawtypes")
-  Map openTables;
-
-  public CachingHTableFactory(HTableFactory tableFactory, Configuration conf) {
-    this(tableFactory, getCacheSize(conf));
-  }
-
-  public CachingHTableFactory(HTableFactory factory, int cacheSize) {
-    this.delegate = factory;
-    openTables = new HTableInterfaceLRUMap(cacheSize);
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public HTableInterface getTable(ImmutableBytesPtr tablename) throws IOException {
-    ImmutableBytesPtr tableBytes = new ImmutableBytesPtr(tablename);
-    synchronized (openTables) {
-      HTableInterface table = (HTableInterface) openTables.get(tableBytes);
-      if (table == null) {
-        table = delegate.getTable(tablename);
-        openTables.put(tableBytes, table);
-      }
-      return table;
-    }
-  }
-
-  @Override
-  public void shutdown() {
-    this.delegate.shutdown();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CoprocessorHTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CoprocessorHTableFactory.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CoprocessorHTableFactory.java
deleted file mode 100644
index dd39b0c..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/CoprocessorHTableFactory.java
+++ /dev/null
@@ -1,68 +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.hadoop.hbase.index.table;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.util.Bytes;
-
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
-
-public class CoprocessorHTableFactory implements HTableFactory {
-
-  /** Number of milliseconds per-interval to retry zookeeper */
-  private static final String ZOOKEEPER_RECOVERY_RETRY_INTERVALMILL = "zookeeper.recovery.retry.intervalmill";
-  /** Number of retries for zookeeper */
-  private static final String ZOOKEEPER_RECOVERY_RETRY_KEY = "zookeeper.recovery.retry";
-  private static final Log LOG = LogFactory.getLog(CoprocessorHTableFactory.class);
-  private CoprocessorEnvironment e;
-
-  public CoprocessorHTableFactory(CoprocessorEnvironment e) {
-    this.e = e;
-  }
-
-  @Override
-  public HTableInterface getTable(ImmutableBytesPtr tablename) throws IOException {
-    Configuration conf = e.getConfiguration();
-    // make sure writers fail fast
-    IndexManagementUtil.setIfNotSet(conf, HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
-    IndexManagementUtil.setIfNotSet(conf, HConstants.HBASE_CLIENT_PAUSE, 1000);
-    IndexManagementUtil.setIfNotSet(conf, ZOOKEEPER_RECOVERY_RETRY_KEY, 3);
-    IndexManagementUtil.setIfNotSet(conf, ZOOKEEPER_RECOVERY_RETRY_INTERVALMILL, 100);
-    IndexManagementUtil.setIfNotSet(conf, HConstants.ZK_SESSION_TIMEOUT, 30000);
-    IndexManagementUtil.setIfNotSet(conf, HConstants.HBASE_RPC_TIMEOUT_KEY, 5000);
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Creating new HTable: " + Bytes.toString(tablename.copyBytesIfNecessary()));
-    }
-    return this.e.getTable(tablename.copyBytesIfNecessary());
-  }
-
-  @Override
-  public void shutdown() {
-    // noop
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableFactory.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableFactory.java
deleted file mode 100644
index cf07a74..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableFactory.java
+++ /dev/null
@@ -1,32 +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.hadoop.hbase.index.table;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.client.HTableInterface;
-
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-public interface HTableFactory {
-
-  public HTableInterface getTable(ImmutableBytesPtr tablename) throws IOException;
-
-  public void shutdown();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableInterfaceReference.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableInterfaceReference.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableInterfaceReference.java
deleted file mode 100644
index c4d6304..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/table/HTableInterfaceReference.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.index.table;
-
-import org.apache.hadoop.hbase.util.Bytes;
-
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-/**
- * Reference to an HTableInterface. Currently, its pretty simple in that it is just a wrapper around
- * the table name.
- */
-public class HTableInterfaceReference {
-
-  private ImmutableBytesPtr tableName;
-
-
-  public HTableInterfaceReference(ImmutableBytesPtr tableName) {
-    this.tableName = tableName;
-  }
-
-  public ImmutableBytesPtr get() {
-    return this.tableName;
-  }
-
-  public String getTableName() {
-    return Bytes.toString(this.tableName.get(),this.tableName.getOffset(), this.tableName.getLength());
-  }
-
-  @Override
-  public int hashCode() {
-      return tableName.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-      if (this == obj) return true;
-      if (obj == null) return false;
-      if (getClass() != obj.getClass()) return false;
-      HTableInterfaceReference other = (HTableInterfaceReference)obj;
-      return tableName.equals(other.tableName);
-  }
-
-  @Override
-  public String toString() {
-    return Bytes.toString(this.tableName.get());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/ImmutableBytesPtr.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/ImmutableBytesPtr.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/ImmutableBytesPtr.java
deleted file mode 100644
index 50a4861..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/ImmutableBytesPtr.java
+++ /dev/null
@@ -1,111 +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.hadoop.hbase.index.util;
-
-import java.io.DataInput;
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Bytes;
-
-public class ImmutableBytesPtr extends ImmutableBytesWritable {
-    private int hashCode;
-    
-    public ImmutableBytesPtr() {
-    }
-
-    public ImmutableBytesPtr(byte[] bytes) {
-        super(bytes);
-        hashCode = super.hashCode();
-    }
-
-    public ImmutableBytesPtr(ImmutableBytesWritable ibw) {
-        super(ibw.get(), ibw.getOffset(), ibw.getLength());
-        hashCode = super.hashCode();
-    }
-
-    public ImmutableBytesPtr(ImmutableBytesPtr ibp) {
-        super(ibp.get(), ibp.getOffset(), ibp.getLength());
-        hashCode = ibp.hashCode;
-    }
-
-    public ImmutableBytesPtr(byte[] bytes, int offset, int length) {
-        super(bytes, offset, length);
-        hashCode = super.hashCode();
-    }
-
-    @Override
-    public int hashCode() {
-        return hashCode;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj) return true;
-        if (obj == null) return false;
-        if (getClass() != obj.getClass()) return false;
-        ImmutableBytesPtr that = (ImmutableBytesPtr)obj;
-        if (this.hashCode != that.hashCode) return false;
-        if (Bytes.compareTo(this.get(), this.getOffset(), this.getLength(), that.get(), that.getOffset(), that.getLength()) != 0) return false;
-        return true;
-    }
-
-    public void set(ImmutableBytesWritable ptr) {
-        set(ptr.get(),ptr.getOffset(),ptr.getLength());
-      }
-
-    /**
-     * @param b Use passed bytes as backing array for this instance.
-     */
-    @Override
-    public void set(final byte [] b) {
-      super.set(b);
-      hashCode = super.hashCode();
-    }
-
-    /**
-     * @param b Use passed bytes as backing array for this instance.
-     * @param offset
-     * @param length
-     */
-    @Override
-    public void set(final byte [] b, final int offset, final int length) {
-        super.set(b,offset,length);
-        hashCode = super.hashCode();
-    }
-
-    @Override
-    public void readFields(final DataInput in) throws IOException {
-        super.readFields(in);
-        hashCode = super.hashCode();
-    }
-    
-    /**
-     * @return the backing byte array, copying only if necessary
-     */
-    public byte[] copyBytesIfNecessary() {
-    return copyBytesIfNecessary(this);
-    }
-
-  public static byte[] copyBytesIfNecessary(ImmutableBytesWritable ptr) {
-    if (ptr.getOffset() == 0 && ptr.getLength() == ptr.get().length) {
-      return ptr.get();
-    }
-    return ptr.copyBytes();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/IndexManagementUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/IndexManagementUtil.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/IndexManagementUtil.java
deleted file mode 100644
index 0a4bd2c..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/util/IndexManagementUtil.java
+++ /dev/null
@@ -1,244 +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.hadoop.hbase.index.util;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Scan;
-
-import com.google.common.collect.Maps;
-import org.apache.hadoop.hbase.index.ValueGetter;
-import org.apache.hadoop.hbase.index.builder.IndexBuildingFailureException;
-import org.apache.hadoop.hbase.index.covered.data.LazyValueGetter;
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-import org.apache.hadoop.hbase.index.scanner.Scanner;
-
-/**
- * Utility class to help manage indexes
- */
-public class IndexManagementUtil {
-
-    private IndexManagementUtil() {
-        // private ctor for util classes
-    }
-
-    // Don't rely on statically defined classes constants from classes that may not exist
-    // in earlier HBase versions
-    public static final String INDEX_WAL_EDIT_CODEC_CLASS_NAME = "org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec";
-    public static final String HLOG_READER_IMPL_KEY = "hbase.regionserver.hlog.reader.impl";
-    public static final String WAL_EDIT_CODEC_CLASS_KEY = "hbase.regionserver.wal.codec";
-
-    private static final String INDEX_HLOG_READER_CLASS_NAME = "org.apache.hadoop.hbase.regionserver.wal.IndexedHLogReader";
-    private static final Log LOG = LogFactory.getLog(IndexManagementUtil.class);
-
-    public static boolean isWALEditCodecSet(Configuration conf) {
-        // check to see if the WALEditCodec is installed
-        try {
-            // Use reflection to load the IndexedWALEditCodec, since it may not load with an older version
-            // of HBase
-            Class.forName(INDEX_WAL_EDIT_CODEC_CLASS_NAME);
-        } catch (Throwable t) {
-            return false;
-        }
-        if (INDEX_WAL_EDIT_CODEC_CLASS_NAME.equals(conf.get(WAL_EDIT_CODEC_CLASS_KEY, null))) {
-            // its installed, and it can handle compression and non-compression cases
-            return true;
-        }
-        return false;
-    }
-
-    public static void ensureMutableIndexingCorrectlyConfigured(Configuration conf) throws IllegalStateException {
-
-        // check to see if the WALEditCodec is installed
-        if (isWALEditCodecSet(conf)) { return; }
-
-        // otherwise, we have to install the indexedhlogreader, but it cannot have compression
-        String codecClass = INDEX_WAL_EDIT_CODEC_CLASS_NAME;
-        String indexLogReaderName = INDEX_HLOG_READER_CLASS_NAME;
-        try {
-            // Use reflection to load the IndexedHLogReader, since it may not load with an older version
-            // of HBase
-            Class.forName(indexLogReaderName);
-        } catch (ClassNotFoundException e) {
-            throw new IllegalStateException(codecClass + " is not installed, but "
-                    + indexLogReaderName + " hasn't been installed in hbase-site.xml under " + HLOG_READER_IMPL_KEY);
-        }
-        if (indexLogReaderName.equals(conf.get(HLOG_READER_IMPL_KEY, indexLogReaderName))) {
-            if (conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false)) { throw new IllegalStateException(
-                    "WAL Compression is only supported with " + codecClass
-                            + ". You can install in hbase-site.xml, under " + WAL_EDIT_CODEC_CLASS_KEY); }
-        } else {
-            throw new IllegalStateException(codecClass + " is not installed, but "
-                    + indexLogReaderName + " hasn't been installed in hbase-site.xml under " + HLOG_READER_IMPL_KEY);
-        }
-
-    }
-
-    public static ValueGetter createGetterFromKeyValues(Collection<KeyValue> pendingUpdates) {
-        final Map<ReferencingColumn, ImmutableBytesPtr> valueMap = Maps.newHashMapWithExpectedSize(pendingUpdates
-                .size());
-        for (KeyValue kv : pendingUpdates) {
-            // create new pointers to each part of the kv
-            ImmutableBytesPtr family = new ImmutableBytesPtr(kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength());
-            ImmutableBytesPtr qual = new ImmutableBytesPtr(kv.getBuffer(), kv.getQualifierOffset(),
-                    kv.getQualifierLength());
-            ImmutableBytesPtr value = new ImmutableBytesPtr(kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
-            valueMap.put(new ReferencingColumn(family, qual), value);
-        }
-        return new ValueGetter() {
-            @Override
-            public ImmutableBytesPtr getLatestValue(ColumnReference ref) throws IOException {
-                return valueMap.get(ReferencingColumn.wrap(ref));
-            }
-        };
-    }
-
-    private static class ReferencingColumn {
-        ImmutableBytesPtr family;
-        ImmutableBytesPtr qual;
-
-        static ReferencingColumn wrap(ColumnReference ref) {
-            ImmutableBytesPtr family = new ImmutableBytesPtr(ref.getFamily());
-            ImmutableBytesPtr qual = new ImmutableBytesPtr(ref.getQualifier());
-            return new ReferencingColumn(family, qual);
-        }
-
-        public ReferencingColumn(ImmutableBytesPtr family, ImmutableBytesPtr qual) {
-            this.family = family;
-            this.qual = qual;
-        }
-
-        @Override
-        public int hashCode() {
-            final int prime = 31;
-            int result = 1;
-            result = prime * result + ((family == null) ? 0 : family.hashCode());
-            result = prime * result + ((qual == null) ? 0 : qual.hashCode());
-            return result;
-        }
-
-        @Override
-        public boolean equals(Object obj) {
-            if (this == obj) return true;
-            if (obj == null) return false;
-            if (getClass() != obj.getClass()) return false;
-            ReferencingColumn other = (ReferencingColumn)obj;
-            if (family == null) {
-                if (other.family != null) return false;
-            } else if (!family.equals(other.family)) return false;
-            if (qual == null) {
-                if (other.qual != null) return false;
-            } else if (!qual.equals(other.qual)) return false;
-            return true;
-        }
-    }
-
-    public static ValueGetter createGetterFromScanner(Scanner scanner, byte[] currentRow) {
-        return new LazyValueGetter(scanner, currentRow);
-    }
-
-    /**
-     * check to see if the kvs in the update match any of the passed columns. Generally, this is useful to for an index
-     * codec to determine if a given update should even be indexed. This assumes that for any index, there are going to
-     * small number of columns, versus the number of kvs in any one batch.
-     */
-    public static boolean updateMatchesColumns(Collection<KeyValue> update, List<ColumnReference> columns) {
-        // check to see if the kvs in the new update even match any of the columns requested
-        // assuming that for any index, there are going to small number of columns, versus the number of
-        // kvs in any one batch.
-        boolean matches = false;
-        outer: for (KeyValue kv : update) {
-            for (ColumnReference ref : columns) {
-                if (ref.matchesFamily(kv.getFamily()) && ref.matchesQualifier(kv.getQualifier())) {
-                    matches = true;
-                    // if a single column matches a single kv, we need to build a whole scanner
-                    break outer;
-                }
-            }
-        }
-        return matches;
-    }
-
-    /**
-     * Check to see if the kvs in the update match any of the passed columns. Generally, this is useful to for an index
-     * codec to determine if a given update should even be indexed. This assumes that for any index, there are going to
-     * small number of kvs, versus the number of columns in any one batch.
-     * <p>
-     * This employs the same logic as {@link #updateMatchesColumns(Collection, List)}, but is flips the iteration logic
-     * to search columns before kvs.
-     */
-    public static boolean columnMatchesUpdate(List<ColumnReference> columns, Collection<KeyValue> update) {
-        boolean matches = false;
-        outer: for (ColumnReference ref : columns) {
-            for (KeyValue kv : update) {
-                if (ref.matchesFamily(kv.getFamily()) && ref.matchesQualifier(kv.getQualifier())) {
-                    matches = true;
-                    // if a single column matches a single kv, we need to build a whole scanner
-                    break outer;
-                }
-            }
-        }
-        return matches;
-    }
-
-    public static Scan newLocalStateScan(List<? extends Iterable<? extends ColumnReference>> refsArray) {
-        Scan s = new Scan();
-        s.setRaw(true);
-        // add the necessary columns to the scan
-        for (Iterable<? extends ColumnReference> refs : refsArray) {
-            for (ColumnReference ref : refs) {
-                s.addFamily(ref.getFamily());
-            }
-        }
-        s.setMaxVersions();
-        return s;
-    }
-
-    /**
-     * Propagate the given failure as a generic {@link IOException}, if it isn't already
-     * 
-     * @param e
-     *            reason indexing failed. If ,tt>null</tt>, throws a {@link NullPointerException}, which should unload
-     *            the coprocessor.
-     */
-    public static void rethrowIndexingException(Throwable e) throws IOException {
-        try {
-            throw e;
-        } catch (IOException e1) {
-            LOG.info("Rethrowing " + e);
-            throw e1;
-        } catch (Throwable e1) {
-            LOG.info("Rethrowing " + e1 + " as a " + IndexBuildingFailureException.class.getSimpleName());
-            throw new IndexBuildingFailureException("Failed to build index for unexpected reason!", e1);
-        }
-    }
-
-    public static void setIfNotSet(Configuration conf, String key, int value) {
-        if (conf.get(key) == null) {
-            conf.setInt(key, value);
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/IndexedKeyValue.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/IndexedKeyValue.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/IndexedKeyValue.java
deleted file mode 100644
index d172153..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/IndexedKeyValue.java
+++ /dev/null
@@ -1,173 +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.hadoop.hbase.index.wal;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.Arrays;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.regionserver.wal.HLog;
-import org.apache.hadoop.hbase.util.Bytes;
-
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-public class IndexedKeyValue extends KeyValue {
-    private static int calcHashCode(ImmutableBytesPtr indexTableName, Mutation mutation) {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + indexTableName.hashCode();
-        result = prime * result + Arrays.hashCode(mutation.getRow());
-        return result;
-    }
-
-    private ImmutableBytesPtr indexTableName;
-    private Mutation mutation;
-    // optimization check to ensure that batches don't get replayed to the index more than once
-    private boolean batchFinished = false;
-    private int hashCode;
-
-    public IndexedKeyValue() {}
-
-    public IndexedKeyValue(byte[] bs, Mutation mutation) {
-        this.indexTableName = new ImmutableBytesPtr(bs);
-        this.mutation = mutation;
-        this.hashCode = calcHashCode(indexTableName, mutation);
-    }
-
-    public byte[] getIndexTable() {
-        return this.indexTableName.get();
-    }
-
-    public Mutation getMutation() {
-        return mutation;
-    }
-
-    /**
-     * This is a KeyValue that shouldn't actually be replayed, so we always mark it as an {@link HLog#METAFAMILY} so it
-     * isn't replayed via the normal replay mechanism
-     */
-    @Override
-    public boolean matchingFamily(final byte[] family) {
-        return Bytes.equals(family, HLog.METAFAMILY);
-    }
-
-    @Override
-    public String toString() {
-        return "IndexWrite:\n\ttable: " + indexTableName + "\n\tmutation:" + mutation;
-    }
-
-    /**
-     * This is a very heavy-weight operation and should only be done when absolutely necessary - it does a full
-     * serialization of the underyling mutation to compare the underlying data.
-     */
-    @Override
-    public boolean equals(Object obj) {
-        if(obj == null) return false;
-        if (this == obj) return true;
-        if (getClass() != obj.getClass()) return false;
-        IndexedKeyValue other = (IndexedKeyValue)obj;
-        if (hashCode() != other.hashCode()) return false;
-        if (!other.indexTableName.equals(this.indexTableName)) return false;
-        byte[] current = this.getMutationBytes();
-        byte[] otherMutation = other.getMutationBytes();
-        return Bytes.equals(current, otherMutation);
-    }
-
-    private byte[] getMutationBytes() {
-        ByteArrayOutputStream bos = null;
-        try {
-            bos = new ByteArrayOutputStream();
-            this.mutation.write(new DataOutputStream(bos));
-            bos.flush();
-            return bos.toByteArray();
-        } catch (IOException e) {
-            throw new IllegalArgumentException("Failed to get bytes for mutation!", e);
-        } finally {
-            if (bos != null) {
-                try {
-                    bos.close();
-                } catch (IOException e) {
-                    throw new IllegalArgumentException("Failed to get bytes for mutation!", e);
-                }
-            }
-        }
-    }
-
-    @Override
-    public int hashCode() {
-        return hashCode;
-    }
-
-    @Override
-    public void write(DataOutput out) throws IOException {
-        KeyValueCodec.write(out, this);
-    }
-
-    /**
-     * Internal write the underlying data for the entry - this does not do any special prefixing. Writing should be done
-     * via {@link KeyValueCodec#write(DataOutput, KeyValue)} to ensure consistent reading/writing of
-     * {@link IndexedKeyValue}s.
-     * 
-     * @param out
-     *            to write data to. Does not close or flush the passed object.
-     * @throws IOException
-     *             if there is a problem writing the underlying data
-     */
-    void writeData(DataOutput out) throws IOException {
-        Bytes.writeByteArray(out, this.indexTableName.get());
-        out.writeUTF(this.mutation.getClass().getName());
-        this.mutation.write(out);
-    }
-
-    /**
-     * This method shouldn't be used - you should use {@link KeyValueCodec#readKeyValue(DataInput)} instead. Its the
-     * complement to {@link #writeData(DataOutput)}.
-     */
-    @SuppressWarnings("javadoc")
-    @Override
-    public void readFields(DataInput in) throws IOException {
-        this.indexTableName = new ImmutableBytesPtr(Bytes.readByteArray(in));
-        Class<? extends Mutation> clazz;
-        try {
-            clazz = Class.forName(in.readUTF()).asSubclass(Mutation.class);
-            this.mutation = clazz.newInstance();
-            this.mutation.readFields(in);
-            this.hashCode = calcHashCode(indexTableName, mutation);
-        } catch (ClassNotFoundException e) {
-            throw new IOException(e);
-        } catch (InstantiationException e) {
-            throw new IOException(e);
-        } catch (IllegalAccessException e) {
-            throw new IOException(e);
-        }
-    }
-
-    public boolean getBatchFinished() {
-        return this.batchFinished;
-    }
-
-    public void markBatchFinished() {
-        this.batchFinished = true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/KeyValueCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/KeyValueCodec.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/KeyValueCodec.java
deleted file mode 100644
index 534dfc0..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/wal/KeyValueCodec.java
+++ /dev/null
@@ -1,97 +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.hadoop.hbase.index.wal;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-
-/**
- * Codec to encode/decode {@link KeyValue}s and {@link IndexedKeyValue}s within a {@link WALEdit}
- */
-public class KeyValueCodec {
-
-  /**
-   * KeyValue length marker specifying that its actually an {@link IndexedKeyValue} rather than a
-   * regular {@link KeyValue}.
-   */
-  public static final int INDEX_TYPE_LENGTH_MARKER = -1;
-
-  /**
-   * Read a {@link List} of {@link KeyValue} from the input stream - may contain regular
-   * {@link KeyValue}s or {@link IndexedKeyValue}s.
-   * @param in to read from
-   * @return the next {@link KeyValue}s
-   * @throws IOException if the next {@link KeyValue} cannot be read
-   */
-  public static List<KeyValue> readKeyValues(DataInput in) throws IOException {
-    int size = in.readInt();
-    if (size == 0) {
-      return Collections.<KeyValue>emptyList();
-    }
-    List<KeyValue> kvs = new ArrayList<KeyValue>(size);
-    for (int i = 0; i < size; i++) {
-      kvs.add(readKeyValue(in));
-    }
-    return kvs;
-  }
-
-  /**
-   * Read a single {@link KeyValue} from the input stream - may either be a regular {@link KeyValue}
-   * or an {@link IndexedKeyValue}.
-   * @param in to read from
-   * @return the next {@link KeyValue}, if one is available
-   * @throws IOException if the next {@link KeyValue} cannot be read
-   */
-  public static KeyValue readKeyValue(DataInput in) throws IOException {
-    int length = in.readInt();
-    KeyValue kv;
-    // its a special IndexedKeyValue
-    if (length == INDEX_TYPE_LENGTH_MARKER) {
-      kv = new IndexedKeyValue();
-      kv.readFields(in);
-    } else {
-      kv = new KeyValue();
-      kv.readFields(length, in);
-    }
-    return kv;
-  }
-
-  /**
-   * Write a {@link KeyValue} or an {@link IndexedKeyValue} to the output stream. These can be read
-   * back via {@link #readKeyValue(DataInput)} or {@link #readKeyValues(DataInput)}.
-   * @param out to write to
-   * @param kv {@link KeyValue} to which to write
-   * @throws IOException if there is an error writing
-   */
-  public static void write(DataOutput out, KeyValue kv) throws IOException {
-    if (kv instanceof IndexedKeyValue) {
-      out.writeInt(INDEX_TYPE_LENGTH_MARKER);
-      ((IndexedKeyValue) kv).writeData(out);
-    } else {
-      kv.write(out);
-    }
-  }
-}
\ No newline at end of file


[11/15] Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexCommitter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexCommitter.java
deleted file mode 100644
index 48e10ce..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexCommitter.java
+++ /dev/null
@@ -1,37 +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.hadoop.hbase.index.write;
-
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-
-import com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.index.exception.IndexWriteException;
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-
-/**
- * Write the index updates to the index tables
- */
-public interface IndexCommitter extends Stoppable {
-
-  void setup(IndexWriter parent, RegionCoprocessorEnvironment env, String name);
-
-  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite)
-      throws IndexWriteException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexFailurePolicy.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexFailurePolicy.java
deleted file mode 100644
index 653e8d6..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexFailurePolicy.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.index.write;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-
-import com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-
-/**
- * Handle failures to write to the index tables.
- */
-public interface IndexFailurePolicy extends Stoppable {
-
-  public void setup(Stoppable parent, RegionCoprocessorEnvironment env);
-
-  /**
-   * Handle the failure of the attempted index updates
-   * @param attempted map of index table -> mutations to apply
-   * @param cause reason why there was a failure
- * @throws IOException 
-   */
-  public void
-      handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause) throws IOException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriter.java
deleted file mode 100644
index f41c55f..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriter.java
+++ /dev/null
@@ -1,224 +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.hadoop.hbase.index.write;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.util.Pair;
-
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.index.exception.IndexWriteException;
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-/**
- * Do the actual work of writing to the index tables. Ensures that if we do fail to write to the
- * index table that we cleanly kill the region/server to ensure that the region's WAL gets replayed.
- * <p>
- * We attempt to do the index updates in parallel using a backing threadpool. All threads are daemon
- * threads, so it will not block the region from shutting down.
- */
-public class IndexWriter implements Stoppable {
-
-  private static final Log LOG = LogFactory.getLog(IndexWriter.class);
-  private static final String INDEX_COMMITTER_CONF_KEY = "index.writer.commiter.class";
-  public static final String INDEX_FAILURE_POLICY_CONF_KEY = "index.writer.failurepolicy.class";
-  private AtomicBoolean stopped = new AtomicBoolean(false);
-  private IndexCommitter writer;
-  private IndexFailurePolicy failurePolicy;
-
-  /**
-   * @throws IOException if the {@link IndexWriter} or {@link IndexFailurePolicy} cannot be
-   *           instantiated
-   */
-  public IndexWriter(RegionCoprocessorEnvironment env, String name) throws IOException {
-    this(getCommitter(env), getFailurePolicy(env), env, name);
-  }
-
-  public static IndexCommitter getCommitter(RegionCoprocessorEnvironment env) throws IOException {
-    Configuration conf = env.getConfiguration();
-    try {
-      IndexCommitter committer =
-          conf.getClass(INDEX_COMMITTER_CONF_KEY, ParallelWriterIndexCommitter.class,
-            IndexCommitter.class).newInstance();
-      return committer;
-    } catch (InstantiationException e) {
-      throw new IOException(e);
-    } catch (IllegalAccessException e) {
-      throw new IOException(e);
-    }
-  }
-
-  public static IndexFailurePolicy getFailurePolicy(RegionCoprocessorEnvironment env)
-      throws IOException {
-    Configuration conf = env.getConfiguration();
-    try {
-      IndexFailurePolicy committer =
-          conf.getClass(INDEX_FAILURE_POLICY_CONF_KEY, KillServerOnFailurePolicy.class,
-            IndexFailurePolicy.class).newInstance();
-      return committer;
-    } catch (InstantiationException e) {
-      throw new IOException(e);
-    } catch (IllegalAccessException e) {
-      throw new IOException(e);
-    }
-  }
-
-  /**
-   * Directly specify the {@link IndexCommitter} and {@link IndexFailurePolicy}. Both are expected
-   * to be fully setup before calling.
-   * @param committer
-   * @param policy
-   * @param env
-   */
-  public IndexWriter(IndexCommitter committer, IndexFailurePolicy policy,
-      RegionCoprocessorEnvironment env, String name) {
-    this(committer, policy);
-    this.writer.setup(this, env, name);
-    this.failurePolicy.setup(this, env);
-  }
-
-  /**
-   * Create an {@link IndexWriter} with an already setup {@link IndexCommitter} and
-   * {@link IndexFailurePolicy}.
-   * @param committer to write updates
-   * @param policy to handle failures
-   */
-  IndexWriter(IndexCommitter committer, IndexFailurePolicy policy) {
-    this.writer = committer;
-    this.failurePolicy = policy;
-  }
-  
-  /**
-   * Write the mutations to their respective table.
-   * <p>
-   * This method is blocking and could potentially cause the writer to block for a long time as we
-   * write the index updates. When we return depends on the specified {@link IndexCommitter}.
-   * <p>
-   * If update fails, we pass along the failure to the installed {@link IndexFailurePolicy}, which
-   * then decides how to handle the failure. By default, we use a {@link KillServerOnFailurePolicy},
-   * which ensures that the server crashes when an index write fails, ensuring that we get WAL
-   * replay of the index edits.
-   * @param indexUpdates Updates to write
- * @throws IOException 
-   */
-  public void writeAndKillYourselfOnFailure(Collection<Pair<Mutation, byte[]>> indexUpdates) throws IOException  {
-    // convert the strings to htableinterfaces to which we can talk and group by TABLE
-    Multimap<HTableInterfaceReference, Mutation> toWrite = resolveTableReferences(indexUpdates);
-    writeAndKillYourselfOnFailure(toWrite);
-  }
-
-  /**
-   * see {@link #writeAndKillYourselfOnFailure(Collection)}.
-   * @param toWrite
- * @throws IOException 
-   */
-  public void writeAndKillYourselfOnFailure(Multimap<HTableInterfaceReference, Mutation> toWrite) throws IOException {
-    try {
-      write(toWrite);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Done writing all index updates!\n\t" + toWrite);
-      }
-    } catch (Exception e) {
-      this.failurePolicy.handleFailure(toWrite, e);
-    }
-  }
-
-  /**
-   * Write the mutations to their respective table.
-   * <p>
-   * This method is blocking and could potentially cause the writer to block for a long time as we
-   * write the index updates. We only return when either:
-   * <ol>
-   * <li>All index writes have returned, OR</li>
-   * <li>Any single index write has failed</li>
-   * </ol>
-   * We attempt to quickly determine if any write has failed and not write to the remaining indexes
-   * to ensure a timely recovery of the failed index writes.
-   * @param toWrite Updates to write
-   * @throws IndexWriteException if we cannot successfully write to the index. Whether or not we
-   *           stop early depends on the {@link IndexCommitter}.
-   */
-  public void write(Collection<Pair<Mutation, byte[]>> toWrite) throws IndexWriteException {
-    write(resolveTableReferences(toWrite));
-  }
-
-  /**
-   * see {@link #write(Collection)}
-   * @param toWrite
-   * @throws IndexWriteException
-   */
-  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite)
-      throws IndexWriteException {
-    this.writer.write(toWrite);
-  }
-
-
-  /**
-   * Convert the passed index updates to {@link HTableInterfaceReference}s.
-   * @param indexUpdates from the index builder
-   * @return pairs that can then be written by an {@link IndexWriter}.
-   */
-  public static Multimap<HTableInterfaceReference, Mutation> resolveTableReferences(
-      Collection<Pair<Mutation, byte[]>> indexUpdates) {
-    Multimap<HTableInterfaceReference, Mutation> updates = ArrayListMultimap
-        .<HTableInterfaceReference, Mutation> create();
-    // simple map to make lookups easy while we build the map of tables to create
-    Map<ImmutableBytesPtr, HTableInterfaceReference> tables =
-        new HashMap<ImmutableBytesPtr, HTableInterfaceReference>(updates.size());
-    for (Pair<Mutation, byte[]> entry : indexUpdates) {
-      byte[] tableName = entry.getSecond();
-      ImmutableBytesPtr ptr = new ImmutableBytesPtr(tableName);
-      HTableInterfaceReference table = tables.get(ptr);
-      if (table == null) {
-        table = new HTableInterfaceReference(ptr);
-        tables.put(ptr, table);
-      }
-      updates.put(table, entry.getFirst());
-    }
-
-    return updates;
-  }
-
-  @Override
-  public void stop(String why) {
-    if (!this.stopped.compareAndSet(false, true)) {
-      // already stopped
-      return;
-    }
-    LOG.debug("Stopping because " + why);
-    this.writer.stop(why);
-    this.failurePolicy.stop(why);
-  }
-
-  @Override
-  public boolean isStopped() {
-    return this.stopped.get();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriterUtils.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriterUtils.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriterUtils.java
deleted file mode 100644
index b56a23e..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/IndexWriterUtils.java
+++ /dev/null
@@ -1,72 +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.hadoop.hbase.index.write;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CoprocessorEnvironment;
-
-import org.apache.hadoop.hbase.index.table.CoprocessorHTableFactory;
-import org.apache.hadoop.hbase.index.table.HTableFactory;
-import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
-
-public class IndexWriterUtils {
-
-  private static final Log LOG = LogFactory.getLog(IndexWriterUtils.class);
-
-  /**
-   * Maximum number of threads to allow per-table when writing. Each writer thread (from
-   * {@link IndexWriterUtils#NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY}) has a single HTable.
-   * However, each table is backed by a threadpool to manage the updates to that table. this
-   * specifies the number of threads to allow in each of those tables. Generally, you shouldn't need
-   * to change this, unless you have a small number of indexes to which most of the writes go.
-   * Defaults to: {@value #DEFAULT_NUM_PER_TABLE_THREADS}.
-   * <p>
-   * For tables to which there are not a lot of writes, the thread pool automatically will decrease
-   * the number of threads to one (though it can burst up to the specified max for any given table),
-   * so increasing this to meet the max case is reasonable.
-   * <p>
-   * Setting this value too small can cause <b>catastrophic cluster failure</b>. The way HTable's
-   * underlying pool works is such that is does direct hand-off of tasks to threads. This works fine
-   * because HTables are assumed to work in a single-threaded context, so we never get more threads
-   * than regionservers. In a multi-threaded context, we can easily grow to more than that number of
-   * threads. Currently, HBase doesn't support a custom thread-pool to back the HTable via the
-   * coprocesor hooks, so we can't modify this behavior.
-   */
-  private static final String INDEX_WRITER_PER_TABLE_THREADS_CONF_KEY =
-      "index.writer.threads.pertable.max";
-  private static final int DEFAULT_NUM_PER_TABLE_THREADS = Integer.MAX_VALUE;
-
-  /** Configuration key that HBase uses to set the max number of threads for an HTable */
-  public static final String HTABLE_THREAD_KEY = "hbase.htable.threads.max";
-  private IndexWriterUtils() {
-    // private ctor for utilites
-  }
-
-  public static HTableFactory getDefaultDelegateHTableFactory(CoprocessorEnvironment env) {
-    // create a simple delegate factory, setup the way we need
-    Configuration conf = env.getConfiguration();
-    // set the number of threads allowed per table.
-    int htableThreads =
-        conf.getInt(IndexWriterUtils.INDEX_WRITER_PER_TABLE_THREADS_CONF_KEY, IndexWriterUtils.DEFAULT_NUM_PER_TABLE_THREADS);
-    LOG.trace("Creating HTableFactory with " + htableThreads + " threads for each HTable.");
-    IndexManagementUtil.setIfNotSet(conf, HTABLE_THREAD_KEY, htableThreads);
-    return new CoprocessorHTableFactory(env);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/KillServerOnFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/KillServerOnFailurePolicy.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/KillServerOnFailurePolicy.java
deleted file mode 100644
index c043a54..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/KillServerOnFailurePolicy.java
+++ /dev/null
@@ -1,80 +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.hadoop.hbase.index.write;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-
-import com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-
-/**
- * Naive failure policy - kills the server on which it resides
- */
-public class KillServerOnFailurePolicy implements IndexFailurePolicy {
-
-  private static final Log LOG = LogFactory.getLog(KillServerOnFailurePolicy.class);
-  private Abortable abortable;
-  private Stoppable stoppable;
-
-  @Override
-  public void setup(Stoppable parent, RegionCoprocessorEnvironment env) {
-    setup(parent, env.getRegionServerServices());
-  }
-
-  public void setup(Stoppable parent, Abortable abort) {
-    this.stoppable = parent;
-    this.abortable = abort;
-  }
-
-  @Override
-  public void stop(String why) {
-    // noop
-  }
-
-  @Override
-  public boolean isStopped() {
-    return this.stoppable.isStopped();
-  }
-
-  @Override
-  public void
-      handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause) throws IOException {
-    // cleanup resources
-    this.stop("Killing ourselves because of an error:" + cause);
-    // notify the regionserver of the failure
-    String msg =
-        "Could not update the index table, killing server region because couldn't write to an index table";
-    LOG.error(msg, cause);
-    try {
-      this.abortable.abort(msg, cause);
-    } catch (Exception e) {
-      LOG.fatal("Couldn't abort this server to preserve index writes, "
-          + "attempting to hard kill the server");
-      System.exit(1);
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/ParallelWriterIndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/ParallelWriterIndexCommitter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/ParallelWriterIndexCommitter.java
deleted file mode 100644
index b06ecf6..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/ParallelWriterIndexCommitter.java
+++ /dev/null
@@ -1,210 +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.hadoop.hbase.index.write;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-
-import com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.index.exception.SingleIndexWriteFailureException;
-import org.apache.hadoop.hbase.index.parallel.EarlyExitFailure;
-import org.apache.hadoop.hbase.index.parallel.QuickFailingTaskRunner;
-import org.apache.hadoop.hbase.index.parallel.Task;
-import org.apache.hadoop.hbase.index.parallel.TaskBatch;
-import org.apache.hadoop.hbase.index.parallel.ThreadPoolBuilder;
-import org.apache.hadoop.hbase.index.parallel.ThreadPoolManager;
-import org.apache.hadoop.hbase.index.table.CachingHTableFactory;
-import org.apache.hadoop.hbase.index.table.HTableFactory;
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-
-/**
- * Write index updates to the index tables in parallel. We attempt to early exit from the writes if
- * any of the index updates fails. Completion is determined by the following criteria: *
- * <ol>
- * <li>All index writes have returned, OR</li>
- * <li>Any single index write has failed</li>
- * </ol>
- * We attempt to quickly determine if any write has failed and not write to the remaining indexes to
- * ensure a timely recovery of the failed index writes.
- */
-public class ParallelWriterIndexCommitter implements IndexCommitter {
-
-  public static final String NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY = "index.writer.threads.max";
-  private static final int DEFAULT_CONCURRENT_INDEX_WRITER_THREADS = 10;
-  private static final String INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY =
-      "index.writer.threads.keepalivetime";
-  private static final Log LOG = LogFactory.getLog(ParallelWriterIndexCommitter.class);
-
-  private HTableFactory factory;
-  private Stoppable stopped;
-  private QuickFailingTaskRunner pool;
-
-  @Override
-  public void setup(IndexWriter parent, RegionCoprocessorEnvironment env, String name) {
-    Configuration conf = env.getConfiguration();
-    setup(IndexWriterUtils.getDefaultDelegateHTableFactory(env),
-      ThreadPoolManager.getExecutor(
-        new ThreadPoolBuilder(name, conf).
-          setMaxThread(NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY,
-            DEFAULT_CONCURRENT_INDEX_WRITER_THREADS).
-          setCoreTimeout(INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY), env),
-      env.getRegionServerServices(), parent, CachingHTableFactory.getCacheSize(conf));
-  }
-
-  /**
-   * Setup <tt>this</tt>.
-   * <p>
-   * Exposed for TESTING
-   */
-  void setup(HTableFactory factory, ExecutorService pool, Abortable abortable, Stoppable stop,
-      int cacheSize) {
-    this.factory = new CachingHTableFactory(factory, cacheSize);
-    this.pool = new QuickFailingTaskRunner(pool);
-    this.stopped = stop;
-  }
-
-  @Override
-  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite)
-      throws SingleIndexWriteFailureException {
-    /*
-     * This bit here is a little odd, so let's explain what's going on. Basically, we want to do the
-     * writes in parallel to each index table, so each table gets its own task and is submitted to
-     * the pool. Where it gets tricky is that we want to block the calling thread until one of two
-     * things happens: (1) all index tables get successfully updated, or (2) any one of the index
-     * table writes fail; in either case, we should return as quickly as possible. We get a little
-     * more complicated in that if we do get a single failure, but any of the index writes hasn't
-     * been started yet (its been queued up, but not submitted to a thread) we want to that task to
-     * fail immediately as we know that write is a waste and will need to be replayed anyways.
-     */
-
-    Set<Entry<HTableInterfaceReference, Collection<Mutation>>> entries = toWrite.asMap().entrySet();
-    TaskBatch<Void> tasks = new TaskBatch<Void>(entries.size());
-    for (Entry<HTableInterfaceReference, Collection<Mutation>> entry : entries) {
-      // get the mutations for each table. We leak the implementation here a little bit to save
-      // doing a complete copy over of all the index update for each table.
-      final List<Mutation> mutations = (List<Mutation>) entry.getValue();
-      final HTableInterfaceReference tableReference = entry.getKey();
-      /*
-       * Write a batch of index updates to an index table. This operation stops (is cancelable) via
-       * two mechanisms: (1) setting aborted or stopped on the IndexWriter or, (2) interrupting the
-       * running thread. The former will only work if we are not in the midst of writing the current
-       * batch to the table, though we do check these status variables before starting and before
-       * writing the batch. The latter usage, interrupting the thread, will work in the previous
-       * situations as was at some points while writing the batch, depending on the underlying
-       * writer implementation (HTableInterface#batch is blocking, but doesn't elaborate when is
-       * supports an interrupt).
-       */
-      tasks.add(new Task<Void>() {
-
-        /**
-         * Do the actual write to the primary table. We don't need to worry about closing the table
-         * because that is handled the {@link CachingHTableFactory}.
-         */
-        @Override
-        public Void call() throws Exception {
-          // this may have been queued, so another task infront of us may have failed, so we should
-          // early exit, if that's the case
-          throwFailureIfDone();
-
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Writing index update:" + mutations + " to table: " + tableReference);
-          }
-          try {
-            HTableInterface table = factory.getTable(tableReference.get());
-            throwFailureIfDone();
-            table.batch(mutations);
-          } catch (SingleIndexWriteFailureException e) {
-            throw e;
-          } catch (IOException e) {
-            throw new SingleIndexWriteFailureException(tableReference.toString(), mutations, e);
-          } catch (InterruptedException e) {
-            // reset the interrupt status on the thread
-            Thread.currentThread().interrupt();
-            throw new SingleIndexWriteFailureException(tableReference.toString(), mutations, e);
-          }
-          return null;
-        }
-
-        private void throwFailureIfDone() throws SingleIndexWriteFailureException {
-          if (this.isBatchFailed() || Thread.currentThread().isInterrupted()) {
-            throw new SingleIndexWriteFailureException(
-                "Pool closed, not attempting to write to the index!", null);
-          }
-
-        }
-      });
-    }
-
-    // actually submit the tasks to the pool and wait for them to finish/fail
-    try {
-      pool.submitUninterruptible(tasks);
-    } catch (EarlyExitFailure e) {
-      propagateFailure(e);
-    } catch (ExecutionException e) {
-      LOG.error("Found a failed index update!");
-      propagateFailure(e.getCause());
-    }
-
-  }
-
-  private void propagateFailure(Throwable throwable) throws SingleIndexWriteFailureException {
-    try {
-      throw throwable;
-    } catch (SingleIndexWriteFailureException e1) {
-      throw e1;
-    } catch (Throwable e1) {
-      throw new SingleIndexWriteFailureException(
-          "Got an abort notification while writing to the index!", e1);
-    }
-
-  }
-
-  /**
-   * {@inheritDoc}
-   * <p>
-   * This method should only be called <b>once</b>. Stopped state ({@link #isStopped()}) is managed
-   * by the external {@link Stoppable}. This call does not delegate the stop down to the
-   * {@link Stoppable} passed in the constructor.
-   * @param why the reason for stopping
-   */
-  @Override
-  public void stop(String why) {
-    LOG.info("Shutting down " + this.getClass().getSimpleName() + " because " + why);
-    this.pool.stop(why);
-    this.factory.shutdown();
-  }
-
-  @Override
-  public boolean isStopped() {
-    return this.stopped.isStopped();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/PerRegionIndexWriteCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/PerRegionIndexWriteCache.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/PerRegionIndexWriteCache.java
deleted file mode 100644
index b0f9d68..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/PerRegionIndexWriteCache.java
+++ /dev/null
@@ -1,63 +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.hadoop.hbase.index.write.recovery;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-
-
-public class PerRegionIndexWriteCache {
-
-  private Map<HRegion, Multimap<HTableInterfaceReference, Mutation>> cache =
-      new HashMap<HRegion, Multimap<HTableInterfaceReference, Mutation>>();
-
-
-  /**
-   * Get the edits for the current region. Removes the edits from the cache. To add them back, call
-   * {@link #addEdits(HRegion, HTableInterfaceReference, Collection)}.
-   * @param region
-   * @return Get the edits for the given region. Returns <tt>null</tt> if there are no pending edits
-   *         for the region
-   */
-  public Multimap<HTableInterfaceReference, Mutation> getEdits(HRegion region) {
-    return cache.remove(region);
-  }
-
-  /**
-   * @param region
-   * @param table
-   * @param collection
-   */
-  public void addEdits(HRegion region, HTableInterfaceReference table,
-      Collection<Mutation> collection) {
-    Multimap<HTableInterfaceReference, Mutation> edits = cache.get(region);
-    if (edits == null) {
-      edits = ArrayListMultimap.<HTableInterfaceReference, Mutation> create();
-      cache.put(region, edits);
-    }
-    edits.putAll(table, collection);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/StoreFailuresInCachePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/StoreFailuresInCachePolicy.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/StoreFailuresInCachePolicy.java
deleted file mode 100644
index a17395e..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/StoreFailuresInCachePolicy.java
+++ /dev/null
@@ -1,84 +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.hadoop.hbase.index.write.recovery;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-
-import com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.index.exception.MultiIndexWriteFailureException;
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-import org.apache.hadoop.hbase.index.write.IndexFailurePolicy;
-import org.apache.hadoop.hbase.index.write.KillServerOnFailurePolicy;
-
-/**
- * Tracks any failed writes in The {@link PerRegionIndexWriteCache}, given a
- * {@link MultiIndexWriteFailureException} (which is thrown from the
- * {@link TrackingParallelWriterIndexCommitter}. Any other exception failure causes the a server
- * abort via the usual {@link KillServerOnFailurePolicy}.
- */
-public class StoreFailuresInCachePolicy implements IndexFailurePolicy {
-
-  private KillServerOnFailurePolicy delegate;
-  private PerRegionIndexWriteCache cache;
-  private HRegion region;
-
-  /**
-   * @param failedIndexEdits cache to update when we find a failure
-   */
-  public StoreFailuresInCachePolicy(PerRegionIndexWriteCache failedIndexEdits) {
-    this.cache = failedIndexEdits;
-  }
-
-  @Override
-  public void setup(Stoppable parent, RegionCoprocessorEnvironment env) {
-    this.region = env.getRegion();
-    this.delegate = new KillServerOnFailurePolicy();
-    this.delegate.setup(parent, env);
-
-  }
-
-  @Override
-  public void handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause) throws IOException {
-    // if its not an exception we can handle, let the delegate take care of it
-    if (!(cause instanceof MultiIndexWriteFailureException)) {
-      delegate.handleFailure(attempted, cause);
-    }
-    List<HTableInterfaceReference> failedTables =
-        ((MultiIndexWriteFailureException) cause).getFailedTables();
-    for (HTableInterfaceReference table : failedTables) {
-      cache.addEdits(this.region, table, attempted.get(table));
-    }
-  }
-
-
-  @Override
-  public void stop(String why) {
-    this.delegate.stop(why);
-  }
-
-  @Override
-  public boolean isStopped() {
-    return this.delegate.isStopped();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
deleted file mode 100644
index 43cc19f..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
+++ /dev/null
@@ -1,226 +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.hadoop.hbase.index.write.recovery;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-
-import com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.index.CapturingAbortable;
-import org.apache.hadoop.hbase.index.exception.MultiIndexWriteFailureException;
-import org.apache.hadoop.hbase.index.exception.SingleIndexWriteFailureException;
-import org.apache.hadoop.hbase.index.parallel.EarlyExitFailure;
-import org.apache.hadoop.hbase.index.parallel.Task;
-import org.apache.hadoop.hbase.index.parallel.TaskBatch;
-import org.apache.hadoop.hbase.index.parallel.TaskRunner;
-import org.apache.hadoop.hbase.index.parallel.ThreadPoolBuilder;
-import org.apache.hadoop.hbase.index.parallel.ThreadPoolManager;
-import org.apache.hadoop.hbase.index.parallel.WaitForCompletionTaskRunner;
-import org.apache.hadoop.hbase.index.table.CachingHTableFactory;
-import org.apache.hadoop.hbase.index.table.HTableFactory;
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-import org.apache.hadoop.hbase.index.write.IndexCommitter;
-import org.apache.hadoop.hbase.index.write.IndexWriter;
-import org.apache.hadoop.hbase.index.write.IndexWriterUtils;
-import org.apache.hadoop.hbase.index.write.ParallelWriterIndexCommitter;
-
-/**
- * Like the {@link ParallelWriterIndexCommitter}, but blocks until all writes have attempted to
- * allow the caller to retrieve the failed and succeeded index updates. Therefore, this class will
- * be a lot slower, in the face of failures, when compared to the
- * {@link ParallelWriterIndexCommitter} (though as fast for writes), so it should be used only when
- * you need to at least attempt all writes and know their result; for instance, this is fine for
- * doing WAL recovery - it's not a performance intensive situation and we want to limit the the
- * edits we need to retry.
- * <p>
- * On failure to {@link #write(Multimap)}, we return a {@link MultiIndexWriteFailureException} that
- * contains the list of {@link HTableInterfaceReference} that didn't complete successfully.
- * <p>
- * Failures to write to the index can happen several different ways:
- * <ol>
- * <li><tt>this</tt> is {@link #stop(String) stopped} or aborted (via the passed {@link Abortable}.
- * This causing any pending tasks to fail whatever they are doing as fast as possible. Any writes
- * that have not begun are not even attempted and marked as failures.</li>
- * <li>A batch write fails. This is the generic HBase write failure - it may occur because the index
- * table is not available, .META. or -ROOT- is unavailable, or any other (of many) possible HBase
- * exceptions.</li>
- * </ol>
- * Regardless of how the write fails, we still wait for all writes to complete before passing the
- * failure back to the client.
- */
-public class TrackingParallelWriterIndexCommitter implements IndexCommitter {
-  private static final Log LOG = LogFactory.getLog(TrackingParallelWriterIndexCommitter.class);
-
-  public static final String NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY = "index.trackingwriter.threads.max";
-  private static final int DEFAULT_CONCURRENT_INDEX_WRITER_THREADS = 10;
-  private static final String INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY =
-      "index.trackingwriter.threads.keepalivetime";
-  
-  private TaskRunner pool;
-  private HTableFactory factory;
-  private CapturingAbortable abortable;
-  private Stoppable stopped;
-
-  @Override
-  public void setup(IndexWriter parent, RegionCoprocessorEnvironment env, String name) {
-    Configuration conf = env.getConfiguration();
-    setup(IndexWriterUtils.getDefaultDelegateHTableFactory(env),
-      ThreadPoolManager.getExecutor(
-        new ThreadPoolBuilder(name, conf).
-          setMaxThread(NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY,
-            DEFAULT_CONCURRENT_INDEX_WRITER_THREADS).
-          setCoreTimeout(INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY), env),
-      env.getRegionServerServices(), parent, CachingHTableFactory.getCacheSize(conf));
-  }
-
-  /**
-   * Setup <tt>this</tt>.
-   * <p>
-   * Exposed for TESTING
-   */
-  void setup(HTableFactory factory, ExecutorService pool, Abortable abortable, Stoppable stop,
-      int cacheSize) {
-    this.pool = new WaitForCompletionTaskRunner(pool);
-    this.factory = new CachingHTableFactory(factory, cacheSize);
-    this.abortable = new CapturingAbortable(abortable);
-    this.stopped = stop;
-  }
-
-  @Override
-  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite)
-      throws MultiIndexWriteFailureException {
-    Set<Entry<HTableInterfaceReference, Collection<Mutation>>> entries = toWrite.asMap().entrySet();
-    TaskBatch<Boolean> tasks = new TaskBatch<Boolean>(entries.size());
-    List<HTableInterfaceReference> tables = new ArrayList<HTableInterfaceReference>(entries.size());
-    for (Entry<HTableInterfaceReference, Collection<Mutation>> entry : entries) {
-      // get the mutations for each table. We leak the implementation here a little bit to save
-      // doing a complete copy over of all the index update for each table.
-      final List<Mutation> mutations = (List<Mutation>) entry.getValue();
-      // track each reference so we can get at it easily later, when determing failures
-      final HTableInterfaceReference tableReference = entry.getKey();
-      tables.add(tableReference);
-
-      /*
-       * Write a batch of index updates to an index table. This operation stops (is cancelable) via
-       * two mechanisms: (1) setting aborted or stopped on the IndexWriter or, (2) interrupting the
-       * running thread. The former will only work if we are not in the midst of writing the current
-       * batch to the table, though we do check these status variables before starting and before
-       * writing the batch. The latter usage, interrupting the thread, will work in the previous
-       * situations as was at some points while writing the batch, depending on the underlying
-       * writer implementation (HTableInterface#batch is blocking, but doesn't elaborate when is
-       * supports an interrupt).
-       */
-      tasks.add(new Task<Boolean>() {
-
-        /**
-         * Do the actual write to the primary table. We don't need to worry about closing the table
-         * because that is handled the {@link CachingHTableFactory}.
-         */
-        @Override
-        public Boolean call() throws Exception {
-          try {
-            // this may have been queued, but there was an abort/stop so we try to early exit
-            throwFailureIfDone();
-
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Writing index update:" + mutations + " to table: " + tableReference);
-            }
-            HTableInterface table = factory.getTable(tableReference.get());
-            throwFailureIfDone();
-            table.batch(mutations);
-          } catch (InterruptedException e) {
-            // reset the interrupt status on the thread
-            Thread.currentThread().interrupt();
-            throw e;
-          } catch (Exception e) {
-            throw e;
-          }
-          return Boolean.TRUE;
-        }
-
-        private void throwFailureIfDone() throws SingleIndexWriteFailureException {
-          if (stopped.isStopped() || abortable.isAborted()
-              || Thread.currentThread().isInterrupted()) {
-            throw new SingleIndexWriteFailureException(
-                "Pool closed, not attempting to write to the index!", null);
-          }
-
-        }
-      });
-    }
-
-    List<Boolean> results = null;
-    try {
-      LOG.debug("Waiting on index update tasks to complete...");
-      results = this.pool.submitUninterruptible(tasks);
-    } catch (ExecutionException e) {
-      throw new RuntimeException(
-          "Should not fail on the results while using a WaitForCompletionTaskRunner", e);
-    } catch (EarlyExitFailure e) {
-      throw new RuntimeException("Stopped while waiting for batch, quiting!", e);
-    }
-    
-    // track the failures. We only ever access this on return from our calls, so no extra
-    // synchronization is needed. We could update all the failures as we find them, but that add a
-    // lot of locking overhead, and just doing the copy later is about as efficient.
-    List<HTableInterfaceReference> failures = new ArrayList<HTableInterfaceReference>();
-    int index = 0;
-    for (Boolean result : results) {
-      // there was a failure
-      if (result == null) {
-        // we know which table failed by the index of the result
-        failures.add(tables.get(index));
-      }
-      index++;
-    }
-
-    // if any of the tasks failed, then we need to propagate the failure
-    if (failures.size() > 0) {
-      // make the list unmodifiable to avoid any more synchronization concerns
-      throw new MultiIndexWriteFailureException(Collections.unmodifiableList(failures));
-    }
-    return;
-  }
-
-  @Override
-  public void stop(String why) {
-    LOG.info("Shutting down " + this.getClass().getSimpleName());
-    this.pool.stop(why);
-    this.factory.shutdown();
-  }
-
-  @Override
-  public boolean isStopped() {
-    return this.stopped.isStopped();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEdit.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEdit.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEdit.java
index 9c31900..59b1aa8 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEdit.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEdit.java
@@ -46,7 +46,7 @@ import java.util.TreeMap;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import org.apache.hadoop.hbase.index.wal.KeyValueCodec;
+import org.apache.phoenix.hbase.index.wal.KeyValueCodec;
 
 /**
  * Read in data for a delegate {@link WALEdit}. This should only be used in concert with an IndexedHLogReader

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEditCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEditCodec.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEditCodec.java
index 5e78d3b..5d87433 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEditCodec.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEditCodec.java
@@ -31,8 +31,8 @@ import org.apache.hadoop.hbase.codec.BaseEncoder;
 import org.apache.hadoop.hbase.codec.Decoder;
 import org.apache.hadoop.hbase.codec.Encoder;
 
-import org.apache.hadoop.hbase.index.wal.IndexedKeyValue;
-import org.apache.hadoop.hbase.index.wal.KeyValueCodec;
+import org.apache.phoenix.hbase.index.wal.IndexedKeyValue;
+import org.apache.phoenix.hbase.index.wal.KeyValueCodec;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
index 624a0e5..01bbf06 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.memory.ChildMemoryManager;
 import org.apache.phoenix.memory.GlobalMemoryManager;
 import org.apache.phoenix.query.QueryServices;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/cache/HashCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/HashCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/HashCache.java
index 4c13210..e604f63 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/HashCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/HashCache.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.http.annotation.Immutable;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.schema.tuple.Tuple;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java
index 6a3e8a0..b968a9b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java
@@ -22,8 +22,8 @@ import java.sql.SQLException;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.coprocessor.ServerCachingProtocol.ServerCacheFactory;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.memory.MemoryManager;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java
index 8d6e81c..d9800c0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java
@@ -25,8 +25,8 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
 import com.google.common.cache.*;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.coprocessor.ServerCachingProtocol.ServerCacheFactory;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.memory.MemoryManager;
 import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
 import org.apache.phoenix.util.Closeables;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillManager.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillManager.java
index ab8bcbd..02ecb05 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillManager.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillManager.java
@@ -35,10 +35,10 @@ import org.apache.hadoop.io.WritableUtils;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.io.Closeables;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.expression.aggregator.Aggregator;
 import org.apache.phoenix.expression.aggregator.ServerAggregators;
 import org.apache.phoenix.expression.function.SingleAggregateFunction;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.ValueBitSet;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillMap.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillMap.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillMap.java
index 9bcb6c8..bb4ce2e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillMap.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillMap.java
@@ -36,7 +36,8 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.hash.BloomFilter;
 import com.google.common.hash.Funnels;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 
 /**
  * Class implements an active spilled partition serialized tuples are first written into an in-memory data structure

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
index b6e3949..fdc2b1d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
@@ -44,7 +44,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.io.Closeables;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.cache.GlobalCache;
 import org.apache.phoenix.cache.TenantCache;
 import org.apache.phoenix.cache.aggcache.SpillManager.CacheEntry;
@@ -53,6 +52,7 @@ import org.apache.phoenix.coprocessor.GroupByCache;
 import org.apache.phoenix.coprocessor.GroupedAggregateRegionObserver;
 import org.apache.phoenix.expression.aggregator.Aggregator;
 import org.apache.phoenix.expression.aggregator.ServerAggregators;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.memory.InsufficientMemoryException;
 import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
 import org.apache.phoenix.util.KeyValueUtil;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/client/GenericKeyValueBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/client/GenericKeyValueBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/client/GenericKeyValueBuilder.java
index 8f91c2f..27124bb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/client/GenericKeyValueBuilder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/client/GenericKeyValueBuilder.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
-import static org.apache.hadoop.hbase.index.util.ImmutableBytesPtr.copyBytesIfNecessary;
+import static org.apache.phoenix.hbase.index.util.ImmutableBytesPtr.copyBytesIfNecessary;
 
 /**
  * {@link KeyValueBuilder} that does simple byte[] copies to build the underlying key-value. This is

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index 9004bb5..c5ddcdf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -29,7 +29,6 @@ import java.util.Map;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.FilterList;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
 import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
@@ -39,6 +38,7 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.AggregatePlan;
 import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.index.IndexMetaDataCacheClient;
 import org.apache.phoenix.index.PhoenixIndexCodec;
 import org.apache.phoenix.iterate.ResultIterator;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
index 324a557..9ead4c5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
@@ -23,7 +23,6 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
 import org.apache.phoenix.compile.JoinCompiler.JoinSpec;
@@ -38,6 +37,7 @@ import org.apache.phoenix.execute.DegenerateQueryPlan;
 import org.apache.phoenix.execute.HashJoinPlan;
 import org.apache.phoenix.execute.ScanPlan;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.iterate.ParallelIterators.ParallelIteratorFactory;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index 86857f4..1a85c38 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -30,7 +30,6 @@ import java.util.Map;
 
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
@@ -43,6 +42,7 @@ import org.apache.phoenix.execute.AggregatePlan;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.index.IndexMetaDataCacheClient;
 import org.apache.phoenix.index.PhoenixIndexCodec;
 import org.apache.phoenix.iterate.ParallelIterators.ParallelIteratorFactory;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
index 4b4c213..d6c0063 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
@@ -52,7 +52,6 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Maps;
 import com.google.common.io.Closeables;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.cache.GlobalCache;
 import org.apache.phoenix.cache.TenantCache;
 import org.apache.phoenix.cache.aggcache.SpillableGroupByCache;
@@ -60,6 +59,7 @@ import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.ExpressionType;
 import org.apache.phoenix.expression.aggregator.Aggregator;
 import org.apache.phoenix.expression.aggregator.ServerAggregators;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.join.HashJoinInfo;
 import org.apache.phoenix.join.ScanProjector;
 import org.apache.phoenix.memory.MemoryManager.MemoryChunk;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
index fd93df9..528c97c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
@@ -31,11 +31,11 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.cache.GlobalCache;
 import org.apache.phoenix.cache.HashCache;
 import org.apache.phoenix.cache.TenantCache;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.join.HashJoinInfo;
 import org.apache.phoenix.join.ScanProjector;
 import org.apache.phoenix.join.ScanProjector.ProjectedValueTuple;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index e85f8c9..9811511 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -75,13 +75,13 @@ import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.phoenix.cache.GlobalCache;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.AmbiguousColumnException;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java
index 4df68a4..6dea838 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java
@@ -23,9 +23,9 @@ import org.apache.hadoop.hbase.coprocessor.BaseEndpointCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.cache.GlobalCache;
 import org.apache.phoenix.cache.TenantCache;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 
 
 

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index 06347a4..2e5f2d7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -20,7 +20,7 @@ package org.apache.phoenix.exception;
 import java.sql.SQLException;
 import java.util.Map;
 
-import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.schema.AmbiguousColumnException;
 import org.apache.phoenix.schema.AmbiguousTableException;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
index ccc9ef4..692b9e2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
@@ -27,7 +27,6 @@ import java.util.concurrent.Future;
 import org.apache.hadoop.hbase.client.Scan;
 
 import com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
 import org.apache.phoenix.compile.ExplainPlan;
 import org.apache.phoenix.compile.QueryPlan;
@@ -37,6 +36,7 @@ import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
 import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.iterate.ResultIterator;
 import org.apache.phoenix.job.JobManager.JobCallable;
 import org.apache.phoenix.join.HashCacheClient;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 8701f2a..19dec71 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -36,11 +36,11 @@ import org.slf4j.LoggerFactory;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.cache.ServerCacheClient;
 import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.index.IndexMetaDataCacheClient;
 import org.apache.phoenix.index.PhoenixIndexCodec;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
index f8b4f2d..1a51107 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/InListExpression.java
@@ -27,11 +27,11 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.schema.ConstraintViolationException;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.SortOrder;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java
index 7044848..576ce7c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java
@@ -21,11 +21,11 @@ import java.math.*;
 import java.util.List;
 import java.util.Map.Entry;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.expression.ColumnExpression;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseStddevAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseStddevAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseStddevAggregator.java
index 33794d4..a1ed1df 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseStddevAggregator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseStddevAggregator.java
@@ -21,9 +21,9 @@ import java.math.BigDecimal;
 import java.util.List;
 import java.util.Map.Entry;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountClientAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountClientAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountClientAggregator.java
index b7dc554..f29f46a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountClientAggregator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountClientAggregator.java
@@ -27,10 +27,10 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.TreeMap;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountServerAggregator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountServerAggregator.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountServerAggregator.java
index 146bfb3..fa44038 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountServerAggregator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/DistinctValueWithCountServerAggregator.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PDataType;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiCQKeyValueComparisonFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiCQKeyValueComparisonFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiCQKeyValueComparisonFilter.java
index e691c41..e46b435 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiCQKeyValueComparisonFilter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiCQKeyValueComparisonFilter.java
@@ -19,8 +19,8 @@ package org.apache.phoenix.filter;
 
 import org.apache.hadoop.hbase.util.Bytes;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/CapturingAbortable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/CapturingAbortable.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/CapturingAbortable.java
new file mode 100644
index 0000000..c52e749
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/CapturingAbortable.java
@@ -0,0 +1,66 @@
+/*
+ * 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.phoenix.hbase.index;
+
+import org.apache.hadoop.hbase.Abortable;
+
+/**
+ * {@link Abortable} that can rethrow the cause of the abort.
+ */
+public class CapturingAbortable implements Abortable {
+
+  private Abortable delegate;
+  private Throwable cause;
+  private String why;
+
+  public CapturingAbortable(Abortable delegate) {
+    this.delegate = delegate;
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+    if (delegate.isAborted()) {
+      return;
+    }
+    this.why = why;
+    this.cause = e;
+    delegate.abort(why, e);
+
+  }
+
+  @Override
+  public boolean isAborted() {
+    return delegate.isAborted();
+  }
+
+  /**
+   * Throw the cause of the abort, if <tt>this</tt> was aborted. If there was an exception causing
+   * the abort, re-throws that. Otherwise, just throws a generic {@link Exception} with the reason
+   * why the abort was caused.
+   * @throws Throwable the cause of the abort.
+   */
+  public void throwCauseIfAborted() throws Throwable {
+    if (!this.isAborted()) {
+      return;
+    }
+    if (cause == null) {
+      throw new Exception(why);
+    }
+    throw cause;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexLogRollSynchronizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexLogRollSynchronizer.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexLogRollSynchronizer.java
new file mode 100644
index 0000000..93f2c3e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexLogRollSynchronizer.java
@@ -0,0 +1,123 @@
+/*
+ * 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.phoenix.hbase.index;
+
+import java.io.IOException;
+import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+
+/**
+ * Ensure that the log isn't rolled while we are the in middle of doing a pending index write.
+ * <p>
+ * The problem we are trying to solve is the following sequence:
+ * <ol>
+ * <li>Write to the indexed table</li>
+ * <li>Write the index-containing WALEdit</li>
+ * <li>Start writing to the index tables in the postXXX hook</li>
+ * <li>WAL gets rolled and archived</li>
+ * <li>An index update fails, in which case we should kill ourselves to get WAL replay</li>
+ * <li>Since the WAL got archived, we won't get the replay of the index writes</li>
+ * </ol>
+ * <p>
+ * The usual course of events should be:
+ * <ol>
+ * <li>In a preXXX hook,
+ * <ol>
+ * <li>Build the {@link WALEdit} + index information</li>
+ * <li>Lock the {@link IndexLogRollSynchronizer#logArchiveLock}</li>
+ * <ul>
+ * <li>This is a reentrant readlock on the WAL archiving, so we can make multiple WAL/index updates
+ * concurrently</li>
+ * </ul>
+ * </li>
+ * </ol>
+ * </li>
+ * <li>Pass that {@link WALEdit} to the WAL, ensuring its durable and replayable</li>
+ * <li>In the corresponding postXXX,
+ * <ol>
+ * <li>make the updates to the index tables</li>
+ * <li>Unlock {@link IndexLogRollSynchronizer#logArchiveLock}</li>
+ * </ol>
+ * </li> </ol>
+ * <p>
+ * <tt>this</tt> should be added as a {@link WALActionsListener} by updating
+ */
+public class IndexLogRollSynchronizer implements WALActionsListener {
+
+  private static final Log LOG = LogFactory.getLog(IndexLogRollSynchronizer.class);
+  private WriteLock logArchiveLock;
+
+  public IndexLogRollSynchronizer(WriteLock logWriteLock){
+    this.logArchiveLock = logWriteLock;
+  }
+
+
+  @Override
+  public void preLogArchive(Path oldPath, Path newPath) throws IOException {
+    //take a write lock on the index - any pending index updates will complete before we finish
+    LOG.debug("Taking INDEX_UPDATE writelock");
+    logArchiveLock.lock();
+    LOG.debug("Got the INDEX_UPDATE writelock");
+  }
+  
+  @Override
+  public void postLogArchive(Path oldPath, Path newPath) throws IOException {
+    // done archiving the logs, any WAL updates will be replayed on failure
+    LOG.debug("Releasing INDEX_UPDATE writelock");
+    logArchiveLock.unlock();
+  }
+
+  @Override
+  public void logCloseRequested() {
+    // don't care- before this is called, all the HRegions are closed, so we can't get any new
+    // requests and all pending request can finish before the WAL closes.
+  }
+
+  @Override
+  public void preLogRoll(Path oldPath, Path newPath) throws IOException {
+    // noop
+  }
+
+  @Override
+  public void postLogRoll(Path oldPath, Path newPath) throws IOException {
+    // noop
+  }
+
+  @Override
+  public void logRollRequested() {
+    // noop
+  }
+
+  @Override
+  public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit) {
+    // noop
+  }
+
+  @Override
+  public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, WALEdit logEdit) {
+    // noop
+  }
+}
\ No newline at end of file


[03/15] Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestEndToEndCoveredColumnsIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestEndToEndCoveredColumnsIndexBuilder.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestEndToEndCoveredColumnsIndexBuilder.java
new file mode 100644
index 0000000..edcb8d5
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestEndToEndCoveredColumnsIndexBuilder.java
@@ -0,0 +1,344 @@
+/*
+ * 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.phoenix.hbase.index.covered;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdge;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.phoenix.hbase.index.IndexTestingUtils;
+import org.apache.phoenix.hbase.index.Indexer;
+import org.apache.phoenix.hbase.index.TableName;
+import org.apache.phoenix.hbase.index.covered.CoveredColumnsIndexBuilder;
+import org.apache.phoenix.hbase.index.covered.IndexCodec;
+import org.apache.phoenix.hbase.index.covered.IndexUpdate;
+import org.apache.phoenix.hbase.index.covered.LocalTableState;
+import org.apache.phoenix.hbase.index.covered.TableState;
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.hbase.index.scanner.Scanner;
+
+/**
+ * End-to-End test of just the {@link CoveredColumnsIndexBuilder}, but with a simple
+ * {@link IndexCodec} and BatchCache implementation.
+ */
+public class TestEndToEndCoveredColumnsIndexBuilder {
+
+  public class TestState {
+
+    private HTable table;
+    private long ts;
+    private VerifyingIndexCodec codec;
+
+    /**
+     * @param primary
+     * @param codec
+     * @param ts
+     */
+    public TestState(HTable primary, VerifyingIndexCodec codec, long ts) {
+      this.table = primary;
+      this.ts = ts;
+      this.codec = codec;
+    }
+
+  }
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static final byte[] row = Bytes.toBytes("row");
+  private static final byte[] family = Bytes.toBytes("FAM");
+  private static final byte[] qual = Bytes.toBytes("qual");
+  private static final HColumnDescriptor FAM1 = new HColumnDescriptor(family);
+
+  @Rule
+  public TableName TestTable = new TableName();
+
+  private TestState state;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    Configuration conf = UTIL.getConfiguration();
+    IndexTestingUtils.setupConfig(conf);
+    // disable version checking, so we can test against whatever version of HBase happens to be
+    // installed (right now, its generally going to be SNAPSHOT versions).
+    conf.setBoolean(Indexer.CHECK_VERSION_CONF_KEY, false);
+    UTIL.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void shutdownCluster() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setup() throws Exception {
+    this.state = setupTest(TestTable.getTableNameString());
+  }
+    
+  private interface TableStateVerifier {
+
+    /**
+     * Verify that the state of the table is correct. Should fail the unit test if it isn't as
+     * expected.
+     * @param state
+     */
+    public void verify(TableState state);
+
+  }
+
+  /**
+   * {@link TableStateVerifier} that ensures the kvs returned from the table match the passed
+   * {@link KeyValue}s when querying on the given columns.
+   */
+  private class ListMatchingVerifier implements TableStateVerifier {
+
+    private List<KeyValue> expectedKvs;
+    private ColumnReference[] columns;
+    private String msg;
+
+    public ListMatchingVerifier(String msg, List<KeyValue> kvs, ColumnReference... columns) {
+      this.expectedKvs = kvs;
+      this.columns = columns;
+      this.msg = msg;
+    }
+
+    @Override
+    public void verify(TableState state) {
+      try {
+        Scanner kvs =
+            ((LocalTableState) state).getIndexedColumnsTableState(Arrays.asList(columns)).getFirst();
+
+        int count = 0;
+        KeyValue kv;
+        while ((kv = kvs.next()) != null) {
+          KeyValue next = expectedKvs.get(count++);
+          assertEquals(
+            msg + ": Unexpected kv in table state!\nexpected v1: "
+                + Bytes.toString(next.getValue()) + "\nactual v1:" + Bytes.toString(kv.getValue()),
+            next, kv);
+        }
+
+        assertEquals(msg + ": Didn't find enough kvs in table state!", expectedKvs.size(), count);
+      } catch (IOException e) {
+        fail(msg + ": Got an exception while reading local table state! " + e.getMessage());
+      }
+    }
+  }
+
+  private class VerifyingIndexCodec extends CoveredIndexCodecForTesting {
+
+    private Queue<TableStateVerifier> verifiers = new ArrayDeque<TableStateVerifier>();
+
+    @Override
+    public Iterable<IndexUpdate> getIndexDeletes(TableState state) {
+      verify(state);
+      return super.getIndexDeletes(state);
+    }
+
+    @Override
+    public Iterable<IndexUpdate> getIndexUpserts(TableState state) {
+      verify(state);
+      return super.getIndexUpserts(state);
+    }
+
+    private void verify(TableState state) {
+      TableStateVerifier verifier = verifiers.poll();
+      if (verifier == null) return;
+      verifier.verify(state);
+    }
+  }
+  
+  /**
+   * Test that we see the expected values in a {@link TableState} when doing single puts against a
+   * region.
+   * @throws Exception on failure
+   */
+  @Test
+  public void testExpectedResultsInTableStateForSinglePut() throws Exception {
+    //just do a simple Put to start with
+    long ts = state.ts;
+    Put p = new Put(row, ts);
+    p.add(family, qual, Bytes.toBytes("v1"));
+    
+    // get all the underlying kvs for the put
+    final List<KeyValue> expectedKvs = new ArrayList<KeyValue>();
+    final List<KeyValue> allKvs = new ArrayList<KeyValue>();
+    allKvs.addAll(p.getFamilyMap().get(family));
+
+    // setup the verifier for the data we expect to write
+    // first call shouldn't have anything in the table
+    final ColumnReference familyRef =
+        new ColumnReference(TestEndToEndCoveredColumnsIndexBuilder.family, ColumnReference.ALL_QUALIFIERS);
+
+    VerifyingIndexCodec codec = state.codec;
+    codec.verifiers.add(new ListMatchingVerifier("cleanup state 1", expectedKvs, familyRef));
+    codec.verifiers.add(new ListMatchingVerifier("put state 1", allKvs, familyRef));
+
+    // do the actual put (no indexing will actually be done)
+    HTable primary = state.table;
+    primary.put(p);
+    primary.flushCommits();
+
+    // now we do another put to the same row. We should see just the old row state, followed by the
+    // new + old
+    p = new Put(row, ts + 1);
+    p.add(family, qual, Bytes.toBytes("v2"));
+    expectedKvs.addAll(allKvs);
+    // add them first b/c the ts is newer
+    allKvs.addAll(0, p.get(family, qual));
+    codec.verifiers.add(new ListMatchingVerifier("cleanup state 2", expectedKvs, familyRef));
+    codec.verifiers.add(new ListMatchingVerifier("put state 2", allKvs, familyRef));
+    
+    // do the actual put
+    primary.put(p);
+    primary.flushCommits();
+
+    // cleanup after ourselves
+    cleanup(state);
+  }
+
+  /**
+   * Similar to {@link #testExpectedResultsInTableStateForSinglePut()}, but against batches of puts.
+   * Previous implementations managed batches by playing current state against each element in the
+   * batch, rather than combining all the per-row updates into a single mutation for the batch. This
+   * test ensures that we see the correct expected state.
+   * @throws Exception on failure
+   */
+  @Test
+  public void testExpectedResultsInTableStateForBatchPuts() throws Exception {
+    long ts = state.ts;
+    // build up a list of puts to make, all on the same row
+    Put p1 = new Put(row, ts);
+    p1.add(family, qual, Bytes.toBytes("v1"));
+    Put p2 = new Put(row, ts + 1);
+    p2.add(family, qual, Bytes.toBytes("v2"));
+
+    // setup all the verifiers we need. This is just the same as above, but will be called twice
+    // since we need to iterate the batch.
+
+    // get all the underlying kvs for the put
+    final List<KeyValue> allKvs = new ArrayList<KeyValue>(2);
+    allKvs.addAll(p2.getFamilyMap().get(family));
+    allKvs.addAll(p1.getFamilyMap().get(family));
+
+    // setup the verifier for the data we expect to write
+    // both puts should be put into a single batch
+    final ColumnReference familyRef =
+        new ColumnReference(TestEndToEndCoveredColumnsIndexBuilder.family, ColumnReference.ALL_QUALIFIERS);
+    VerifyingIndexCodec codec = state.codec;
+    // no previous state in the table
+    codec.verifiers.add(new ListMatchingVerifier("cleanup state 1", Collections
+        .<KeyValue> emptyList(), familyRef));
+    codec.verifiers.add(new ListMatchingVerifier("put state 1", p1.getFamilyMap().get(family),
+        familyRef));
+
+    codec.verifiers.add(new ListMatchingVerifier("cleanup state 2", p1.getFamilyMap().get(family),
+        familyRef));
+    // kvs from both puts should be in the table now
+    codec.verifiers.add(new ListMatchingVerifier("put state 2", allKvs, familyRef));
+
+    // do the actual put (no indexing will actually be done)
+    HTable primary = state.table;
+    primary.setAutoFlush(false);
+    primary.put(Arrays.asList(p1, p2));
+    primary.flushCommits();
+
+    // cleanup after ourselves
+    cleanup(state);
+  }
+
+  /**
+   * @param tableName name of the table to create for the test
+   * @return the supporting state for the test
+   */
+  private TestState setupTest(String tableName) throws IOException {
+    byte[] tableNameBytes = Bytes.toBytes(tableName);
+    HTableDescriptor desc = new HTableDescriptor(tableNameBytes);
+    desc.addFamily(FAM1);
+    // add the necessary simple options to create the builder
+    Map<String, String> indexerOpts = new HashMap<String, String>();
+    // just need to set the codec - we are going to set it later, but we need something here or the
+    // initializer blows up.
+    indexerOpts.put(CoveredColumnsIndexBuilder.CODEC_CLASS_NAME_KEY,
+      CoveredIndexCodecForTesting.class.getName());
+    Indexer.enableIndexing(desc, CoveredColumnsIndexBuilder.class, indexerOpts);
+
+    // create the table
+    HBaseAdmin admin = UTIL.getHBaseAdmin();
+    admin.createTable(desc);
+    HTable primary = new HTable(UTIL.getConfiguration(), tableNameBytes);
+
+    // overwrite the codec so we can verify the current state
+    HRegion region = UTIL.getMiniHBaseCluster().getRegions(tableNameBytes).get(0);
+    Indexer indexer =
+        (Indexer) region.getCoprocessorHost().findCoprocessor(Indexer.class.getName());
+    CoveredColumnsIndexBuilder builder =
+        (CoveredColumnsIndexBuilder) indexer.getBuilderForTesting();
+    VerifyingIndexCodec codec = new VerifyingIndexCodec();
+    builder.setIndexCodecForTesting(codec);
+
+    // setup the Puts we want to write
+    final long ts = System.currentTimeMillis();
+    EnvironmentEdge edge = new EnvironmentEdge() {
+
+      @Override
+      public long currentTimeMillis() {
+        return ts;
+      }
+    };
+    EnvironmentEdgeManager.injectEdge(edge);
+
+    return new TestState(primary, codec, ts);
+  }
+
+  /**
+   * Cleanup the test based on the passed state.
+   * @param state
+   */
+  private void cleanup(TestState state) throws IOException {
+    EnvironmentEdgeManager.reset();
+    state.table.close();
+    UTIL.deleteTable(state.table.getTableName());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestLocalTableState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestLocalTableState.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestLocalTableState.java
new file mode 100644
index 0000000..95607ca
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestLocalTableState.java
@@ -0,0 +1,198 @@
+/*
+ * 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.phoenix.hbase.index.covered;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import org.apache.phoenix.hbase.index.covered.IndexUpdate;
+import org.apache.phoenix.hbase.index.covered.LocalTableState;
+import org.apache.phoenix.hbase.index.covered.data.LocalHBaseState;
+import org.apache.phoenix.hbase.index.covered.data.LocalTable;
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.hbase.index.scanner.Scanner;
+
+/**
+ *
+ */
+public class TestLocalTableState {
+
+  private static final byte[] row = Bytes.toBytes("row");
+  private static final byte[] fam = Bytes.toBytes("fam");
+  private static final byte[] qual = Bytes.toBytes("qual");
+  private static final byte[] val = Bytes.toBytes("val");
+  private static final long ts = 10;
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testCorrectOrderingWithLazyLoadingColumns() throws Exception {
+    Put m = new Put(row);
+    m.add(fam, qual, ts, val);
+    // setup mocks
+    Configuration conf = new Configuration(false);
+    RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
+    Mockito.when(env.getConfiguration()).thenReturn(conf);
+
+    HRegion region = Mockito.mock(HRegion.class);
+    Mockito.when(env.getRegion()).thenReturn(region);
+    RegionScanner scanner = Mockito.mock(RegionScanner.class);
+    Mockito.when(region.getScanner(Mockito.any(Scan.class))).thenReturn(scanner);
+    final byte[] stored = Bytes.toBytes("stored-value");
+    Mockito.when(scanner.next(Mockito.any(List.class))).thenAnswer(new Answer<Boolean>() {
+      @Override
+      public Boolean answer(InvocationOnMock invocation) throws Throwable {
+        List<KeyValue> list = (List<KeyValue>) invocation.getArguments()[0];
+        KeyValue kv = new KeyValue(row, fam, qual, ts, Type.Put, stored);
+        kv.setMemstoreTS(0);
+        list.add(kv);
+        return false;
+      }
+    });
+
+
+    LocalHBaseState state = new LocalTable(env);
+    LocalTableState table = new LocalTableState(env, state, m);
+    //add the kvs from the mutation
+    table.addPendingUpdates(m.get(fam, qual));
+
+    // setup the lookup
+    ColumnReference col = new ColumnReference(fam, qual);
+    table.setCurrentTimestamp(ts);
+    //check that our value still shows up first on scan, even though this is a lazy load
+    Pair<Scanner, IndexUpdate> p = table.getIndexedColumnsTableState(Arrays.asList(col));
+    Scanner s = p.getFirst();
+    assertEquals("Didn't get the pending mutation's value first", m.get(fam, qual).get(0), s.next());
+  }
+
+  /**
+   * Test that we correctly rollback the state of keyvalue
+   * @throws Exception
+   */
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testCorrectRollback() throws Exception {
+    Put m = new Put(row);
+    m.add(fam, qual, ts, val);
+    // setup mocks
+    RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
+
+    HRegion region = Mockito.mock(HRegion.class);
+    Mockito.when(env.getRegion()).thenReturn(region);
+    RegionScanner scanner = Mockito.mock(RegionScanner.class);
+    Mockito.when(region.getScanner(Mockito.any(Scan.class))).thenReturn(scanner);
+    final byte[] stored = Bytes.toBytes("stored-value");
+    final KeyValue storedKv = new KeyValue(row, fam, qual, ts, Type.Put, stored);
+    storedKv.setMemstoreTS(2);
+    Mockito.when(scanner.next(Mockito.any(List.class))).thenAnswer(new Answer<Boolean>() {
+      @Override
+      public Boolean answer(InvocationOnMock invocation) throws Throwable {
+        List<KeyValue> list = (List<KeyValue>) invocation.getArguments()[0];
+
+        list.add(storedKv);
+        return false;
+      }
+    });
+    LocalHBaseState state = new LocalTable(env);
+    LocalTableState table = new LocalTableState(env, state, m);
+    // add the kvs from the mutation
+    KeyValue kv = m.get(fam, qual).get(0);
+    kv.setMemstoreTS(0);
+    table.addPendingUpdates(kv);
+
+    // setup the lookup
+    ColumnReference col = new ColumnReference(fam, qual);
+    table.setCurrentTimestamp(ts);
+    // check that the value is there
+    Pair<Scanner, IndexUpdate> p = table.getIndexedColumnsTableState(Arrays.asList(col));
+    Scanner s = p.getFirst();
+    assertEquals("Didn't get the pending mutation's value first", kv, s.next());
+
+    // rollback that value
+    table.rollback(Arrays.asList(kv));
+    p = table.getIndexedColumnsTableState(Arrays.asList(col));
+    s = p.getFirst();
+    assertEquals("Didn't correctly rollback the row - still found it!", null, s.next());
+    Mockito.verify(env, Mockito.times(1)).getRegion();
+    Mockito.verify(region, Mockito.times(1)).getScanner(Mockito.any(Scan.class));
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testOnlyLoadsRequestedColumns() throws Exception {
+    // setup mocks
+    RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
+
+    HRegion region = Mockito.mock(HRegion.class);
+    Mockito.when(env.getRegion()).thenReturn(region);
+    RegionScanner scanner = Mockito.mock(RegionScanner.class);
+    Mockito.when(region.getScanner(Mockito.any(Scan.class))).thenReturn(scanner);
+    final KeyValue storedKv =
+        new KeyValue(row, fam, qual, ts, Type.Put, Bytes.toBytes("stored-value"));
+    storedKv.setMemstoreTS(2);
+    Mockito.when(scanner.next(Mockito.any(List.class))).thenAnswer(new Answer<Boolean>() {
+      @Override
+      public Boolean answer(InvocationOnMock invocation) throws Throwable {
+        List<KeyValue> list = (List<KeyValue>) invocation.getArguments()[0];
+
+        list.add(storedKv);
+        return false;
+      }
+    });
+    LocalHBaseState state = new LocalTable(env);
+    Put pendingUpdate = new Put(row);
+    pendingUpdate.add(fam, qual, ts, val);
+    LocalTableState table = new LocalTableState(env, state, pendingUpdate);
+
+    // do the lookup for the given column
+    ColumnReference col = new ColumnReference(fam, qual);
+    table.setCurrentTimestamp(ts);
+    // check that the value is there
+    Pair<Scanner, IndexUpdate> p = table.getIndexedColumnsTableState(Arrays.asList(col));
+    Scanner s = p.getFirst();
+    // make sure it read the table the one time
+    assertEquals("Didn't get the stored keyvalue!", storedKv, s.next());
+
+    // on the second lookup it shouldn't access the underlying table again - the cached columns
+    // should know they are done
+    p = table.getIndexedColumnsTableState(Arrays.asList(col));
+    s = p.getFirst();
+    assertEquals("Lost already loaded update!", storedKv, s.next());
+    Mockito.verify(env, Mockito.times(1)).getRegion();
+    Mockito.verify(region, Mockito.times(1)).getScanner(Mockito.any(Scan.class));
+  }
+
+  // TODO add test here for making sure multiple column references with the same column family don't
+  // cause an infinite loop
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/data/TestIndexMemStore.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/data/TestIndexMemStore.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/data/TestIndexMemStore.java
new file mode 100644
index 0000000..87fed3b
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/data/TestIndexMemStore.java
@@ -0,0 +1,94 @@
+/*
+ * 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.phoenix.hbase.index.covered.data;
+
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.hbase.index.covered.data.IndexMemStore;
+import org.junit.Test;
+
+public class TestIndexMemStore {
+
+  private static final byte[] row = Bytes.toBytes("row");
+  private static final byte[] family = Bytes.toBytes("family");
+  private static final byte[] qual = Bytes.toBytes("qual");
+  private static final byte[] val = Bytes.toBytes("val");
+  private static final byte[] val2 = Bytes.toBytes("val2");
+
+  @Test
+  public void testCorrectOverwritting() throws Exception {
+    IndexMemStore store = new IndexMemStore(IndexMemStore.COMPARATOR);
+    long ts = 10;
+    KeyValue kv = new KeyValue(row, family, qual, ts, Type.Put, val);
+    kv.setMemstoreTS(2);
+    KeyValue kv2 = new KeyValue(row, family, qual, ts, Type.Put, val2);
+    kv2.setMemstoreTS(0);
+    store.add(kv, true);
+    // adding the exact same kv shouldn't change anything stored if not overwritting
+    store.add(kv2, false);
+    KeyValueScanner scanner = store.getScanner();
+    KeyValue first = KeyValue.createFirstOnRow(row);
+    scanner.seek(first);
+    assertTrue("Overwrote kv when specifically not!", kv == scanner.next());
+    scanner.close();
+
+    // now when we overwrite, we should get the newer one
+    store.add(kv2, true);
+    scanner = store.getScanner();
+    scanner.seek(first);
+    assertTrue("Didn't overwrite kv when specifically requested!", kv2 == scanner.next());
+    scanner.close();
+  }
+
+  /**
+   * We don't expect custom KeyValue creation, so we can't get into weird situations, where a
+   * {@link Type#DeleteFamily} has a column qualifier specified.
+   * @throws Exception
+   */
+  @Test
+  public void testExpectedOrdering() throws Exception {
+    IndexMemStore store = new IndexMemStore();
+    KeyValue kv = new KeyValue(row, family, qual, 12, Type.Put, val);
+    store.add(kv, true);
+    KeyValue kv2 = new KeyValue(row, family, qual, 10, Type.Put, val2);
+    store.add(kv2, true);
+    KeyValue df = new KeyValue(row, family, null, 11, Type.DeleteFamily, null);
+    store.add(df, true);
+    KeyValue dc = new KeyValue(row, family, qual, 11, Type.DeleteColumn, null);
+    store.add(dc, true);
+    KeyValue d = new KeyValue(row, family, qual, 12, Type.Delete, null);
+    store.add(d, true);
+
+    // null qualifiers should always sort before the non-null cases
+    KeyValueScanner scanner = store.getScanner();
+    KeyValue first = KeyValue.createFirstOnRow(row);
+    assertTrue("Didn't have any data in the scanner", scanner.seek(first));
+    assertTrue("Didn't get delete family first (no qualifier == sort first)", df == scanner.next());
+    assertTrue("Didn't get point delete before corresponding put", d == scanner.next());
+    assertTrue("Didn't get larger ts Put", kv == scanner.next());
+    assertTrue("Didn't get delete column before corresponding put(delete sorts first)",
+      dc == scanner.next());
+    assertTrue("Didn't get smaller ts Put", kv2 == scanner.next());
+    assertNull("Have more data in the scanner", scanner.next());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestColumnTracker.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestColumnTracker.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestColumnTracker.java
new file mode 100644
index 0000000..02555e9
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestColumnTracker.java
@@ -0,0 +1,61 @@
+/*
+ * 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.phoenix.hbase.index.covered.example;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.junit.Test;
+
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.hbase.index.covered.update.ColumnTracker;
+
+public class TestColumnTracker {
+
+  @Test
+  public void testEnsureGuarranteedMinValid() {
+    assertFalse("Guarranted min wasn't recognized as having newer timestamps!",
+      ColumnTracker.isNewestTime(ColumnTracker.GUARANTEED_NEWER_UPDATES));
+  }
+
+  @Test
+  public void testOnlyKeepsOlderTimestamps() {
+    Collection<ColumnReference> columns = new ArrayList<ColumnReference>();
+    ColumnTracker tracker = new ColumnTracker(columns);
+    tracker.setTs(10);
+    assertEquals("Column tracker didn't set original TS", 10, tracker.getTS());
+    tracker.setTs(12);
+    assertEquals("Column tracker allowed newer timestamp to be set.", 10, tracker.getTS());
+    tracker.setTs(9);
+    assertEquals("Column tracker didn't decrease set timestamp for smaller value", 9,
+      tracker.getTS());
+  }
+
+  @Test
+  public void testHasNewerTimestamps() throws Exception {
+    Collection<ColumnReference> columns = new ArrayList<ColumnReference>();
+    ColumnTracker tracker = new ColumnTracker(columns);
+    assertFalse("Tracker has newer timestamps when no ts set", tracker.hasNewerTimestamps());
+    tracker.setTs(10);
+    assertTrue("Tracker doesn't have newer timetamps with set ts", tracker.hasNewerTimestamps());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestCoveredColumnIndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestCoveredColumnIndexCodec.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestCoveredColumnIndexCodec.java
new file mode 100644
index 0000000..8bf04a3
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestCoveredColumnIndexCodec.java
@@ -0,0 +1,251 @@
+/*
+ * 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.phoenix.hbase.index.covered.example;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.hbase.index.covered.IndexCodec;
+import org.apache.phoenix.hbase.index.covered.IndexUpdate;
+import org.apache.phoenix.hbase.index.covered.LocalTableState;
+import org.apache.phoenix.hbase.index.covered.data.LocalHBaseState;
+import org.apache.phoenix.hbase.index.covered.example.ColumnGroup;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumn;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumnIndexCodec;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumnIndexCodec.ColumnEntry;
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+
+public class TestCoveredColumnIndexCodec {
+  private static final byte[] PK = new byte[] { 'a' };
+  private static final String FAMILY_STRING = "family";
+  private static final byte[] FAMILY = Bytes.toBytes(FAMILY_STRING);
+  private static final byte[] QUAL = Bytes.toBytes("qual");
+  private static final CoveredColumn COLUMN_REF = new CoveredColumn(FAMILY_STRING, QUAL);
+  private static final byte[] EMPTY_INDEX_KEY = CoveredColumnIndexCodec.composeRowKey(PK, 0,
+    Arrays.asList(toColumnEntry(new byte[0])));
+  private static final byte[] BLANK_INDEX_KEY = CoveredColumnIndexCodec.composeRowKey(PK, 0,
+    Collections.<ColumnEntry> emptyList());
+
+  private static ColumnEntry toColumnEntry(byte[] bytes) {
+    return new ColumnEntry(bytes, COLUMN_REF);
+  }
+
+  /**
+   * Convert between an index and a bunch of values
+   * @throws Exception
+   */
+  @Test
+  public void toFromIndexKey() throws Exception {
+    // start with empty values
+    byte[] indexKey = BLANK_INDEX_KEY;
+    List<byte[]> stored = CoveredColumnIndexCodec.getValues(indexKey);
+    assertEquals("Found some stored values in an index row key that wasn't created with values!",
+      0, stored.size());
+
+    // a single, empty value
+    indexKey = EMPTY_INDEX_KEY;
+    stored = CoveredColumnIndexCodec.getValues(indexKey);
+    assertEquals("Found some stored values in an index row key that wasn't created with values!",
+      1, stored.size());
+    assertEquals("Found a non-zero length value: " + Bytes.toString(stored.get(0)), 0,
+      stored.get(0).length);
+
+    // try with a couple values, some different lengths
+    byte[] v1 = new byte[] { 'a' };
+    byte[] v2 = new byte[] { 'b' };
+    byte[] v3 = Bytes.toBytes("v3");
+    int len = v1.length + v2.length + v3.length;
+    indexKey =
+        CoveredColumnIndexCodec.composeRowKey(PK, len,
+          Arrays.asList(toColumnEntry(v1), toColumnEntry(v2), toColumnEntry(v3)));
+    stored = CoveredColumnIndexCodec.getValues(indexKey);
+    assertEquals("Didn't find expected number of values in index key!", 3, stored.size());
+    assertTrue("First index keys don't match!", Bytes.equals(v1, stored.get(0)));
+    assertTrue("Second index keys don't match!", Bytes.equals(v2, stored.get(1)));
+    assertTrue("Third index keys don't match!", Bytes.equals(v3, stored.get(2)));
+  }
+
+  /**
+   * Ensure that we correctly can determine when a row key is empty (no values).
+   */
+  @Test
+  public void testCheckRowKeyForAllNulls() {
+    byte[] pk = new byte[] { 'a', 'b', 'z' };
+    // check positive cases first
+    byte[] result = EMPTY_INDEX_KEY;
+    assertTrue("Didn't correctly read single element as being null in row key",
+      CoveredColumnIndexCodec.checkRowKeyForAllNulls(result));
+    result =
+        CoveredColumnIndexCodec.composeRowKey(pk, 0,
+          Lists.newArrayList(toColumnEntry(new byte[0]), toColumnEntry(new byte[0])));
+    assertTrue("Didn't correctly read two elements as being null in row key",
+      CoveredColumnIndexCodec.checkRowKeyForAllNulls(result));
+
+    // check cases where it isn't null
+    result =
+        CoveredColumnIndexCodec.composeRowKey(pk, 2,
+          Arrays.asList(toColumnEntry(new byte[] { 1, 2 })));
+    assertFalse("Found a null key, when it wasn't!",
+      CoveredColumnIndexCodec.checkRowKeyForAllNulls(result));
+    result =
+        CoveredColumnIndexCodec.composeRowKey(pk, 2,
+          Arrays.asList(toColumnEntry(new byte[] { 1, 2 }), toColumnEntry(new byte[0])));
+    assertFalse("Found a null key, when it wasn't!",
+      CoveredColumnIndexCodec.checkRowKeyForAllNulls(result));
+  }
+
+  private static class SimpleTableState implements LocalHBaseState {
+
+    private Result r;
+
+    public SimpleTableState(Result r) {
+      this.r = r;
+    }
+
+    @Override
+    public Result getCurrentRowState(Mutation m, Collection<? extends ColumnReference> toCover)
+        throws IOException {
+      return r;
+    }
+
+  }
+
+  /**
+   * Test that we get back the correct index updates for a given column group
+   * @throws Exception on failure
+   */
+  @Test
+  public void testGeneratedIndexUpdates() throws Exception {
+    ColumnGroup group = new ColumnGroup("test-column-group");
+    group.add(COLUMN_REF);
+
+    final Result emptyState = new Result(Collections.<KeyValue> emptyList());
+    
+    // setup the state we expect for the codec
+    RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
+    Configuration conf = new Configuration(false);
+    Mockito.when(env.getConfiguration()).thenReturn(conf);
+    LocalHBaseState table = new SimpleTableState(emptyState);
+
+    // make a new codec on those kvs
+    CoveredColumnIndexCodec codec =
+        CoveredColumnIndexCodec.getCodecForTesting(Arrays.asList(group));
+
+    // start with a basic put that has some keyvalues
+    Put p = new Put(PK);
+    // setup the kvs to add
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    byte[] v1 = Bytes.toBytes("v1");
+    KeyValue kv = new KeyValue(PK, FAMILY, QUAL, 1, v1);
+    kvs.add(kv);
+    p.add(kv);
+    byte[] v2 = Bytes.toBytes("v2");
+    kv = new KeyValue(PK, Bytes.toBytes("family2"), QUAL, 1, v2);
+    kvs.add(kv);
+    p.add(kv);
+
+    // check the codec for deletes it should send
+    LocalTableState state = new LocalTableState(env, table, p);
+    Iterable<IndexUpdate> updates = codec.getIndexDeletes(state);
+    assertFalse("Found index updates without any existing kvs in table!", updates.iterator().next()
+        .isValid());
+
+    // get the updates with the pending update
+    state.setCurrentTimestamp(1);
+    state.addPendingUpdates(kvs);
+    updates = codec.getIndexUpserts(state);
+    assertTrue("Didn't find index updates for pending primary table update!", updates.iterator()
+        .hasNext());
+    for (IndexUpdate update : updates) {
+      assertTrue("Update marked as invalid, but should be a pending index write!", update.isValid());
+      Put m = (Put) update.getUpdate();
+      // should just be the single update for the column reference
+      byte[] expected =
+          CoveredColumnIndexCodec.composeRowKey(PK, v1.length, Arrays.asList(toColumnEntry(v1)));
+      assertArrayEquals("Didn't get expected index value", expected, m.getRow());
+    }
+
+    // then apply a delete
+    Delete d = new Delete(PK, 2);
+    // need to set the timestamp here, as would actually happen on the server, unlike what happens
+    // with puts, where the get the constructor specified timestamp for unspecified methods.
+    d.deleteFamily(FAMILY, 2);
+    // setup the next batch of 'current state', basically just ripping out the current state from
+    // the last round
+    table = new SimpleTableState(new Result(kvs));
+    state = new LocalTableState(env, table, d);
+    state.setCurrentTimestamp(2);
+    // check the cleanup of the current table, after the puts (mocking a 'next' update)
+    updates = codec.getIndexDeletes(state);
+    for (IndexUpdate update : updates) {
+      assertTrue("Didn't have any index cleanup, even though there is current state",
+        update.isValid());
+      Delete m = (Delete) update.getUpdate();
+      // should just be the single update for the column reference
+      byte[] expected =
+          CoveredColumnIndexCodec.composeRowKey(PK, v1.length, Arrays.asList(toColumnEntry(v1)));
+      assertArrayEquals("Didn't get expected index value", expected, m.getRow());
+    }
+    ensureNoUpdatesWhenCoveredByDelete(env, codec, kvs, d);
+
+    // now with the delete of the columns
+    d = new Delete(PK, 2);
+    d.deleteColumns(FAMILY, QUAL, 2);
+    ensureNoUpdatesWhenCoveredByDelete(env, codec, kvs, d);
+
+    // this delete needs to match timestamps exactly, by contract, to have any effect
+    d = new Delete(PK, 1);
+    d.deleteColumn(FAMILY, QUAL, 1);
+    ensureNoUpdatesWhenCoveredByDelete(env, codec, kvs, d);
+  }
+
+  private void ensureNoUpdatesWhenCoveredByDelete(RegionCoprocessorEnvironment env, IndexCodec codec, List<KeyValue> currentState,
+      Delete d) throws IOException {
+    LocalHBaseState table = new SimpleTableState(new Result(currentState));
+    LocalTableState state = new LocalTableState(env, table, d);
+    state.setCurrentTimestamp(d.getTimeStamp());
+    // now we shouldn't see anything when getting the index update
+    state.addPendingUpdates(d.getFamilyMap().get(FAMILY));
+    Iterable<IndexUpdate> updates = codec.getIndexUpserts(state);
+    for (IndexUpdate update : updates) {
+      assertFalse("Had some index updates, though it should have been covered by the delete",
+        update.isValid());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestCoveredIndexSpecifierBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestCoveredIndexSpecifierBuilder.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestCoveredIndexSpecifierBuilder.java
new file mode 100644
index 0000000..dbed613
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestCoveredIndexSpecifierBuilder.java
@@ -0,0 +1,75 @@
+/*
+ * 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.phoenix.hbase.index.covered.example;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.hbase.index.covered.example.ColumnGroup;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumn;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumnIndexSpecifierBuilder;
+import org.junit.Test;
+
+public class TestCoveredIndexSpecifierBuilder {
+  private static final String FAMILY = "FAMILY";
+  private static final String FAMILY2 = "FAMILY2";
+  private static final String INDEX_TABLE = "INDEX_TABLE";
+  private static final String INDEX_TABLE2 = "INDEX_TABLE2";
+
+
+  @Test
+  public void testSimpleSerialziationDeserialization() throws Exception {
+    byte[] indexed_qualifer = Bytes.toBytes("indexed_qual");
+
+    //setup the index 
+    CoveredColumnIndexSpecifierBuilder builder = new CoveredColumnIndexSpecifierBuilder();
+    ColumnGroup fam1 = new ColumnGroup(INDEX_TABLE);
+    // match a single family:qualifier pair
+    CoveredColumn col1 = new CoveredColumn(FAMILY, indexed_qualifer);
+    fam1.add(col1);
+    // matches the family2:* columns
+    CoveredColumn col2 = new CoveredColumn(FAMILY2, null);
+    fam1.add(col2);
+    builder.addIndexGroup(fam1);
+    ColumnGroup fam2 = new ColumnGroup(INDEX_TABLE2);
+    // match a single family2:qualifier pair
+    CoveredColumn col3 = new CoveredColumn(FAMILY2, indexed_qualifer);
+    fam2.add(col3);
+    builder.addIndexGroup(fam2);
+    
+    Configuration conf = new Configuration(false);
+    //convert the map that HTableDescriptor gets into the conf the coprocessor receives
+    Map<String, String> map = builder.convertToMap();
+    for(Entry<String, String> entry: map.entrySet()){
+      conf.set(entry.getKey(), entry.getValue());
+    }
+
+    List<ColumnGroup> columns = CoveredColumnIndexSpecifierBuilder.getColumns(conf);
+    assertEquals("Didn't deserialize the expected number of column groups", 2, columns.size());
+    ColumnGroup group = columns.get(0);
+    assertEquals("Didn't deserialize expected column in first group", col1, group.getColumnForTesting(0));
+    assertEquals("Didn't deserialize expected column in first group", col2, group.getColumnForTesting(1));
+    group = columns.get(1);
+    assertEquals("Didn't deserialize expected column in second group", col3, group.getColumnForTesting(0));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestEndToEndCoveredIndexing.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestEndToEndCoveredIndexing.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestEndToEndCoveredIndexing.java
new file mode 100644
index 0000000..5ac1ce9
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestEndToEndCoveredIndexing.java
@@ -0,0 +1,882 @@
+/*
+ * 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.phoenix.hbase.index.covered.example;
+
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.phoenix.hbase.index.IndexTestingUtils;
+import org.apache.phoenix.hbase.index.Indexer;
+import org.apache.phoenix.hbase.index.TableName;
+import org.apache.phoenix.hbase.index.covered.example.ColumnGroup;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumn;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumnIndexCodec;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumnIndexSpecifierBuilder;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumnIndexer;
+
+/**
+ * Test Covered Column indexing in an 'end-to-end' manner on a minicluster. This covers cases where
+ * we manage custom timestamped updates that arrive in and out of order as well as just using the
+ * generically timestamped updates.
+ */
+public class TestEndToEndCoveredIndexing {
+  private static final Log LOG = LogFactory.getLog(TestEndToEndCoveredIndexing.class);
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static final String FAM_STRING = "FAMILY";
+  private static final byte[] FAM = Bytes.toBytes(FAM_STRING);
+  private static final String FAM2_STRING = "FAMILY2";
+  private static final byte[] FAM2 = Bytes.toBytes(FAM2_STRING);
+  private static final byte[] EMPTY_BYTES = new byte[0];
+  private static final byte[] indexed_qualifer = Bytes.toBytes("indexed_qual");
+  private static final byte[] regular_qualifer = Bytes.toBytes("reg_qual");
+  private static final byte[] row1 = Bytes.toBytes("row1");
+  private static final byte[] value1 = Bytes.toBytes("val1");
+  private static final byte[] value2 = Bytes.toBytes("val2");
+  private static final byte[] value3 = Bytes.toBytes("val3");
+  // match a single family:qualifier pair
+  private static final CoveredColumn col1 = new CoveredColumn(FAM_STRING, indexed_qualifer);
+  // matches the family2:* columns
+  private static final CoveredColumn col2 = new CoveredColumn(FAM2_STRING, null);
+  private static final CoveredColumn col3 = new CoveredColumn(FAM2_STRING, indexed_qualifer);
+  
+  @Rule
+  public TableName TestTable = new TableName();
+  
+  private ColumnGroup fam1;
+  private ColumnGroup fam2;
+
+  // setup a couple of index columns
+  private void setupColumns() {
+    fam1 = new ColumnGroup(getIndexTableName());
+    fam2 = new ColumnGroup(getIndexTableName() + "2");
+    // values are [col1][col2_1]...[col2_n]
+    fam1.add(col1);
+    fam1.add(col2);
+    // value is [col2]
+    fam2.add(col3);
+  }
+
+  private String getIndexTableName() {
+    return Bytes.toString(TestTable.getTableName()) + "_index";
+  }
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    Configuration conf = UTIL.getConfiguration();
+    IndexTestingUtils.setupConfig(conf);
+    // disable version checking, so we can test against whatever version of HBase happens to be
+    // installed (right now, its generally going to be SNAPSHOT versions).
+    conf.setBoolean(Indexer.CHECK_VERSION_CONF_KEY, false);
+    UTIL.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setup() throws Exception {
+    setupColumns();
+  }
+
+  /**
+   * Test that a bunch of puts with a single timestamp across all the puts builds and inserts index
+   * entries as expected
+   * @throws Exception on failure
+   */
+  @Test
+  public void testSimpleTimestampedUpdates() throws Exception {
+    HTable primary = createSetupTables(fam1);
+
+    // do a put to the primary table
+    Put p = new Put(row1);
+    long ts = 10;
+    p.add(FAM, indexed_qualifer, ts, value1);
+    p.add(FAM, regular_qualifer, ts, value2);
+    primary.put(p);
+    primary.flushCommits();
+
+    // read the index for the expected values
+    HTable index1 = new HTable(UTIL.getConfiguration(), getIndexTableName());
+
+    // build the expected kvs
+    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
+    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
+    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts, pairs);
+
+    // verify that the index matches
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value1);
+
+    // cleanup
+    closeAndCleanupTables(primary, index1);
+  }
+
+  /**
+   * Test that the multiple timestamps in a single put build the correct index updates.
+   * @throws Exception on failure
+   */
+  @Test
+  public void testMultipleTimestampsInSinglePut() throws Exception {
+    HTable primary = createSetupTables(fam1);
+
+    // do a put to the primary table
+    Put p = new Put(row1);
+    long ts1 = 10;
+    long ts2 = 11;
+    p.add(FAM, indexed_qualifer, ts1, value1);
+    p.add(FAM, regular_qualifer, ts1, value2);
+    // our group indexes all columns in the this family, so any qualifier here is ok
+    p.add(FAM2, regular_qualifer, ts2, value3);
+    primary.put(p);
+    primary.flushCommits();
+
+    // read the index for the expected values
+    HTable index1 = new HTable(UTIL.getConfiguration(), getIndexTableName());
+
+    // build the expected kvs
+    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
+    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
+
+    // check the first entry at ts1
+    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts1, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts1, value1);
+
+    // check the second entry at ts2
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(value3, col2));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts2, value1);
+
+    // cleanup
+    closeAndCleanupTables(primary, index1);
+  }
+
+  /**
+   * Test that we make updates to multiple {@link ColumnGroup}s across a single put/delete 
+   * @throws Exception on failure
+   */
+  @Test
+  public void testMultipleConcurrentGroupsUpdated() throws Exception {
+    HTable primary = createSetupTables(fam1, fam2);
+
+    // do a put to the primary table
+    Put p = new Put(row1);
+    long ts = 10;
+    p.add(FAM, indexed_qualifer, ts, value1);
+    p.add(FAM, regular_qualifer, ts, value2);
+    p.add(FAM2, indexed_qualifer, ts, value3);
+    primary.put(p);
+    primary.flushCommits();
+
+    // read the index for the expected values
+    HTable index1 = new HTable(UTIL.getConfiguration(), fam1.getTable());
+    HTable index2 = new HTable(UTIL.getConfiguration(), fam2.getTable());
+
+    // build the expected kvs
+    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
+    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(value3, col2));
+    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value1);
+
+    // and check the second index as well
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value3, col3));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index2, expected, ts, value3);
+
+    // cleanup
+    closeAndCleanupTables(primary, index1, index2);
+  }
+
+  /**
+   * HBase has a 'fun' property wherein you can completely clobber an existing row if you make a
+   * {@link Put} at the exact same dimension (row, cf, cq, ts) as an existing row. The old row
+   * disappears and the new value (since the rest of the row is the same) completely subsumes it.
+   * This test ensures that we remove the old entry and put a new entry in its place.
+   * @throws Exception on failure
+   */
+  @Test
+  public void testOverwritingPutsCorrectlyGetIndexed() throws Exception {
+    HTable primary = createSetupTables(fam1);
+
+    // do a put to the primary table
+    Put p = new Put(row1);
+    long ts = 10;
+    p.add(FAM, indexed_qualifer, ts, value1);
+    p.add(FAM, regular_qualifer, ts, value2);
+    primary.put(p);
+    primary.flushCommits();
+
+    // read the index for the expected values
+    HTable index1 = new HTable(UTIL.getConfiguration(), fam1.getTable());
+
+    // build the expected kvs
+    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
+    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
+
+    // check the first entry at ts
+    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value1);
+
+    // now overwrite the put in the primary table with a new value
+    p = new Put(row1);
+    p.add(FAM, indexed_qualifer, ts, value3);
+    primary.put(p);
+    primary.flushCommits();
+
+    pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
+    pairs.add(new Pair<byte[], CoveredColumn>(value3, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
+
+    // check the first entry at ts
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value3);
+    // and verify that a scan at the first entry returns nothing (ignore the updated row)
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, Collections.<KeyValue> emptyList(), ts,
+      value1, value2);
+
+    // cleanup
+    closeAndCleanupTables(primary, index1);
+  }
+  
+  @Test
+  public void testSimpleDeletes() throws Exception {
+    HTable primary = createSetupTables(fam1);
+
+    // do a simple Put
+    long ts = 10;
+    Put p = new Put(row1);
+    p.add(FAM, indexed_qualifer, ts, value1);
+    p.add(FAM, regular_qualifer, ts, value2);
+    primary.put(p);
+    primary.flushCommits();
+
+    Delete d = new Delete(row1);
+    primary.delete(d);
+
+    HTable index = new HTable(UTIL.getConfiguration(), fam1.getTable());
+    List<KeyValue> expected = Collections.<KeyValue> emptyList();
+    // scan over all time should cause the delete to be covered
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index, expected, 0, Long.MAX_VALUE, value1,
+      HConstants.EMPTY_END_ROW);
+
+    // scan at the older timestamp should still show the older value
+    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
+    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index, expected, ts, value1);
+
+    // cleanup
+    closeAndCleanupTables(index, primary);
+  }
+
+  /**
+   * If we don't have any updates to make to the index, we don't take a lock on the WAL. However, we
+   * need to make sure that we don't try to unlock the WAL on write time when we don't write
+   * anything, since that will cause an java.lang.IllegalMonitorStateException
+   * @throws Exception on failure
+   */
+  @Test
+  public void testDeletesWithoutPreviousState() throws Exception {
+    HTable primary = createSetupTables(fam1);
+
+    // do a delete on the primary table (no data, so no index updates...hopefully).
+    long ts = 10;
+    Delete d = new Delete(row1);
+    primary.delete(d);
+
+    HTable index1 = new HTable(UTIL.getConfiguration(), fam1.getTable());
+    List<KeyValue> expected = Collections.<KeyValue> emptyList();
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value1);
+
+    // a delete of a specific family/column should also not show any index updates
+    d = new Delete(row1);
+    d.deleteColumn(FAM, indexed_qualifer);
+    primary.delete(d);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value1);
+
+    // also just a family marker should have the same effect
+    d = new Delete(row1);
+    d.deleteFamily(FAM);
+    primary.delete(d);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value1);
+
+    // also just a family marker should have the same effect
+    d = new Delete(row1);
+    d.deleteColumns(FAM, indexed_qualifer);
+    primary.delete(d);
+    primary.flushCommits();
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, value1);
+
+    // cleanup
+    closeAndCleanupTables(primary, index1);
+  }
+
+  /**
+   * Similar to the {@link #testMultipleTimestampsInSinglePut()}, this check the same with deletes
+   * @throws Exception on failure
+   */
+  @Test
+  public void testMultipleTimestampsInSingleDelete() throws Exception {
+    HTable primary = createSetupTables(fam1);
+
+    // do a put to the primary table
+    Put p = new Put(row1);
+    long ts1 = 10, ts2 = 11, ts3 = 12;
+    p.add(FAM, indexed_qualifer, ts1, value1);
+    // our group indexes all columns in the this family, so any qualifier here is ok
+    p.add(FAM2, regular_qualifer, ts2, value3);
+    primary.put(p);
+    primary.flushCommits();
+
+    // check to make sure everything we expect is there
+    HTable index1 = new HTable(UTIL.getConfiguration(), fam1.getTable());
+
+    // ts1, we just have v1
+    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
+    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts1, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts1, value1);
+
+    // at ts2, don't have the above anymore
+    pairs.clear();
+    expected = Collections.emptyList();
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts2, ts2 + 1, value1, value1);
+    // but we do have the new entry at ts2
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(value3, col2));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts2, value1);
+
+    // now build up a delete with a couple different timestamps
+    Delete d = new Delete(row1);
+    // these deletes have to match the exact ts since we are doing an exact match (deleteColumn).
+    d.deleteColumn(FAM, indexed_qualifer, ts1);
+    // since this doesn't match exactly, we actually shouldn't see a change in table state
+    d.deleteColumn(FAM2, regular_qualifer, ts3);
+    primary.delete(d);
+
+    // at ts1, we should have the put covered exactly by the delete and into the entire future
+    expected = Collections.emptyList();
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts1, Long.MAX_VALUE, value1,
+      value1);
+
+    // at ts2, we should just see value3
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(value3, col2));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts2, value1);
+
+    // the later delete is a point delete, so we shouldn't see any change at ts3
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts2, ts3, value1,
+      HConstants.EMPTY_END_ROW);
+
+    // cleanup
+    closeAndCleanupTables(primary, index1);
+  }
+
+  /**
+   * Covering deletes (via {@link Delete#deleteColumns}) cover everything back in time from the
+   * given time. If its modifying the latest state, we don't need to do anything but add deletes. If
+   * its modifying back in time state, we need to just fix up the surrounding elements as anything
+   * else ahead of it will be fixed up by later updates.
+   * <p>
+   * similar to {@link #testMultipleTimestampsInSingleDelete()}, but with covering deletes.
+   * @throws Exception on failure
+   */
+  @Test
+  public void testDeleteColumnsInThePast() throws Exception {
+    HTable primary = createSetupTables(fam1);
+
+    // do a put to the primary table
+    Put p = new Put(row1);
+    long ts1 = 10, ts2 = 11, ts3 = 12;
+    p.add(FAM, indexed_qualifer, ts1, value1);
+    p.add(FAM2, regular_qualifer, ts2, value3);
+    primary.put(p);
+    primary.flushCommits();
+
+    // now build up a delete with a couple different timestamps
+    Delete d = new Delete(row1);
+    // these deletes don't need to match the exact ts because they cover everything earlier
+    d.deleteColumns(FAM, indexed_qualifer, ts2);
+    d.deleteColumns(FAM2, regular_qualifer, ts3);
+    primary.delete(d);
+
+    // read the index for the expected values
+    HTable index1 = new HTable(UTIL.getConfiguration(), fam1.getTable());
+
+    // build the expected kvs
+    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
+    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
+
+    // check the first entry at ts1
+    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts1, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts1, value1);
+
+    // delete at ts2 changes what the put would insert
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(value3, col2));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts2, value1);
+
+    // final delete clears out everything
+    expected = Collections.emptyList();
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts3, value1);
+
+    // cleanup
+    closeAndCleanupTables(primary, index1);
+  }
+  
+  /**
+   * If the client is using custom timestamps is possible that the updates come out-of-order (i.e.
+   * update to ts 10 comes after the update to ts 12). In the case, we need to be sure that the
+   * index is correctly updated when the out of order put arrives.
+   * @throws Exception
+   */
+  @Test
+  public void testOutOfOrderUpdates() throws Exception {
+    HTable primary = createSetupTables(fam1);
+
+    // do a put to the primary table
+    Put p = new Put(row1);
+    long ts = 12;
+    p.add(FAM, indexed_qualifer, ts, value1);
+    primary.put(p);
+    primary.flushCommits();
+
+    // read the index for the expected values
+    HTable index1 = new HTable(UTIL.getConfiguration(), fam1.getTable());
+
+    // build the expected kvs
+    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
+    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
+
+    // check the first entry at ts
+    List<KeyValue> expectedTs1 = CoveredColumnIndexCodec
+        .getIndexKeyValueForTesting(row1, ts, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expectedTs1, ts, value1);
+
+    // now make a put back in time
+    long ts2 = ts - 2;
+    p = new Put(row1);
+    p.add(FAM, indexed_qualifer, ts2, value2);
+    primary.put(p);
+    primary.flushCommits();
+
+    pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
+    pairs.add(new Pair<byte[], CoveredColumn>(value2, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
+
+    // check to make sure the back in time entry exists
+    List<KeyValue> expectedTs2 = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2,
+      pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expectedTs2, ts2, value2);
+    // then it should be gone at the newer ts (because it deletes itself)
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, Collections.<KeyValue> emptyList(), ts2,
+      ts + 1, value2, HConstants.EMPTY_END_ROW);
+
+    // but that the original index entry is still visible at ts, just fine
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expectedTs1, ts, value1);
+
+    // cleanup
+    closeAndCleanupTables(primary, index1);
+  }
+
+  /**
+   * Its possible (i.e. from a fast, frequently writing client) that they put more than the
+   * 'visible' number of versions in a row before a client make a put 'back in time' on that row. If
+   * we don't scan the current table properly, we won't see an index update for that 'back in time'
+   * update since the usual lookup will only see the regular number of versions. This ability to see
+   * back in time depends on running HBase 0.94.9
+   * @throws Exception on failure
+   */
+  @Test
+  public void testExceedVersionsOutOfOrderPut() throws Exception {
+    // setup the index
+    HTable primary = createSetupTables(fam2);
+
+    // do a put to the primary table
+    Put p = new Put(row1);
+    long ts1 = 1, ts2 = 2, ts3 = 3, ts4 = 4, ts5 = 5;
+    byte[] value4 = Bytes.toBytes("val4");
+    byte[] value5 = Bytes.toBytes("val5");
+    p.add(FAM2, indexed_qualifer, ts1, value1);
+    primary.put(p);
+    primary.flushCommits();
+
+    p = new Put(row1);
+    p.add(FAM2, indexed_qualifer, ts3, value3);
+    primary.put(p);
+    primary.flushCommits();
+
+    p = new Put(row1);
+    p.add(FAM2, indexed_qualifer, ts4, value4);
+    primary.put(p);
+    primary.flushCommits();
+
+    p = new Put(row1);
+    p.add(FAM2, indexed_qualifer, ts5, value5);
+    primary.put(p);
+    primary.flushCommits();
+
+    // read the index for the expected values
+    HTable index = new HTable(UTIL.getConfiguration(), fam2.getTable());
+
+    // do a raw scan of everything in the table
+    if (LOG.isDebugEnabled()) {
+      // the whole table, all the keys
+      Scan s = new Scan();
+      s.setRaw(true);
+      ResultScanner scanner = index.getScanner(s);
+      for (Result r : scanner) {
+        LOG.debug("Found row:" + r);
+      }
+      scanner.close();
+    }
+
+    /*
+     * now we have definitely exceeded the number of versions visible to a usual client of the
+     * primary table, so we should try doing a put 'back in time' an make sure that has the correct
+     * index values and cleanup
+     */
+    p = new Put(row1);
+    p.add(FAM2, indexed_qualifer, ts2, value2);
+    primary.put(p);
+    primary.flushCommits();
+
+    // // read the index for the expected values
+    // HTable index = new HTable(UTIL.getConfiguration(), fam2.getTable());
+    //
+    // do a raw scan of everything in the table
+    if (LOG.isDebugEnabled()) {
+      // the whole table, all the keys
+      Scan s = new Scan();
+      s.setRaw(true);
+      ResultScanner scanner = index.getScanner(s);
+      for (Result r : scanner) {
+        LOG.debug("Found row:" + r);
+      }
+      scanner.close();
+    }
+
+    // build the expected kvs
+    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
+    pairs.add(new Pair<byte[], CoveredColumn>(value1, col3));
+
+    // check the value1 should be present at the earliest timestamp
+    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts1, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index, expected, ts1, value1, value2);
+
+    // and value1 should be removed at ts2 (even though it came later)
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index, Collections.<KeyValue> emptyList(), ts1,
+      ts2 + 1, value1, value2); // timestamp + 1 since its exclusive end timestamp
+
+    // late added column should be there just fine at its timestamp
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value2, col3));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index, expected, ts2, value2);
+
+    // and check that the late entry also removes its self at the next timestamp up
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index, Collections.<KeyValue> emptyList(), ts3,
+      value2, value3);
+
+    // then we should have the rest of the inserts at their appropriate timestamps. Everything else
+    // should be exactly the same, except we shouldn't see ts0 anymore at ts2
+
+    // check the third entry
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value3, col3));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts3, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index, expected, ts3, value3);
+
+    // check the fourth entry
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value4, col3));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts4, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index, expected, ts4, value4);
+
+    // check the first entry at ts4
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value2, col3));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index, expected, ts2, value2);
+    // verify that we remove the entry, even though its too far 'back in time'
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index, Collections.<KeyValue> emptyList(), ts3,
+      value4);
+
+    // cleanup
+    closeAndCleanupTables(primary, index);
+  }
+
+  /**
+   * Similar to {@link #testExceedVersionsOutOfOrderPut()}, but mingles deletes and puts.
+   * @throws Exception on failure
+   */
+  @Test
+  public void testExceedVersionsOutOfOrderUpdates() throws Exception {
+    HTable primary = createSetupTables(fam1);
+
+    // setup the data to store
+    long ts1 = 1, ts2 = 2, ts3 = 3, ts4 = 4, ts5 = 5, ts6 = 6;
+    byte[] value4 = Bytes.toBytes("val4"), value5 = Bytes.toBytes("val5"), value6 =
+        Bytes.toBytes("val6");
+    // values for the other column to index
+    byte[] v1_1 = ArrayUtils.addAll(value1, Bytes.toBytes("_otherCol")), v3_1 =
+        ArrayUtils.addAll(value3, Bytes.toBytes("_otherCol")), v5_1 =
+        ArrayUtils.addAll(value5, Bytes.toBytes("_otherCol")), v6_1 =
+        ArrayUtils.addAll(value6, Bytes.toBytes("_otherCol"));
+
+    // make some puts to the primary table
+    Put p = new Put(row1);
+    p.add(FAM, indexed_qualifer, ts1, value1);
+    p.add(FAM2, indexed_qualifer, ts1, v1_1);
+    primary.put(p);
+    primary.flushCommits();
+
+    p = new Put(row1);
+    p.add(FAM, indexed_qualifer, ts3, value3);
+    p.add(FAM2, indexed_qualifer, ts3, v3_1);
+    primary.put(p);
+    primary.flushCommits();
+
+    p = new Put(row1);
+    p.add(FAM, indexed_qualifer, ts5, value5);
+    p.add(FAM2, indexed_qualifer, ts5, v5_1);
+    primary.put(p);
+    primary.flushCommits();
+
+    p = new Put(row1);
+    p.add(FAM, indexed_qualifer, ts6, value6);
+    p.add(FAM2, indexed_qualifer, ts6, v6_1);
+    primary.put(p);
+    primary.flushCommits();
+
+    /*
+     * now we have definitely exceeded the number of versions visible to a usual client of the
+     * primary table, so we should try doing a put 'back in time' an make sure that has the correct
+     * index values and cleanup
+     */
+    p = new Put(row1);
+    p.add(FAM, indexed_qualifer, ts2, value2);
+    primary.put(p);
+    primary.flushCommits();
+
+    // read the index for the expected values
+    HTable index1 = new HTable(UTIL.getConfiguration(), fam1.getTable());
+
+    // do a raw scan of everything in the table
+    if (LOG.isDebugEnabled()) {
+      Scan s = new Scan();
+      s.setRaw(true);
+      ResultScanner scanner = index1.getScanner(s);
+      for (Result r : scanner) {
+        LOG.debug("Found row:" + r);
+      }
+      scanner.close();
+    }
+
+    // build the expected kvs
+    List<Pair<byte[], CoveredColumn>> pairs = new ArrayList<Pair<byte[], CoveredColumn>>();
+    pairs.add(new Pair<byte[], CoveredColumn>(value1, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(v1_1, col2));
+
+    // check the value1 should be present at the earliest timestamp
+    List<KeyValue> expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts1, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts1, value1, value2);
+
+    // and value1 should be removed at ts2 (even though it came later)
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, Collections.<KeyValue> emptyList(), ts1,
+      ts2 + 1, value1, value2); // timestamp + 1 since its exclusive end timestamp
+
+    // late added column should be there just fine at its timestamp
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value2, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(v1_1, col2));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts2, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts2, value2);
+
+    // and check that the late entry also removes its self at the next timestamp up
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, Collections.<KeyValue> emptyList(), ts3,
+      value2, value3);
+
+    // -----------------------------------------------
+    // Check Delete intermingled
+    // -----------------------------------------------
+
+    // verify that the old row is there
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value3, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(v3_1, col2));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts3, pairs);
+    // scan from the start key forward (should only include [value3][v3_3])
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts3, expected.get(0).getKey(),
+      value4);
+
+    // then do a delete of just one of the indexed columns. This should insert a delete for all just
+    // the single value, then a put & a later corresponding in the past for the new value
+    Delete d = new Delete(row1);
+    d.deleteColumn(FAM2, indexed_qualifer, ts3);
+    primary.delete(d);
+
+    // we shouldn't find that entry, but we should find [value3][v1_1] since that is next entry back
+    // in time from the current
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value3, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(v1_1, col2));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts3, pairs);
+    // it should be re-written at 3
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts3, value3, value4);
+
+    // but we shouldn't find it at ts5 since it should be covered again
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, Collections.<KeyValue> emptyList(), ts5,
+      value3, value4);
+
+    // now remove all the older columns in FAM2 at 4
+    d = new Delete(row1);
+    d.deleteColumns(FAM2, indexed_qualifer, ts4);
+    primary.delete(d);
+
+    // we shouldn't find that entry, but we should find [value3][null] since deleteColumns removes
+    // all the entries for that column
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value3, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts4, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts4, value3, value4);
+
+    // same as above, but now do it at 3 (on earlier)
+    d = new Delete(row1);
+    d.deleteColumns(FAM2, indexed_qualifer, ts3);
+    primary.delete(d);
+
+    // we shouldn't find that entry, but we should find [value3][null] since deleteColumns removes
+    // all the entries for that column
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value3, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(EMPTY_BYTES, col2));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts3, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts3, value3, value4);
+
+    // -----------------------------------------------
+    // then we should have the rest of the inserts at their appropriate timestamps. Everything else
+    // should be exactly the same, except we shouldn't see ts0 anymore at ts2
+    // -----------------------------------------------
+
+    // check the entry at 5
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value5, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(v5_1, col2));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts5, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts5, value5);
+
+    // check the entry at 6
+    pairs.clear();
+    pairs.add(new Pair<byte[], CoveredColumn>(value6, col1));
+    pairs.add(new Pair<byte[], CoveredColumn>(v6_1, col2));
+    expected = CoveredColumnIndexCodec.getIndexKeyValueForTesting(row1, ts6, pairs);
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts6, value5);
+
+    // cleanup
+    closeAndCleanupTables(primary, index1);
+  }
+
+  /**
+   * Create the primary table (to which you should write), setup properly for indexing the given
+   * {@link ColumnGroup}s. Also creates the necessary index tables to match the passes groups.
+   * @param groups {@link ColumnGroup}s to index, creating one index table per column group.
+   * @return reference to the primary table
+   * @throws IOException if there is an issue communicating with HBase
+   */
+  private HTable createSetupTables(ColumnGroup... groups) throws IOException {
+    HBaseAdmin admin = UTIL.getHBaseAdmin();
+    // setup the index
+    CoveredColumnIndexSpecifierBuilder builder = new CoveredColumnIndexSpecifierBuilder();
+    for (ColumnGroup group : groups) {
+      builder.addIndexGroup(group);
+      // create the index tables
+      CoveredColumnIndexer.createIndexTable(admin, group.getTable());
+    }
+
+    // setup the primary table
+    String indexedTableName = Bytes.toString(TestTable.getTableName());
+    HTableDescriptor pTable = new HTableDescriptor(indexedTableName);
+    pTable.addFamily(new HColumnDescriptor(FAM));
+    pTable.addFamily(new HColumnDescriptor(FAM2));
+    builder.build(pTable);
+
+    // create the primary table
+    admin.createTable(pTable);
+    HTable primary = new HTable(UTIL.getConfiguration(), indexedTableName);
+    primary.setAutoFlush(false);
+    return primary;
+  }
+
+  private void closeAndCleanupTables(HTable... tables) throws IOException {
+    if (tables == null) {
+      return;
+    }
+
+    for (HTable table : tables) {
+      table.close();
+      UTIL.deleteTable(table.getTableName());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestEndtoEndIndexingWithCompression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestEndtoEndIndexingWithCompression.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestEndtoEndIndexingWithCompression.java
new file mode 100644
index 0000000..7cd495d
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestEndtoEndIndexingWithCompression.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.phoenix.hbase.index.covered.example;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec;
+import org.apache.hadoop.hbase.regionserver.wal.WALEditCodec;
+import org.junit.BeforeClass;
+
+import org.apache.phoenix.hbase.index.IndexTestingUtils;
+import org.apache.phoenix.hbase.index.Indexer;
+
+/**
+ * Test secondary indexing from an end-to-end perspective (client to server to index table).
+ */
+public class TestEndtoEndIndexingWithCompression extends TestEndToEndCoveredIndexing {
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    //add our codec and enable WAL compression
+    Configuration conf = UTIL.getConfiguration();
+    IndexTestingUtils.setupConfig(conf);
+    // disable version checking, so we can test against whatever version of HBase happens to be
+    // installed (right now, its generally going to be SNAPSHOT versions).
+    conf.setBoolean(Indexer.CHECK_VERSION_CONF_KEY, false);
+    conf.set(WALEditCodec.WAL_EDIT_CODEC_CLASS_KEY,
+    IndexedWALEditCodec.class.getName());
+    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
+    
+    //start the mini-cluster
+    UTIL.startMiniCluster();
+  }
+}
\ No newline at end of file


[07/15] Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/EmptyScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/EmptyScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/EmptyScanner.java
new file mode 100644
index 0000000..a28268c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/EmptyScanner.java
@@ -0,0 +1,50 @@
+/*
+ * 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.phoenix.hbase.index.scanner;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+
+/**
+ * {@link Scanner} that has no underlying data
+ */
+public class EmptyScanner implements Scanner {
+
+  @Override
+  public KeyValue next() throws IOException {
+    return null;
+  }
+
+  @Override
+  public boolean seek(KeyValue next) throws IOException {
+    return false;
+  }
+
+  @Override
+  public KeyValue peek() throws IOException {
+    return null;
+  }
+
+  @Override
+  public void close() throws IOException {
+    // noop
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/FilteredKeyValueScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/FilteredKeyValueScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/FilteredKeyValueScanner.java
new file mode 100644
index 0000000..d91aaaf
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/FilteredKeyValueScanner.java
@@ -0,0 +1,162 @@
+/*
+ * 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.phoenix.hbase.index.scanner;
+
+import java.io.IOException;
+import java.util.SortedSet;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+
+import org.apache.phoenix.hbase.index.covered.KeyValueStore;
+
+/**
+ * Combine a simplified version of the logic in the ScanQueryMatcher and the KeyValueScanner. We can get away with this
+ * here because we are only concerned with a single MemStore for the index; we don't need to worry about multiple column
+ * families or minimizing seeking through file - we just want to iterate the kvs quickly, in-memory.
+ */
+public class FilteredKeyValueScanner implements KeyValueScanner {
+
+    private KeyValueScanner delegate;
+    private Filter filter;
+
+    public FilteredKeyValueScanner(Filter filter, KeyValueStore store) {
+        this(filter, store.getScanner());
+    }
+
+    private FilteredKeyValueScanner(Filter filter, KeyValueScanner delegate) {
+        this.delegate = delegate;
+        this.filter = filter;
+    }
+
+    @Override
+    public KeyValue peek() {
+        return delegate.peek();
+    }
+
+    /**
+     * Same a {@link KeyValueScanner#next()} except that we filter out the next {@link KeyValue} until we find one that
+     * passes the filter.
+     * 
+     * @return the next {@link KeyValue} or <tt>null</tt> if no next {@link KeyValue} is present and passes all the
+     *         filters.
+     */
+    @Override
+    public KeyValue next() throws IOException {
+        seekToNextUnfilteredKeyValue();
+        return delegate.next();
+    }
+
+    @Override
+    public boolean seek(KeyValue key) throws IOException {
+        if (filter.filterAllRemaining()) { return false; }
+        // see if we can seek to the next key
+        if (!delegate.seek(key)) { return false; }
+
+        return seekToNextUnfilteredKeyValue();
+    }
+
+    private boolean seekToNextUnfilteredKeyValue() throws IOException {
+        while (true) {
+            KeyValue peeked = delegate.peek();
+            // no more key values, so we are done
+            if (peeked == null) { return false; }
+
+            // filter the peeked value to see if it should be served
+            ReturnCode code = filter.filterKeyValue(peeked);
+            switch (code) {
+            // included, so we are done
+            case INCLUDE:
+            case INCLUDE_AND_NEXT_COL:
+                return true;
+                // not included, so we need to go to the next row
+            case SKIP:
+            case NEXT_COL:
+            case NEXT_ROW:
+                delegate.next();
+                break;
+            // use a seek hint to find out where we should go
+            case SEEK_NEXT_USING_HINT:
+                delegate.seek(filter.getNextKeyHint(peeked));
+            }
+        }
+    }
+
+    @Override
+    public boolean reseek(KeyValue key) throws IOException {
+        this.delegate.reseek(key);
+        return this.seekToNextUnfilteredKeyValue();
+    }
+
+    @Override
+    public boolean requestSeek(KeyValue kv, boolean forward, boolean useBloom) throws IOException {
+        return this.reseek(kv);
+    }
+
+    @Override
+    public boolean isFileScanner() {
+        return false;
+    }
+
+    @Override
+    public long getSequenceID() {
+        return this.delegate.getSequenceID();
+    }
+
+    @Override
+    public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns, long oldestUnexpiredTS) {
+        throw new UnsupportedOperationException(this.getClass().getName()
+                + " doesn't support checking to see if it should use a scanner!");
+    }
+
+    @Override
+    public boolean realSeekDone() {
+        return this.delegate.realSeekDone();
+    }
+
+    @Override
+    public void enforceSeek() throws IOException {
+        this.delegate.enforceSeek();
+    }
+
+    @Override
+    public void close() {
+        this.delegate.close();
+    }
+
+    /*
+    @Override
+    public boolean backwardSeek(KeyValue arg0) throws IOException {
+        return this.delegate.backwardSeek(arg0);
+    }
+
+    @Override
+    public boolean seekToLastRow() throws IOException {
+        return this.delegate.seekToLastRow();
+    }
+
+    @Override
+    public boolean seekToPreviousRow(KeyValue arg0) throws IOException {
+        return this.delegate.seekToPreviousRow(arg0);
+    }
+    */
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/Scanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/Scanner.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/Scanner.java
new file mode 100644
index 0000000..868e892
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/Scanner.java
@@ -0,0 +1,55 @@
+/*
+ * 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.phoenix.hbase.index.scanner;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+/**
+ * Scan the primary table. This is similar to HBase's scanner, but ensures that you will never see
+ * deleted columns/rows
+ */
+public interface Scanner extends Closeable {
+
+  /**
+   * @return the next keyvalue in the scanner or <tt>null</tt> if there is no next {@link KeyValue}
+   * @throws IOException if there is an underlying error reading the data
+   */
+  public KeyValue next() throws IOException;
+
+  /**
+   * Seek to immediately before the given {@link KeyValue}. If that exact {@link KeyValue} is
+   * present in <tt>this</tt>, it will be returned by the next call to {@link #next()}. Otherwise,
+   * returns the next {@link KeyValue} after the seeked {@link KeyValue}.
+   * @param next {@link KeyValue} to seek to. Doesn't need to already be present in <tt>this</tt>
+   * @return <tt>true</tt> if there are values left in <tt>this</tt>, <tt>false</tt> otherwise
+   * @throws IOException if there is an error reading the underlying data.
+   */
+  public boolean seek(KeyValue next) throws IOException;
+
+  /**
+   * Read the {@link KeyValue} at the top of <tt>this</tt> without 'popping' it off the top of the
+   * scanner.
+   * @return the next {@link KeyValue} or <tt>null</tt> if there are no more values in <tt>this</tt>
+   * @throws IOException if there is an error reading the underlying data.
+   */
+  public KeyValue peek() throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
new file mode 100644
index 0000000..edc26d5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
@@ -0,0 +1,165 @@
+/*
+ * 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.phoenix.hbase.index.scanner;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.FamilyFilter;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.QualifierFilter;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.collect.Lists;
+import org.apache.phoenix.hbase.index.covered.KeyValueStore;
+import org.apache.phoenix.hbase.index.covered.filter.ApplyAndFilterDeletesFilter;
+import org.apache.phoenix.hbase.index.covered.filter.ColumnTrackingNextLargestTimestampFilter;
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.hbase.index.covered.update.ColumnTracker;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ *
+ */
+public class ScannerBuilder {
+
+  private KeyValueStore memstore;
+  private Mutation update;
+
+
+  public ScannerBuilder(KeyValueStore memstore, Mutation update) {
+    this.memstore = memstore;
+    this.update = update;
+  }
+
+  public Scanner buildIndexedColumnScanner(Collection<? extends ColumnReference> indexedColumns, ColumnTracker tracker, long ts) {
+
+    Filter columnFilters = getColumnFilters(indexedColumns);
+    FilterList filters = new FilterList(Lists.newArrayList(columnFilters));
+
+    // skip to the right TS. This needs to come before the deletes since the deletes will hide any
+    // state that comes before the actual kvs, so we need to capture those TS as they change the row
+    // state.
+    filters.addFilter(new ColumnTrackingNextLargestTimestampFilter(ts, tracker));
+
+    // filter out kvs based on deletes
+    filters.addFilter(new ApplyAndFilterDeletesFilter(getAllFamilies(indexedColumns)));
+
+    // combine the family filters and the rest of the filters as a
+    return getFilteredScanner(filters);
+  }
+
+  /**
+   * @param columns columns to filter
+   * @return filter that will skip any {@link KeyValue} that doesn't match one of the passed columns
+   *         and the
+   */
+  private Filter
+      getColumnFilters(Collection<? extends ColumnReference> columns) {
+    // each column needs to be added as an OR, so we need to separate them out
+    FilterList columnFilters = new FilterList(FilterList.Operator.MUST_PASS_ONE);
+
+    // create a filter that matches each column reference
+    for (ColumnReference ref : columns) {
+      Filter columnFilter =
+          new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(ref.getFamily()));
+      // combine with a match for the qualifier, if the qualifier is a specific qualifier
+      if (!Bytes.equals(ColumnReference.ALL_QUALIFIERS, ref.getQualifier())) {
+        columnFilter =
+            new FilterList(columnFilter, new QualifierFilter(CompareOp.EQUAL, new BinaryComparator(
+                ref.getQualifier())));
+      }
+      columnFilters.addFilter(columnFilter);
+    }
+    return columnFilters;
+  }
+
+  private Set<ImmutableBytesPtr>
+      getAllFamilies(Collection<? extends ColumnReference> columns) {
+    Set<ImmutableBytesPtr> families = new HashSet<ImmutableBytesPtr>();
+    for (ColumnReference ref : columns) {
+      families.add(new ImmutableBytesPtr(ref.getFamily()));
+    }
+    return families;
+  }
+
+  private Scanner getFilteredScanner(Filter filters) {
+    // create a scanner and wrap it as an iterator, meaning you can only go forward
+    final FilteredKeyValueScanner kvScanner = new FilteredKeyValueScanner(filters, memstore);
+    // seek the scanner to initialize it
+    KeyValue start = KeyValue.createFirstOnRow(update.getRow());
+    try {
+      if (!kvScanner.seek(start)) {
+        return new EmptyScanner();
+      }
+    } catch (IOException e) {
+      // This should never happen - everything should explode if so.
+      throw new RuntimeException(
+          "Failed to seek to first key from update on the memstore scanner!", e);
+    }
+
+    // we have some info in the scanner, so wrap it in an iterator and return.
+    return new Scanner() {
+
+      @Override
+      public KeyValue next() {
+        try {
+          return kvScanner.next();
+        } catch (IOException e) {
+          throw new RuntimeException("Error reading kvs from local memstore!");
+        }
+      }
+
+      @Override
+      public boolean seek(KeyValue next) throws IOException {
+        // check to see if the next kv is after the current key, in which case we can use reseek,
+        // which will be more efficient
+        KeyValue peek = kvScanner.peek();
+        // there is another value and its before the requested one - we can do a reseek!
+        if (peek != null) {
+          int compare = KeyValue.COMPARATOR.compare(peek, next);
+          if (compare < 0) {
+            return kvScanner.reseek(next);
+          } else if (compare == 0) {
+            // we are already at the given key!
+            return true;
+          }
+        }
+        return kvScanner.seek(next);
+      }
+
+      @Override
+      public KeyValue peek() throws IOException {
+        return kvScanner.peek();
+      }
+
+      @Override
+      public void close() {
+        kvScanner.close();
+      }
+    };
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/CachingHTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/CachingHTableFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/CachingHTableFactory.java
new file mode 100644
index 0000000..0c06e2b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/CachingHTableFactory.java
@@ -0,0 +1,110 @@
+/*
+ * 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.phoenix.hbase.index.table;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.commons.collections.map.LRUMap;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * A simple cache that just uses usual GC mechanisms to cleanup unused {@link HTableInterface}s.
+ * When requesting an {@link HTableInterface} via {@link #getTable}, you may get the same table as
+ * last time, or it may be a new table.
+ * <p>
+ * You <b>should not call {@link HTableInterface#close()} </b> that is handled when the table goes
+ * out of scope. Along the same lines, you must ensure to not keep a reference to the table for
+ * longer than necessary - this leak will ensure that the table never gets closed.
+ */
+public class CachingHTableFactory implements HTableFactory {
+
+  /**
+   * LRUMap that closes the {@link HTableInterface} when the table is evicted
+   */
+  @SuppressWarnings("serial")
+  public class HTableInterfaceLRUMap extends LRUMap {
+
+    public HTableInterfaceLRUMap(int cacheSize) {
+      super(cacheSize);
+    }
+
+    @Override
+    protected boolean removeLRU(LinkEntry entry) {
+      HTableInterface table = (HTableInterface) entry.getValue();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Closing connection to table: " + Bytes.toString(table.getTableName())
+            + " because it was evicted from the cache.");
+      }
+      try {
+        table.close();
+      } catch (IOException e) {
+        LOG.info("Failed to correctly close HTable: " + Bytes.toString(table.getTableName())
+            + " ignoring since being removed from queue.");
+      }
+      return true;
+    }
+  }
+
+  public static int getCacheSize(Configuration conf) {
+    return conf.getInt(CACHE_SIZE_KEY, DEFAULT_CACHE_SIZE);
+  }
+
+  private static final Log LOG = LogFactory.getLog(CachingHTableFactory.class);
+  private static final String CACHE_SIZE_KEY = "index.tablefactory.cache.size";
+  private static final int DEFAULT_CACHE_SIZE = 10;
+
+  private HTableFactory delegate;
+
+  @SuppressWarnings("rawtypes")
+  Map openTables;
+
+  public CachingHTableFactory(HTableFactory tableFactory, Configuration conf) {
+    this(tableFactory, getCacheSize(conf));
+  }
+
+  public CachingHTableFactory(HTableFactory factory, int cacheSize) {
+    this.delegate = factory;
+    openTables = new HTableInterfaceLRUMap(cacheSize);
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public HTableInterface getTable(ImmutableBytesPtr tablename) throws IOException {
+    ImmutableBytesPtr tableBytes = new ImmutableBytesPtr(tablename);
+    synchronized (openTables) {
+      HTableInterface table = (HTableInterface) openTables.get(tableBytes);
+      if (table == null) {
+        table = delegate.getTable(tablename);
+        openTables.put(tableBytes, table);
+      }
+      return table;
+    }
+  }
+
+  @Override
+  public void shutdown() {
+    this.delegate.shutdown();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/CoprocessorHTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/CoprocessorHTableFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/CoprocessorHTableFactory.java
new file mode 100644
index 0000000..33559da
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/CoprocessorHTableFactory.java
@@ -0,0 +1,68 @@
+/*
+ * 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.phoenix.hbase.index.table;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
+
+public class CoprocessorHTableFactory implements HTableFactory {
+
+  /** Number of milliseconds per-interval to retry zookeeper */
+  private static final String ZOOKEEPER_RECOVERY_RETRY_INTERVALMILL = "zookeeper.recovery.retry.intervalmill";
+  /** Number of retries for zookeeper */
+  private static final String ZOOKEEPER_RECOVERY_RETRY_KEY = "zookeeper.recovery.retry";
+  private static final Log LOG = LogFactory.getLog(CoprocessorHTableFactory.class);
+  private CoprocessorEnvironment e;
+
+  public CoprocessorHTableFactory(CoprocessorEnvironment e) {
+    this.e = e;
+  }
+
+  @Override
+  public HTableInterface getTable(ImmutableBytesPtr tablename) throws IOException {
+    Configuration conf = e.getConfiguration();
+    // make sure writers fail fast
+    IndexManagementUtil.setIfNotSet(conf, HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
+    IndexManagementUtil.setIfNotSet(conf, HConstants.HBASE_CLIENT_PAUSE, 1000);
+    IndexManagementUtil.setIfNotSet(conf, ZOOKEEPER_RECOVERY_RETRY_KEY, 3);
+    IndexManagementUtil.setIfNotSet(conf, ZOOKEEPER_RECOVERY_RETRY_INTERVALMILL, 100);
+    IndexManagementUtil.setIfNotSet(conf, HConstants.ZK_SESSION_TIMEOUT, 30000);
+    IndexManagementUtil.setIfNotSet(conf, HConstants.HBASE_RPC_TIMEOUT_KEY, 5000);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Creating new HTable: " + Bytes.toString(tablename.copyBytesIfNecessary()));
+    }
+    return this.e.getTable(tablename.copyBytesIfNecessary());
+  }
+
+  @Override
+  public void shutdown() {
+    // noop
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/HTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/HTableFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/HTableFactory.java
new file mode 100644
index 0000000..bef3d34
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/HTableFactory.java
@@ -0,0 +1,32 @@
+/*
+ * 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.phoenix.hbase.index.table;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.HTableInterface;
+
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+
+public interface HTableFactory {
+
+  public HTableInterface getTable(ImmutableBytesPtr tablename) throws IOException;
+
+  public void shutdown();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/HTableInterfaceReference.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/HTableInterfaceReference.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/HTableInterfaceReference.java
new file mode 100644
index 0000000..b6d8d8e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/HTableInterfaceReference.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.hbase.index.table;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * Reference to an HTableInterface. Currently, its pretty simple in that it is just a wrapper around
+ * the table name.
+ */
+public class HTableInterfaceReference {
+
+  private ImmutableBytesPtr tableName;
+
+
+  public HTableInterfaceReference(ImmutableBytesPtr tableName) {
+    this.tableName = tableName;
+  }
+
+  public ImmutableBytesPtr get() {
+    return this.tableName;
+  }
+
+  public String getTableName() {
+    return Bytes.toString(this.tableName.get(),this.tableName.getOffset(), this.tableName.getLength());
+  }
+
+  @Override
+  public int hashCode() {
+      return tableName.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+      if (this == obj) return true;
+      if (obj == null) return false;
+      if (getClass() != obj.getClass()) return false;
+      HTableInterfaceReference other = (HTableInterfaceReference)obj;
+      return tableName.equals(other.tableName);
+  }
+
+  @Override
+  public String toString() {
+    return Bytes.toString(this.tableName.get());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/util/ImmutableBytesPtr.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/util/ImmutableBytesPtr.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/util/ImmutableBytesPtr.java
new file mode 100644
index 0000000..9825c77
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/util/ImmutableBytesPtr.java
@@ -0,0 +1,111 @@
+/*
+ * 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.phoenix.hbase.index.util;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+public class ImmutableBytesPtr extends ImmutableBytesWritable {
+    private int hashCode;
+    
+    public ImmutableBytesPtr() {
+    }
+
+    public ImmutableBytesPtr(byte[] bytes) {
+        super(bytes);
+        hashCode = super.hashCode();
+    }
+
+    public ImmutableBytesPtr(ImmutableBytesWritable ibw) {
+        super(ibw.get(), ibw.getOffset(), ibw.getLength());
+        hashCode = super.hashCode();
+    }
+
+    public ImmutableBytesPtr(ImmutableBytesPtr ibp) {
+        super(ibp.get(), ibp.getOffset(), ibp.getLength());
+        hashCode = ibp.hashCode;
+    }
+
+    public ImmutableBytesPtr(byte[] bytes, int offset, int length) {
+        super(bytes, offset, length);
+        hashCode = super.hashCode();
+    }
+
+    @Override
+    public int hashCode() {
+        return hashCode;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        ImmutableBytesPtr that = (ImmutableBytesPtr)obj;
+        if (this.hashCode != that.hashCode) return false;
+        if (Bytes.compareTo(this.get(), this.getOffset(), this.getLength(), that.get(), that.getOffset(), that.getLength()) != 0) return false;
+        return true;
+    }
+
+    public void set(ImmutableBytesWritable ptr) {
+        set(ptr.get(),ptr.getOffset(),ptr.getLength());
+      }
+
+    /**
+     * @param b Use passed bytes as backing array for this instance.
+     */
+    @Override
+    public void set(final byte [] b) {
+      super.set(b);
+      hashCode = super.hashCode();
+    }
+
+    /**
+     * @param b Use passed bytes as backing array for this instance.
+     * @param offset
+     * @param length
+     */
+    @Override
+    public void set(final byte [] b, final int offset, final int length) {
+        super.set(b,offset,length);
+        hashCode = super.hashCode();
+    }
+
+    @Override
+    public void readFields(final DataInput in) throws IOException {
+        super.readFields(in);
+        hashCode = super.hashCode();
+    }
+    
+    /**
+     * @return the backing byte array, copying only if necessary
+     */
+    public byte[] copyBytesIfNecessary() {
+    return copyBytesIfNecessary(this);
+    }
+
+  public static byte[] copyBytesIfNecessary(ImmutableBytesWritable ptr) {
+    if (ptr.getOffset() == 0 && ptr.getLength() == ptr.get().length) {
+      return ptr.get();
+    }
+    return ptr.copyBytes();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/util/IndexManagementUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/util/IndexManagementUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/util/IndexManagementUtil.java
new file mode 100644
index 0000000..76ec9ce
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/util/IndexManagementUtil.java
@@ -0,0 +1,244 @@
+/*
+ * 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.phoenix.hbase.index.util;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
+
+import com.google.common.collect.Maps;
+import org.apache.phoenix.hbase.index.ValueGetter;
+import org.apache.phoenix.hbase.index.builder.IndexBuildingFailureException;
+import org.apache.phoenix.hbase.index.covered.data.LazyValueGetter;
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.hbase.index.scanner.Scanner;
+
+/**
+ * Utility class to help manage indexes
+ */
+public class IndexManagementUtil {
+
+    private IndexManagementUtil() {
+        // private ctor for util classes
+    }
+
+    // Don't rely on statically defined classes constants from classes that may not exist
+    // in earlier HBase versions
+    public static final String INDEX_WAL_EDIT_CODEC_CLASS_NAME = "org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec";
+    public static final String HLOG_READER_IMPL_KEY = "hbase.regionserver.hlog.reader.impl";
+    public static final String WAL_EDIT_CODEC_CLASS_KEY = "hbase.regionserver.wal.codec";
+
+    private static final String INDEX_HLOG_READER_CLASS_NAME = "org.apache.hadoop.hbase.regionserver.wal.IndexedHLogReader";
+    private static final Log LOG = LogFactory.getLog(IndexManagementUtil.class);
+
+    public static boolean isWALEditCodecSet(Configuration conf) {
+        // check to see if the WALEditCodec is installed
+        try {
+            // Use reflection to load the IndexedWALEditCodec, since it may not load with an older version
+            // of HBase
+            Class.forName(INDEX_WAL_EDIT_CODEC_CLASS_NAME);
+        } catch (Throwable t) {
+            return false;
+        }
+        if (INDEX_WAL_EDIT_CODEC_CLASS_NAME.equals(conf.get(WAL_EDIT_CODEC_CLASS_KEY, null))) {
+            // its installed, and it can handle compression and non-compression cases
+            return true;
+        }
+        return false;
+    }
+
+    public static void ensureMutableIndexingCorrectlyConfigured(Configuration conf) throws IllegalStateException {
+
+        // check to see if the WALEditCodec is installed
+        if (isWALEditCodecSet(conf)) { return; }
+
+        // otherwise, we have to install the indexedhlogreader, but it cannot have compression
+        String codecClass = INDEX_WAL_EDIT_CODEC_CLASS_NAME;
+        String indexLogReaderName = INDEX_HLOG_READER_CLASS_NAME;
+        try {
+            // Use reflection to load the IndexedHLogReader, since it may not load with an older version
+            // of HBase
+            Class.forName(indexLogReaderName);
+        } catch (ClassNotFoundException e) {
+            throw new IllegalStateException(codecClass + " is not installed, but "
+                    + indexLogReaderName + " hasn't been installed in hbase-site.xml under " + HLOG_READER_IMPL_KEY);
+        }
+        if (indexLogReaderName.equals(conf.get(HLOG_READER_IMPL_KEY, indexLogReaderName))) {
+            if (conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false)) { throw new IllegalStateException(
+                    "WAL Compression is only supported with " + codecClass
+                            + ". You can install in hbase-site.xml, under " + WAL_EDIT_CODEC_CLASS_KEY); }
+        } else {
+            throw new IllegalStateException(codecClass + " is not installed, but "
+                    + indexLogReaderName + " hasn't been installed in hbase-site.xml under " + HLOG_READER_IMPL_KEY);
+        }
+
+    }
+
+    public static ValueGetter createGetterFromKeyValues(Collection<KeyValue> pendingUpdates) {
+        final Map<ReferencingColumn, ImmutableBytesPtr> valueMap = Maps.newHashMapWithExpectedSize(pendingUpdates
+                .size());
+        for (KeyValue kv : pendingUpdates) {
+            // create new pointers to each part of the kv
+            ImmutableBytesPtr family = new ImmutableBytesPtr(kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength());
+            ImmutableBytesPtr qual = new ImmutableBytesPtr(kv.getBuffer(), kv.getQualifierOffset(),
+                    kv.getQualifierLength());
+            ImmutableBytesPtr value = new ImmutableBytesPtr(kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
+            valueMap.put(new ReferencingColumn(family, qual), value);
+        }
+        return new ValueGetter() {
+            @Override
+            public ImmutableBytesPtr getLatestValue(ColumnReference ref) throws IOException {
+                return valueMap.get(ReferencingColumn.wrap(ref));
+            }
+        };
+    }
+
+    private static class ReferencingColumn {
+        ImmutableBytesPtr family;
+        ImmutableBytesPtr qual;
+
+        static ReferencingColumn wrap(ColumnReference ref) {
+            ImmutableBytesPtr family = new ImmutableBytesPtr(ref.getFamily());
+            ImmutableBytesPtr qual = new ImmutableBytesPtr(ref.getQualifier());
+            return new ReferencingColumn(family, qual);
+        }
+
+        public ReferencingColumn(ImmutableBytesPtr family, ImmutableBytesPtr qual) {
+            this.family = family;
+            this.qual = qual;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + ((family == null) ? 0 : family.hashCode());
+            result = prime * result + ((qual == null) ? 0 : qual.hashCode());
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj) return true;
+            if (obj == null) return false;
+            if (getClass() != obj.getClass()) return false;
+            ReferencingColumn other = (ReferencingColumn)obj;
+            if (family == null) {
+                if (other.family != null) return false;
+            } else if (!family.equals(other.family)) return false;
+            if (qual == null) {
+                if (other.qual != null) return false;
+            } else if (!qual.equals(other.qual)) return false;
+            return true;
+        }
+    }
+
+    public static ValueGetter createGetterFromScanner(Scanner scanner, byte[] currentRow) {
+        return new LazyValueGetter(scanner, currentRow);
+    }
+
+    /**
+     * check to see if the kvs in the update match any of the passed columns. Generally, this is useful to for an index
+     * codec to determine if a given update should even be indexed. This assumes that for any index, there are going to
+     * small number of columns, versus the number of kvs in any one batch.
+     */
+    public static boolean updateMatchesColumns(Collection<KeyValue> update, List<ColumnReference> columns) {
+        // check to see if the kvs in the new update even match any of the columns requested
+        // assuming that for any index, there are going to small number of columns, versus the number of
+        // kvs in any one batch.
+        boolean matches = false;
+        outer: for (KeyValue kv : update) {
+            for (ColumnReference ref : columns) {
+                if (ref.matchesFamily(kv.getFamily()) && ref.matchesQualifier(kv.getQualifier())) {
+                    matches = true;
+                    // if a single column matches a single kv, we need to build a whole scanner
+                    break outer;
+                }
+            }
+        }
+        return matches;
+    }
+
+    /**
+     * Check to see if the kvs in the update match any of the passed columns. Generally, this is useful to for an index
+     * codec to determine if a given update should even be indexed. This assumes that for any index, there are going to
+     * small number of kvs, versus the number of columns in any one batch.
+     * <p>
+     * This employs the same logic as {@link #updateMatchesColumns(Collection, List)}, but is flips the iteration logic
+     * to search columns before kvs.
+     */
+    public static boolean columnMatchesUpdate(List<ColumnReference> columns, Collection<KeyValue> update) {
+        boolean matches = false;
+        outer: for (ColumnReference ref : columns) {
+            for (KeyValue kv : update) {
+                if (ref.matchesFamily(kv.getFamily()) && ref.matchesQualifier(kv.getQualifier())) {
+                    matches = true;
+                    // if a single column matches a single kv, we need to build a whole scanner
+                    break outer;
+                }
+            }
+        }
+        return matches;
+    }
+
+    public static Scan newLocalStateScan(List<? extends Iterable<? extends ColumnReference>> refsArray) {
+        Scan s = new Scan();
+        s.setRaw(true);
+        // add the necessary columns to the scan
+        for (Iterable<? extends ColumnReference> refs : refsArray) {
+            for (ColumnReference ref : refs) {
+                s.addFamily(ref.getFamily());
+            }
+        }
+        s.setMaxVersions();
+        return s;
+    }
+
+    /**
+     * Propagate the given failure as a generic {@link IOException}, if it isn't already
+     * 
+     * @param e
+     *            reason indexing failed. If ,tt>null</tt>, throws a {@link NullPointerException}, which should unload
+     *            the coprocessor.
+     */
+    public static void rethrowIndexingException(Throwable e) throws IOException {
+        try {
+            throw e;
+        } catch (IOException e1) {
+            LOG.info("Rethrowing " + e);
+            throw e1;
+        } catch (Throwable e1) {
+            LOG.info("Rethrowing " + e1 + " as a " + IndexBuildingFailureException.class.getSimpleName());
+            throw new IndexBuildingFailureException("Failed to build index for unexpected reason!", e1);
+        }
+    }
+
+    public static void setIfNotSet(Configuration conf, String key, int value) {
+        if (conf.get(key) == null) {
+            conf.setInt(key, value);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/wal/IndexedKeyValue.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/wal/IndexedKeyValue.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/wal/IndexedKeyValue.java
new file mode 100644
index 0000000..253fd0d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/wal/IndexedKeyValue.java
@@ -0,0 +1,173 @@
+/*
+ * 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.phoenix.hbase.index.wal;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.regionserver.wal.HLog;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+
+public class IndexedKeyValue extends KeyValue {
+    private static int calcHashCode(ImmutableBytesPtr indexTableName, Mutation mutation) {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + indexTableName.hashCode();
+        result = prime * result + Arrays.hashCode(mutation.getRow());
+        return result;
+    }
+
+    private ImmutableBytesPtr indexTableName;
+    private Mutation mutation;
+    // optimization check to ensure that batches don't get replayed to the index more than once
+    private boolean batchFinished = false;
+    private int hashCode;
+
+    public IndexedKeyValue() {}
+
+    public IndexedKeyValue(byte[] bs, Mutation mutation) {
+        this.indexTableName = new ImmutableBytesPtr(bs);
+        this.mutation = mutation;
+        this.hashCode = calcHashCode(indexTableName, mutation);
+    }
+
+    public byte[] getIndexTable() {
+        return this.indexTableName.get();
+    }
+
+    public Mutation getMutation() {
+        return mutation;
+    }
+
+    /**
+     * This is a KeyValue that shouldn't actually be replayed, so we always mark it as an {@link HLog#METAFAMILY} so it
+     * isn't replayed via the normal replay mechanism
+     */
+    @Override
+    public boolean matchingFamily(final byte[] family) {
+        return Bytes.equals(family, HLog.METAFAMILY);
+    }
+
+    @Override
+    public String toString() {
+        return "IndexWrite:\n\ttable: " + indexTableName + "\n\tmutation:" + mutation;
+    }
+
+    /**
+     * This is a very heavy-weight operation and should only be done when absolutely necessary - it does a full
+     * serialization of the underyling mutation to compare the underlying data.
+     */
+    @Override
+    public boolean equals(Object obj) {
+        if(obj == null) return false;
+        if (this == obj) return true;
+        if (getClass() != obj.getClass()) return false;
+        IndexedKeyValue other = (IndexedKeyValue)obj;
+        if (hashCode() != other.hashCode()) return false;
+        if (!other.indexTableName.equals(this.indexTableName)) return false;
+        byte[] current = this.getMutationBytes();
+        byte[] otherMutation = other.getMutationBytes();
+        return Bytes.equals(current, otherMutation);
+    }
+
+    private byte[] getMutationBytes() {
+        ByteArrayOutputStream bos = null;
+        try {
+            bos = new ByteArrayOutputStream();
+            this.mutation.write(new DataOutputStream(bos));
+            bos.flush();
+            return bos.toByteArray();
+        } catch (IOException e) {
+            throw new IllegalArgumentException("Failed to get bytes for mutation!", e);
+        } finally {
+            if (bos != null) {
+                try {
+                    bos.close();
+                } catch (IOException e) {
+                    throw new IllegalArgumentException("Failed to get bytes for mutation!", e);
+                }
+            }
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        return hashCode;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        KeyValueCodec.write(out, this);
+    }
+
+    /**
+     * Internal write the underlying data for the entry - this does not do any special prefixing. Writing should be done
+     * via {@link KeyValueCodec#write(DataOutput, KeyValue)} to ensure consistent reading/writing of
+     * {@link IndexedKeyValue}s.
+     * 
+     * @param out
+     *            to write data to. Does not close or flush the passed object.
+     * @throws IOException
+     *             if there is a problem writing the underlying data
+     */
+    void writeData(DataOutput out) throws IOException {
+        Bytes.writeByteArray(out, this.indexTableName.get());
+        out.writeUTF(this.mutation.getClass().getName());
+        this.mutation.write(out);
+    }
+
+    /**
+     * This method shouldn't be used - you should use {@link KeyValueCodec#readKeyValue(DataInput)} instead. Its the
+     * complement to {@link #writeData(DataOutput)}.
+     */
+    @SuppressWarnings("javadoc")
+    @Override
+    public void readFields(DataInput in) throws IOException {
+        this.indexTableName = new ImmutableBytesPtr(Bytes.readByteArray(in));
+        Class<? extends Mutation> clazz;
+        try {
+            clazz = Class.forName(in.readUTF()).asSubclass(Mutation.class);
+            this.mutation = clazz.newInstance();
+            this.mutation.readFields(in);
+            this.hashCode = calcHashCode(indexTableName, mutation);
+        } catch (ClassNotFoundException e) {
+            throw new IOException(e);
+        } catch (InstantiationException e) {
+            throw new IOException(e);
+        } catch (IllegalAccessException e) {
+            throw new IOException(e);
+        }
+    }
+
+    public boolean getBatchFinished() {
+        return this.batchFinished;
+    }
+
+    public void markBatchFinished() {
+        this.batchFinished = true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/wal/KeyValueCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/wal/KeyValueCodec.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/wal/KeyValueCodec.java
new file mode 100644
index 0000000..3815937
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/wal/KeyValueCodec.java
@@ -0,0 +1,97 @@
+/*
+ * 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.phoenix.hbase.index.wal;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+
+/**
+ * Codec to encode/decode {@link KeyValue}s and {@link IndexedKeyValue}s within a {@link WALEdit}
+ */
+public class KeyValueCodec {
+
+  /**
+   * KeyValue length marker specifying that its actually an {@link IndexedKeyValue} rather than a
+   * regular {@link KeyValue}.
+   */
+  public static final int INDEX_TYPE_LENGTH_MARKER = -1;
+
+  /**
+   * Read a {@link List} of {@link KeyValue} from the input stream - may contain regular
+   * {@link KeyValue}s or {@link IndexedKeyValue}s.
+   * @param in to read from
+   * @return the next {@link KeyValue}s
+   * @throws IOException if the next {@link KeyValue} cannot be read
+   */
+  public static List<KeyValue> readKeyValues(DataInput in) throws IOException {
+    int size = in.readInt();
+    if (size == 0) {
+      return Collections.<KeyValue>emptyList();
+    }
+    List<KeyValue> kvs = new ArrayList<KeyValue>(size);
+    for (int i = 0; i < size; i++) {
+      kvs.add(readKeyValue(in));
+    }
+    return kvs;
+  }
+
+  /**
+   * Read a single {@link KeyValue} from the input stream - may either be a regular {@link KeyValue}
+   * or an {@link IndexedKeyValue}.
+   * @param in to read from
+   * @return the next {@link KeyValue}, if one is available
+   * @throws IOException if the next {@link KeyValue} cannot be read
+   */
+  public static KeyValue readKeyValue(DataInput in) throws IOException {
+    int length = in.readInt();
+    KeyValue kv;
+    // its a special IndexedKeyValue
+    if (length == INDEX_TYPE_LENGTH_MARKER) {
+      kv = new IndexedKeyValue();
+      kv.readFields(in);
+    } else {
+      kv = new KeyValue();
+      kv.readFields(length, in);
+    }
+    return kv;
+  }
+
+  /**
+   * Write a {@link KeyValue} or an {@link IndexedKeyValue} to the output stream. These can be read
+   * back via {@link #readKeyValue(DataInput)} or {@link #readKeyValues(DataInput)}.
+   * @param out to write to
+   * @param kv {@link KeyValue} to which to write
+   * @throws IOException if there is an error writing
+   */
+  public static void write(DataOutput out, KeyValue kv) throws IOException {
+    if (kv instanceof IndexedKeyValue) {
+      out.writeInt(INDEX_TYPE_LENGTH_MARKER);
+      ((IndexedKeyValue) kv).writeData(out);
+    } else {
+      kv.write(out);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexCommitter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexCommitter.java
new file mode 100644
index 0000000..d7fef5e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexCommitter.java
@@ -0,0 +1,37 @@
+/*
+ * 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.phoenix.hbase.index.write;
+
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import com.google.common.collect.Multimap;
+import org.apache.phoenix.hbase.index.exception.IndexWriteException;
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+
+/**
+ * Write the index updates to the index tables
+ */
+public interface IndexCommitter extends Stoppable {
+
+  void setup(IndexWriter parent, RegionCoprocessorEnvironment env, String name);
+
+  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite)
+      throws IndexWriteException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexFailurePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexFailurePolicy.java
new file mode 100644
index 0000000..5964647
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexFailurePolicy.java
@@ -0,0 +1,45 @@
+/*
+ * 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.phoenix.hbase.index.write;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import com.google.common.collect.Multimap;
+
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+
+/**
+ * Handle failures to write to the index tables.
+ */
+public interface IndexFailurePolicy extends Stoppable {
+
+  public void setup(Stoppable parent, RegionCoprocessorEnvironment env);
+
+  /**
+   * Handle the failure of the attempted index updates
+   * @param attempted map of index table -> mutations to apply
+   * @param cause reason why there was a failure
+ * @throws IOException 
+   */
+  public void
+      handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java
new file mode 100644
index 0000000..30797b2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java
@@ -0,0 +1,224 @@
+/*
+ * 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.phoenix.hbase.index.write;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.apache.phoenix.hbase.index.exception.IndexWriteException;
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * Do the actual work of writing to the index tables. Ensures that if we do fail to write to the
+ * index table that we cleanly kill the region/server to ensure that the region's WAL gets replayed.
+ * <p>
+ * We attempt to do the index updates in parallel using a backing threadpool. All threads are daemon
+ * threads, so it will not block the region from shutting down.
+ */
+public class IndexWriter implements Stoppable {
+
+  private static final Log LOG = LogFactory.getLog(IndexWriter.class);
+  private static final String INDEX_COMMITTER_CONF_KEY = "index.writer.commiter.class";
+  public static final String INDEX_FAILURE_POLICY_CONF_KEY = "index.writer.failurepolicy.class";
+  private AtomicBoolean stopped = new AtomicBoolean(false);
+  private IndexCommitter writer;
+  private IndexFailurePolicy failurePolicy;
+
+  /**
+   * @throws IOException if the {@link IndexWriter} or {@link IndexFailurePolicy} cannot be
+   *           instantiated
+   */
+  public IndexWriter(RegionCoprocessorEnvironment env, String name) throws IOException {
+    this(getCommitter(env), getFailurePolicy(env), env, name);
+  }
+
+  public static IndexCommitter getCommitter(RegionCoprocessorEnvironment env) throws IOException {
+    Configuration conf = env.getConfiguration();
+    try {
+      IndexCommitter committer =
+          conf.getClass(INDEX_COMMITTER_CONF_KEY, ParallelWriterIndexCommitter.class,
+            IndexCommitter.class).newInstance();
+      return committer;
+    } catch (InstantiationException e) {
+      throw new IOException(e);
+    } catch (IllegalAccessException e) {
+      throw new IOException(e);
+    }
+  }
+
+  public static IndexFailurePolicy getFailurePolicy(RegionCoprocessorEnvironment env)
+      throws IOException {
+    Configuration conf = env.getConfiguration();
+    try {
+      IndexFailurePolicy committer =
+          conf.getClass(INDEX_FAILURE_POLICY_CONF_KEY, KillServerOnFailurePolicy.class,
+            IndexFailurePolicy.class).newInstance();
+      return committer;
+    } catch (InstantiationException e) {
+      throw new IOException(e);
+    } catch (IllegalAccessException e) {
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Directly specify the {@link IndexCommitter} and {@link IndexFailurePolicy}. Both are expected
+   * to be fully setup before calling.
+   * @param committer
+   * @param policy
+   * @param env
+   */
+  public IndexWriter(IndexCommitter committer, IndexFailurePolicy policy,
+      RegionCoprocessorEnvironment env, String name) {
+    this(committer, policy);
+    this.writer.setup(this, env, name);
+    this.failurePolicy.setup(this, env);
+  }
+
+  /**
+   * Create an {@link IndexWriter} with an already setup {@link IndexCommitter} and
+   * {@link IndexFailurePolicy}.
+   * @param committer to write updates
+   * @param policy to handle failures
+   */
+  IndexWriter(IndexCommitter committer, IndexFailurePolicy policy) {
+    this.writer = committer;
+    this.failurePolicy = policy;
+  }
+  
+  /**
+   * Write the mutations to their respective table.
+   * <p>
+   * This method is blocking and could potentially cause the writer to block for a long time as we
+   * write the index updates. When we return depends on the specified {@link IndexCommitter}.
+   * <p>
+   * If update fails, we pass along the failure to the installed {@link IndexFailurePolicy}, which
+   * then decides how to handle the failure. By default, we use a {@link KillServerOnFailurePolicy},
+   * which ensures that the server crashes when an index write fails, ensuring that we get WAL
+   * replay of the index edits.
+   * @param indexUpdates Updates to write
+ * @throws IOException 
+   */
+  public void writeAndKillYourselfOnFailure(Collection<Pair<Mutation, byte[]>> indexUpdates) throws IOException  {
+    // convert the strings to htableinterfaces to which we can talk and group by TABLE
+    Multimap<HTableInterfaceReference, Mutation> toWrite = resolveTableReferences(indexUpdates);
+    writeAndKillYourselfOnFailure(toWrite);
+  }
+
+  /**
+   * see {@link #writeAndKillYourselfOnFailure(Collection)}.
+   * @param toWrite
+ * @throws IOException 
+   */
+  public void writeAndKillYourselfOnFailure(Multimap<HTableInterfaceReference, Mutation> toWrite) throws IOException {
+    try {
+      write(toWrite);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Done writing all index updates!\n\t" + toWrite);
+      }
+    } catch (Exception e) {
+      this.failurePolicy.handleFailure(toWrite, e);
+    }
+  }
+
+  /**
+   * Write the mutations to their respective table.
+   * <p>
+   * This method is blocking and could potentially cause the writer to block for a long time as we
+   * write the index updates. We only return when either:
+   * <ol>
+   * <li>All index writes have returned, OR</li>
+   * <li>Any single index write has failed</li>
+   * </ol>
+   * We attempt to quickly determine if any write has failed and not write to the remaining indexes
+   * to ensure a timely recovery of the failed index writes.
+   * @param toWrite Updates to write
+   * @throws IndexWriteException if we cannot successfully write to the index. Whether or not we
+   *           stop early depends on the {@link IndexCommitter}.
+   */
+  public void write(Collection<Pair<Mutation, byte[]>> toWrite) throws IndexWriteException {
+    write(resolveTableReferences(toWrite));
+  }
+
+  /**
+   * see {@link #write(Collection)}
+   * @param toWrite
+   * @throws IndexWriteException
+   */
+  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite)
+      throws IndexWriteException {
+    this.writer.write(toWrite);
+  }
+
+
+  /**
+   * Convert the passed index updates to {@link HTableInterfaceReference}s.
+   * @param indexUpdates from the index builder
+   * @return pairs that can then be written by an {@link IndexWriter}.
+   */
+  public static Multimap<HTableInterfaceReference, Mutation> resolveTableReferences(
+      Collection<Pair<Mutation, byte[]>> indexUpdates) {
+    Multimap<HTableInterfaceReference, Mutation> updates = ArrayListMultimap
+        .<HTableInterfaceReference, Mutation> create();
+    // simple map to make lookups easy while we build the map of tables to create
+    Map<ImmutableBytesPtr, HTableInterfaceReference> tables =
+        new HashMap<ImmutableBytesPtr, HTableInterfaceReference>(updates.size());
+    for (Pair<Mutation, byte[]> entry : indexUpdates) {
+      byte[] tableName = entry.getSecond();
+      ImmutableBytesPtr ptr = new ImmutableBytesPtr(tableName);
+      HTableInterfaceReference table = tables.get(ptr);
+      if (table == null) {
+        table = new HTableInterfaceReference(ptr);
+        tables.put(ptr, table);
+      }
+      updates.put(table, entry.getFirst());
+    }
+
+    return updates;
+  }
+
+  @Override
+  public void stop(String why) {
+    if (!this.stopped.compareAndSet(false, true)) {
+      // already stopped
+      return;
+    }
+    LOG.debug("Stopping because " + why);
+    this.writer.stop(why);
+    this.failurePolicy.stop(why);
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped.get();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriterUtils.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriterUtils.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriterUtils.java
new file mode 100644
index 0000000..db95970
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriterUtils.java
@@ -0,0 +1,72 @@
+/*
+ * 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.phoenix.hbase.index.write;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+
+import org.apache.phoenix.hbase.index.table.CoprocessorHTableFactory;
+import org.apache.phoenix.hbase.index.table.HTableFactory;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
+
+public class IndexWriterUtils {
+
+  private static final Log LOG = LogFactory.getLog(IndexWriterUtils.class);
+
+  /**
+   * Maximum number of threads to allow per-table when writing. Each writer thread (from
+   * {@link IndexWriterUtils#NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY}) has a single HTable.
+   * However, each table is backed by a threadpool to manage the updates to that table. this
+   * specifies the number of threads to allow in each of those tables. Generally, you shouldn't need
+   * to change this, unless you have a small number of indexes to which most of the writes go.
+   * Defaults to: {@value #DEFAULT_NUM_PER_TABLE_THREADS}.
+   * <p>
+   * For tables to which there are not a lot of writes, the thread pool automatically will decrease
+   * the number of threads to one (though it can burst up to the specified max for any given table),
+   * so increasing this to meet the max case is reasonable.
+   * <p>
+   * Setting this value too small can cause <b>catastrophic cluster failure</b>. The way HTable's
+   * underlying pool works is such that is does direct hand-off of tasks to threads. This works fine
+   * because HTables are assumed to work in a single-threaded context, so we never get more threads
+   * than regionservers. In a multi-threaded context, we can easily grow to more than that number of
+   * threads. Currently, HBase doesn't support a custom thread-pool to back the HTable via the
+   * coprocesor hooks, so we can't modify this behavior.
+   */
+  private static final String INDEX_WRITER_PER_TABLE_THREADS_CONF_KEY =
+      "index.writer.threads.pertable.max";
+  private static final int DEFAULT_NUM_PER_TABLE_THREADS = Integer.MAX_VALUE;
+
+  /** Configuration key that HBase uses to set the max number of threads for an HTable */
+  public static final String HTABLE_THREAD_KEY = "hbase.htable.threads.max";
+  private IndexWriterUtils() {
+    // private ctor for utilites
+  }
+
+  public static HTableFactory getDefaultDelegateHTableFactory(CoprocessorEnvironment env) {
+    // create a simple delegate factory, setup the way we need
+    Configuration conf = env.getConfiguration();
+    // set the number of threads allowed per table.
+    int htableThreads =
+        conf.getInt(IndexWriterUtils.INDEX_WRITER_PER_TABLE_THREADS_CONF_KEY, IndexWriterUtils.DEFAULT_NUM_PER_TABLE_THREADS);
+    LOG.trace("Creating HTableFactory with " + htableThreads + " threads for each HTable.");
+    IndexManagementUtil.setIfNotSet(conf, HTABLE_THREAD_KEY, htableThreads);
+    return new CoprocessorHTableFactory(env);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/KillServerOnFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/KillServerOnFailurePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/KillServerOnFailurePolicy.java
new file mode 100644
index 0000000..0b84cdf
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/KillServerOnFailurePolicy.java
@@ -0,0 +1,81 @@
+/*
+ * 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.phoenix.hbase.index.write;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import com.google.common.collect.Multimap;
+
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+
+/**
+ * Naive failure policy - kills the server on which it resides
+ */
+public class KillServerOnFailurePolicy implements IndexFailurePolicy {
+
+  private static final Log LOG = LogFactory.getLog(KillServerOnFailurePolicy.class);
+  private Abortable abortable;
+  private Stoppable stoppable;
+
+  @Override
+  public void setup(Stoppable parent, RegionCoprocessorEnvironment env) {
+    setup(parent, env.getRegionServerServices());
+  }
+
+  public void setup(Stoppable parent, Abortable abort) {
+    this.stoppable = parent;
+    this.abortable = abort;
+  }
+
+  @Override
+  public void stop(String why) {
+    // noop
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stoppable.isStopped();
+  }
+
+  @Override
+  public void
+      handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause) throws IOException {
+    // cleanup resources
+    this.stop("Killing ourselves because of an error:" + cause);
+    // notify the regionserver of the failure
+    String msg =
+        "Could not update the index table, killing server region because couldn't write to an index table";
+    LOG.error(msg, cause);
+    try {
+      this.abortable.abort(msg, cause);
+    } catch (Exception e) {
+      LOG.fatal("Couldn't abort this server to preserve index writes, "
+          + "attempting to hard kill the server");
+      System.exit(1);
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/ParallelWriterIndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/ParallelWriterIndexCommitter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/ParallelWriterIndexCommitter.java
new file mode 100644
index 0000000..55695ff
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/ParallelWriterIndexCommitter.java
@@ -0,0 +1,210 @@
+/*
+ * 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.phoenix.hbase.index.write;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import com.google.common.collect.Multimap;
+import org.apache.phoenix.hbase.index.exception.SingleIndexWriteFailureException;
+import org.apache.phoenix.hbase.index.parallel.EarlyExitFailure;
+import org.apache.phoenix.hbase.index.parallel.QuickFailingTaskRunner;
+import org.apache.phoenix.hbase.index.parallel.Task;
+import org.apache.phoenix.hbase.index.parallel.TaskBatch;
+import org.apache.phoenix.hbase.index.parallel.ThreadPoolBuilder;
+import org.apache.phoenix.hbase.index.parallel.ThreadPoolManager;
+import org.apache.phoenix.hbase.index.table.CachingHTableFactory;
+import org.apache.phoenix.hbase.index.table.HTableFactory;
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+
+/**
+ * Write index updates to the index tables in parallel. We attempt to early exit from the writes if
+ * any of the index updates fails. Completion is determined by the following criteria: *
+ * <ol>
+ * <li>All index writes have returned, OR</li>
+ * <li>Any single index write has failed</li>
+ * </ol>
+ * We attempt to quickly determine if any write has failed and not write to the remaining indexes to
+ * ensure a timely recovery of the failed index writes.
+ */
+public class ParallelWriterIndexCommitter implements IndexCommitter {
+
+  public static final String NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY = "index.writer.threads.max";
+  private static final int DEFAULT_CONCURRENT_INDEX_WRITER_THREADS = 10;
+  private static final String INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY =
+      "index.writer.threads.keepalivetime";
+  private static final Log LOG = LogFactory.getLog(ParallelWriterIndexCommitter.class);
+
+  private HTableFactory factory;
+  private Stoppable stopped;
+  private QuickFailingTaskRunner pool;
+
+  @Override
+  public void setup(IndexWriter parent, RegionCoprocessorEnvironment env, String name) {
+    Configuration conf = env.getConfiguration();
+    setup(IndexWriterUtils.getDefaultDelegateHTableFactory(env),
+      ThreadPoolManager.getExecutor(
+        new ThreadPoolBuilder(name, conf).
+          setMaxThread(NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY,
+            DEFAULT_CONCURRENT_INDEX_WRITER_THREADS).
+          setCoreTimeout(INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY), env),
+      env.getRegionServerServices(), parent, CachingHTableFactory.getCacheSize(conf));
+  }
+
+  /**
+   * Setup <tt>this</tt>.
+   * <p>
+   * Exposed for TESTING
+   */
+  void setup(HTableFactory factory, ExecutorService pool, Abortable abortable, Stoppable stop,
+      int cacheSize) {
+    this.factory = new CachingHTableFactory(factory, cacheSize);
+    this.pool = new QuickFailingTaskRunner(pool);
+    this.stopped = stop;
+  }
+
+  @Override
+  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite)
+      throws SingleIndexWriteFailureException {
+    /*
+     * This bit here is a little odd, so let's explain what's going on. Basically, we want to do the
+     * writes in parallel to each index table, so each table gets its own task and is submitted to
+     * the pool. Where it gets tricky is that we want to block the calling thread until one of two
+     * things happens: (1) all index tables get successfully updated, or (2) any one of the index
+     * table writes fail; in either case, we should return as quickly as possible. We get a little
+     * more complicated in that if we do get a single failure, but any of the index writes hasn't
+     * been started yet (its been queued up, but not submitted to a thread) we want to that task to
+     * fail immediately as we know that write is a waste and will need to be replayed anyways.
+     */
+
+    Set<Entry<HTableInterfaceReference, Collection<Mutation>>> entries = toWrite.asMap().entrySet();
+    TaskBatch<Void> tasks = new TaskBatch<Void>(entries.size());
+    for (Entry<HTableInterfaceReference, Collection<Mutation>> entry : entries) {
+      // get the mutations for each table. We leak the implementation here a little bit to save
+      // doing a complete copy over of all the index update for each table.
+      final List<Mutation> mutations = (List<Mutation>) entry.getValue();
+      final HTableInterfaceReference tableReference = entry.getKey();
+      /*
+       * Write a batch of index updates to an index table. This operation stops (is cancelable) via
+       * two mechanisms: (1) setting aborted or stopped on the IndexWriter or, (2) interrupting the
+       * running thread. The former will only work if we are not in the midst of writing the current
+       * batch to the table, though we do check these status variables before starting and before
+       * writing the batch. The latter usage, interrupting the thread, will work in the previous
+       * situations as was at some points while writing the batch, depending on the underlying
+       * writer implementation (HTableInterface#batch is blocking, but doesn't elaborate when is
+       * supports an interrupt).
+       */
+      tasks.add(new Task<Void>() {
+
+        /**
+         * Do the actual write to the primary table. We don't need to worry about closing the table
+         * because that is handled the {@link CachingHTableFactory}.
+         */
+        @Override
+        public Void call() throws Exception {
+          // this may have been queued, so another task infront of us may have failed, so we should
+          // early exit, if that's the case
+          throwFailureIfDone();
+
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Writing index update:" + mutations + " to table: " + tableReference);
+          }
+          try {
+            HTableInterface table = factory.getTable(tableReference.get());
+            throwFailureIfDone();
+            table.batch(mutations);
+          } catch (SingleIndexWriteFailureException e) {
+            throw e;
+          } catch (IOException e) {
+            throw new SingleIndexWriteFailureException(tableReference.toString(), mutations, e);
+          } catch (InterruptedException e) {
+            // reset the interrupt status on the thread
+            Thread.currentThread().interrupt();
+            throw new SingleIndexWriteFailureException(tableReference.toString(), mutations, e);
+          }
+          return null;
+        }
+
+        private void throwFailureIfDone() throws SingleIndexWriteFailureException {
+          if (this.isBatchFailed() || Thread.currentThread().isInterrupted()) {
+            throw new SingleIndexWriteFailureException(
+                "Pool closed, not attempting to write to the index!", null);
+          }
+
+        }
+      });
+    }
+
+    // actually submit the tasks to the pool and wait for them to finish/fail
+    try {
+      pool.submitUninterruptible(tasks);
+    } catch (EarlyExitFailure e) {
+      propagateFailure(e);
+    } catch (ExecutionException e) {
+      LOG.error("Found a failed index update!");
+      propagateFailure(e.getCause());
+    }
+
+  }
+
+  private void propagateFailure(Throwable throwable) throws SingleIndexWriteFailureException {
+    try {
+      throw throwable;
+    } catch (SingleIndexWriteFailureException e1) {
+      throw e1;
+    } catch (Throwable e1) {
+      throw new SingleIndexWriteFailureException(
+          "Got an abort notification while writing to the index!", e1);
+    }
+
+  }
+
+  /**
+   * {@inheritDoc}
+   * <p>
+   * This method should only be called <b>once</b>. Stopped state ({@link #isStopped()}) is managed
+   * by the external {@link Stoppable}. This call does not delegate the stop down to the
+   * {@link Stoppable} passed in the constructor.
+   * @param why the reason for stopping
+   */
+  @Override
+  public void stop(String why) {
+    LOG.info("Shutting down " + this.getClass().getSimpleName() + " because " + why);
+    this.pool.stop(why);
+    this.factory.shutdown();
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped.isStopped();
+  }
+}
\ No newline at end of file


[02/15] Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestFailWithoutRetries.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestFailWithoutRetries.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestFailWithoutRetries.java
new file mode 100644
index 0000000..8d9d65e
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/example/TestFailWithoutRetries.java
@@ -0,0 +1,148 @@
+/*
+ * 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.phoenix.hbase.index.covered.example;
+
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.phoenix.hbase.index.IndexTestingUtils;
+import org.apache.phoenix.hbase.index.TableName;
+import org.apache.phoenix.hbase.index.covered.IndexUpdate;
+import org.apache.phoenix.hbase.index.covered.TableState;
+import org.apache.phoenix.hbase.index.covered.example.ColumnGroup;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumn;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumnIndexSpecifierBuilder;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
+import org.apache.phoenix.index.BaseIndexCodec;
+
+/**
+ * If {@link DoNotRetryIOException} is not subclassed correctly (with the {@link String}
+ * constructor), {@link MultiResponse#readFields(java.io.DataInput)} will not correctly deserialize
+ * the exception, and just return <tt>null</tt> to the client, which then just goes and retries.
+ */
+public class TestFailWithoutRetries {
+
+  private static final Log LOG = LogFactory.getLog(TestFailWithoutRetries.class);
+  @Rule
+  public TableName table = new TableName();
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private String getIndexTableName() {
+    return Bytes.toString(table.getTableName()) + "_index";
+  }
+
+  public static class FailingTestCodec extends BaseIndexCodec {
+
+    @Override
+    public Iterable<IndexUpdate> getIndexDeletes(TableState state) throws IOException {
+      throw new RuntimeException("Intentionally failing deletes for "
+          + TestFailWithoutRetries.class.getName());
+    }
+
+    @Override
+    public Iterable<IndexUpdate> getIndexUpserts(TableState state) throws IOException {
+      throw new RuntimeException("Intentionally failing upserts for "
+          + TestFailWithoutRetries.class.getName());
+    }
+
+  }
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    // setup and verify the config
+    Configuration conf = UTIL.getConfiguration();
+    IndexTestingUtils.setupConfig(conf);
+    IndexManagementUtil.ensureMutableIndexingCorrectlyConfigured(conf);
+    // start the cluster
+    UTIL.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * If this test times out, then we didn't fail quickly enough. {@link Indexer} maybe isn't
+   * rethrowing the exception correctly?
+   * <p>
+   * We use a custom codec to enforce the thrown exception.
+   * @throws Exception
+   */
+  @Test(timeout = 300000)
+  public void testQuickFailure() throws Exception {
+    // incorrectly setup indexing for the primary table - target index table doesn't exist, which
+    // should quickly return to the client
+    byte[] family = Bytes.toBytes("family");
+    ColumnGroup fam1 = new ColumnGroup(getIndexTableName());
+    // values are [col1]
+    fam1.add(new CoveredColumn(family, CoveredColumn.ALL_QUALIFIERS));
+    CoveredColumnIndexSpecifierBuilder builder = new CoveredColumnIndexSpecifierBuilder();
+    // add the index family
+    builder.addIndexGroup(fam1);
+    // usually, we would create the index table here, but we don't for the sake of the test.
+
+    // setup the primary table
+    String primaryTable = Bytes.toString(table.getTableName());
+    HTableDescriptor pTable = new HTableDescriptor(primaryTable);
+    pTable.addFamily(new HColumnDescriptor(family));
+    // override the codec so we can use our test one
+    builder.build(pTable, FailingTestCodec.class);
+
+    // create the primary table
+    HBaseAdmin admin = UTIL.getHBaseAdmin();
+    admin.createTable(pTable);
+    Configuration conf = new Configuration(UTIL.getConfiguration());
+    // up the number of retries/wait time to make it obvious that we are failing with retries here
+    conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 20);
+    conf.setLong(HConstants.HBASE_CLIENT_PAUSE, 1000);
+    HTable primary = new HTable(conf, primaryTable);
+    primary.setAutoFlush(false, true);
+
+    // do a simple put that should be indexed
+    Put p = new Put(Bytes.toBytes("row"));
+    p.add(family, null, Bytes.toBytes("value"));
+    primary.put(p);
+    try {
+      primary.flushCommits();
+      fail("Shouldn't have gotten a successful write to the primary table");
+    } catch (RetriesExhaustedWithDetailsException e) {
+      LOG.info("Correclty got a failure of the put!");
+    }
+    primary.close();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestApplyAndFilterDeletesFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestApplyAndFilterDeletesFilter.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestApplyAndFilterDeletesFilter.java
new file mode 100644
index 0000000..d57f36a
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestApplyAndFilterDeletesFilter.java
@@ -0,0 +1,211 @@
+/*
+ * 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.phoenix.hbase.index.covered.filter;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+
+import org.apache.phoenix.hbase.index.covered.filter.ApplyAndFilterDeletesFilter;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * Test filter to ensure that it correctly handles KVs of different types correctly
+ */
+public class TestApplyAndFilterDeletesFilter {
+
+  private static final Set<ImmutableBytesPtr> EMPTY_SET = Collections
+      .<ImmutableBytesPtr> emptySet();
+  private byte[] row = Bytes.toBytes("row");
+  private byte[] family = Bytes.toBytes("family");
+  private byte[] qualifier = Bytes.toBytes("qualifier");
+  private byte[] value = Bytes.toBytes("value");
+  private long ts = 10;
+
+  @Test
+  public void testDeletesAreNotReturned() {
+    KeyValue kv = createKvForType(Type.Delete);
+    ApplyAndFilterDeletesFilter filter = new ApplyAndFilterDeletesFilter(EMPTY_SET);
+    assertEquals("Didn't skip point delete!", ReturnCode.SKIP, filter.filterKeyValue(kv));
+
+    filter.reset();
+    kv = createKvForType(Type.DeleteColumn);
+    assertEquals("Didn't skip from column delete!", ReturnCode.SKIP, filter.filterKeyValue(kv));
+
+    filter.reset();
+    kv = createKvForType(Type.DeleteFamily);
+    assertEquals("Didn't skip from family delete!", ReturnCode.SKIP, filter.filterKeyValue(kv));
+  }
+
+  /**
+   * Hinting with this filter is a little convoluted as we binary search the list of families to
+   * attempt to find the right one to seek.
+   */
+  @Test
+  public void testHintCorrectlyToNextFamily() {
+    // start with doing a family delete, so we will seek to the next column
+    KeyValue kv = createKvForType(Type.DeleteFamily);
+    ApplyAndFilterDeletesFilter filter = new ApplyAndFilterDeletesFilter(EMPTY_SET);
+    assertEquals(ReturnCode.SKIP, filter.filterKeyValue(kv));
+    KeyValue next = createKvForType(Type.Put);
+    // make sure the hint is our attempt at the end key, because we have no more families to seek
+    assertEquals("Didn't get a hint from a family delete", ReturnCode.SEEK_NEXT_USING_HINT,
+      filter.filterKeyValue(next));
+    assertEquals("Didn't get END_KEY with no families to match", KeyValue.LOWESTKEY,
+      filter.getNextKeyHint(next));
+
+    // check for a family that comes before our family, so we always seek to the end as well
+    filter = new ApplyAndFilterDeletesFilter(asSet(Bytes.toBytes("afamily")));
+    assertEquals(ReturnCode.SKIP, filter.filterKeyValue(kv));
+    // make sure the hint is our attempt at the end key, because we have no more families to seek
+    assertEquals("Didn't get a hint from a family delete", ReturnCode.SEEK_NEXT_USING_HINT,
+      filter.filterKeyValue(next));
+    assertEquals("Didn't get END_KEY with no families to match", KeyValue.LOWESTKEY,
+      filter.getNextKeyHint(next));
+
+    // check that we seek to the correct family that comes after our family
+    byte[] laterFamily = Bytes.toBytes("zfamily");
+    filter = new ApplyAndFilterDeletesFilter(asSet(laterFamily));
+    assertEquals(ReturnCode.SKIP, filter.filterKeyValue(kv));
+    KeyValue expected = KeyValue.createFirstOnRow(kv.getRow(), laterFamily, new byte[0]);
+    assertEquals("Didn't get a hint from a family delete", ReturnCode.SEEK_NEXT_USING_HINT,
+      filter.filterKeyValue(next));
+    assertEquals("Didn't get correct next key with a next family", expected,
+      filter.getNextKeyHint(next));
+  }
+
+  /**
+   * Point deletes should only cover the exact entry they are tied to. Earlier puts should always
+   * show up.
+   */
+  @Test
+  public void testCoveringPointDelete() {
+    // start with doing a family delete, so we will seek to the next column
+    KeyValue kv = createKvForType(Type.Delete);
+    ApplyAndFilterDeletesFilter filter = new ApplyAndFilterDeletesFilter(EMPTY_SET);
+    filter.filterKeyValue(kv);
+    KeyValue put = createKvForType(Type.Put);
+    assertEquals("Didn't filter out put with same timestamp!", ReturnCode.SKIP,
+      filter.filterKeyValue(put));
+    // we should filter out the exact same put again, which could occur with the kvs all kept in the
+    // same memstore
+    assertEquals("Didn't filter out put with same timestamp on second call!", ReturnCode.SKIP,
+      filter.filterKeyValue(put));
+
+    // ensure then that we don't filter out a put with an earlier timestamp (though everything else
+    // matches)
+    put = createKvForType(Type.Put, ts - 1);
+    assertEquals("Didn't accept put that has an earlier ts than the covering delete!",
+      ReturnCode.INCLUDE, filter.filterKeyValue(put));
+  }
+
+  private KeyValue createKvForType(Type t) {
+    return createKvForType(t, this.ts);
+  }
+
+  private KeyValue createKvForType(Type t, long timestamp) {
+    return new KeyValue(row, family, qualifier, 0, qualifier.length, timestamp, t, value, 0,
+        value.length);
+  }
+
+  /**
+   * Test that when we do a column delete at a given timestamp that we delete the entire column.
+   * @throws Exception
+   */
+  @Test
+  public void testCoverForDeleteColumn() throws Exception {
+    ApplyAndFilterDeletesFilter filter = new ApplyAndFilterDeletesFilter(EMPTY_SET);
+    KeyValue dc = createKvForType(Type.DeleteColumn, 11);
+    KeyValue put = createKvForType(Type.Put, 10);
+    assertEquals("Didn't filter out delete column.", ReturnCode.SKIP, filter.filterKeyValue(dc));
+    assertEquals("Didn't get a seek hint for the deleted column", ReturnCode.SEEK_NEXT_USING_HINT,
+      filter.filterKeyValue(put));
+    // seek past the given put
+    KeyValue seek = filter.getNextKeyHint(put);
+    assertTrue("Seeked key wasn't past the expected put - didn't skip the column",
+      KeyValue.COMPARATOR.compare(seek, put) > 0);
+  }
+
+  /**
+   * DeleteFamily markers should delete everything from that timestamp backwards, but not hide
+   * anything forwards
+   */
+  @Test
+  public void testDeleteFamilyCorrectlyCoversColumns() {
+    ApplyAndFilterDeletesFilter filter = new ApplyAndFilterDeletesFilter(EMPTY_SET);
+    KeyValue df = createKvForType(Type.DeleteFamily, 11);
+    KeyValue put = createKvForType(Type.Put, 12);
+
+    assertEquals("Didn't filter out delete family", ReturnCode.SKIP, filter.filterKeyValue(df));
+    assertEquals("Filtered out put with newer TS than delete family", ReturnCode.INCLUDE,
+      filter.filterKeyValue(put));
+
+    // older kv shouldn't be visible
+    put = createKvForType(Type.Put, 10);
+    assertEquals("Didn't filter out older put, covered by DeleteFamily marker",
+      ReturnCode.SEEK_NEXT_USING_HINT, filter.filterKeyValue(put));
+
+    // next seek should be past the families
+    assertEquals(KeyValue.LOWESTKEY, filter.getNextKeyHint(put));
+  }
+
+  /**
+   * Test that we don't cover other columns when we have a delete column.
+   */
+  @Test
+  public void testDeleteColumnCorrectlyCoversColumns() {
+    ApplyAndFilterDeletesFilter filter = new ApplyAndFilterDeletesFilter(EMPTY_SET);
+    KeyValue d = createKvForType(Type.DeleteColumn, 12);
+    byte[] qual2 = Bytes.add(qualifier, Bytes.toBytes("-other"));
+    KeyValue put =
+        new KeyValue(row, family, qual2, 0, qual2.length, 11, Type.Put, value, 0,
+            value.length);
+
+    assertEquals("Didn't filter out delete column", ReturnCode.SKIP, filter.filterKeyValue(d));
+    // different column put should still be visible
+    assertEquals("Filtered out put with different column than the delete", ReturnCode.INCLUDE,
+      filter.filterKeyValue(put));
+
+    // set a delete family, but in the past
+    d = createKvForType(Type.DeleteFamily, 10);
+    assertEquals("Didn't filter out delete column", ReturnCode.SKIP, filter.filterKeyValue(d));
+    // add back in the original delete column
+    d = createKvForType(Type.DeleteColumn, 11);
+    assertEquals("Didn't filter out delete column", ReturnCode.SKIP, filter.filterKeyValue(d));
+    // onto a different family, so that must be visible too
+    assertEquals("Filtered out put with different column than the delete", ReturnCode.INCLUDE,
+      filter.filterKeyValue(put));
+  }
+
+  private static Set<ImmutableBytesPtr> asSet(byte[]... strings) {
+    Set<ImmutableBytesPtr> set = new HashSet<ImmutableBytesPtr>();
+    for (byte[] s : strings) {
+      set.add(new ImmutableBytesPtr(s));
+    }
+    return set;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestFamilyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestFamilyOnlyFilter.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestFamilyOnlyFilter.java
new file mode 100644
index 0000000..216f548
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestFamilyOnlyFilter.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.hbase.index.covered.filter;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.hbase.index.covered.filter.FamilyOnlyFilter;
+import org.junit.Test;
+
+/**
+ * Test that the family only filter only allows a single family through
+ */
+public class TestFamilyOnlyFilter {
+
+  byte[] row = new byte[] { 'a' };
+  byte[] qual = new byte[] { 'b' };
+  byte[] val = Bytes.toBytes("val");
+
+  @Test
+  public void testPassesFirstFamily() {
+    byte[] fam = Bytes.toBytes("fam");
+    byte[] fam2 = Bytes.toBytes("fam2");
+
+    FamilyOnlyFilter filter = new FamilyOnlyFilter(fam);
+
+    KeyValue kv = new KeyValue(row, fam, qual, 10, val);
+    ReturnCode code = filter.filterKeyValue(kv);
+    assertEquals("Didn't pass matching family!", ReturnCode.INCLUDE, code);
+
+    kv = new KeyValue(row, fam2, qual, 10, val);
+    code = filter.filterKeyValue(kv);
+    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
+  }
+
+  @Test
+  public void testPassesTargetFamilyAsNonFirstFamily() {
+    byte[] fam = Bytes.toBytes("fam");
+    byte[] fam2 = Bytes.toBytes("fam2");
+    byte[] fam3 = Bytes.toBytes("way_after_family");
+
+    FamilyOnlyFilter filter = new FamilyOnlyFilter(fam2);
+
+    KeyValue kv = new KeyValue(row, fam, qual, 10, val);
+
+    ReturnCode code = filter.filterKeyValue(kv);
+    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
+
+    kv = new KeyValue(row, fam2, qual, 10, val);
+    code = filter.filterKeyValue(kv);
+    assertEquals("Didn't pass matching family", ReturnCode.INCLUDE, code);
+
+    kv = new KeyValue(row, fam3, qual, 10, val);
+    code = filter.filterKeyValue(kv);
+    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
+  }
+
+  @Test
+  public void testResetFilter() {
+    byte[] fam = Bytes.toBytes("fam");
+    byte[] fam2 = Bytes.toBytes("fam2");
+    byte[] fam3 = Bytes.toBytes("way_after_family");
+
+    FamilyOnlyFilter filter = new FamilyOnlyFilter(fam2);
+
+    KeyValue kv = new KeyValue(row, fam, qual, 10, val);
+
+    ReturnCode code = filter.filterKeyValue(kv);
+    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
+
+    KeyValue accept = new KeyValue(row, fam2, qual, 10, val);
+    code = filter.filterKeyValue(accept);
+    assertEquals("Didn't pass matching family", ReturnCode.INCLUDE, code);
+
+    kv = new KeyValue(row, fam3, qual, 10, val);
+    code = filter.filterKeyValue(kv);
+    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
+
+    // we shouldn't match the family again - everything after a switched family should be ignored
+    code = filter.filterKeyValue(accept);
+    assertEquals("Should have skipped a 'matching' family if it arrives out of order",
+      ReturnCode.SKIP, code);
+
+    // reset the filter and we should accept it again
+    filter.reset();
+    code = filter.filterKeyValue(accept);
+    assertEquals("Didn't pass matching family after reset", ReturnCode.INCLUDE, code);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestNewerTimestampFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestNewerTimestampFilter.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestNewerTimestampFilter.java
new file mode 100644
index 0000000..0677a38
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestNewerTimestampFilter.java
@@ -0,0 +1,48 @@
+/*
+ * 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.phoenix.hbase.index.covered.filter;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.hbase.index.covered.filter.NewerTimestampFilter;
+import org.junit.Test;
+
+public class TestNewerTimestampFilter {
+  byte[] row = new byte[] { 'a' };
+  byte[] fam = Bytes.toBytes("family");
+  byte[] qual = new byte[] { 'b' };
+  byte[] val = Bytes.toBytes("val");
+
+  @Test
+  public void testOnlyAllowsOlderTimestamps() {
+    long ts = 100;
+    NewerTimestampFilter filter = new NewerTimestampFilter(ts);
+
+    KeyValue kv = new KeyValue(row, fam, qual, ts, val);
+    assertEquals("Didn't accept kv with matching ts", ReturnCode.INCLUDE, filter.filterKeyValue(kv));
+
+    kv = new KeyValue(row, fam, qual, ts + 1, val);
+    assertEquals("Didn't skip kv with greater ts", ReturnCode.SKIP, filter.filterKeyValue(kv));
+
+    kv = new KeyValue(row, fam, qual, ts - 1, val);
+    assertEquals("Didn't accept kv with lower ts", ReturnCode.INCLUDE, filter.filterKeyValue(kv));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/update/TestIndexUpdateManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/update/TestIndexUpdateManager.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/update/TestIndexUpdateManager.java
new file mode 100644
index 0000000..a0592f3
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/update/TestIndexUpdateManager.java
@@ -0,0 +1,140 @@
+/*
+ * 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.phoenix.hbase.index.covered.update;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.Test;
+
+import org.apache.phoenix.hbase.index.covered.update.IndexUpdateManager;
+
+public class TestIndexUpdateManager {
+
+  private static final byte[] row = Bytes.toBytes("row");
+  private static final String TABLE_NAME = "table";
+  private static final byte[] table = Bytes.toBytes(TABLE_NAME);
+
+  @Test
+  public void testMutationComparator() throws Exception {
+    IndexUpdateManager manager = new IndexUpdateManager();
+    Comparator<Mutation> comparator = manager.COMPARATOR;
+    Put p = new Put(row, 10);
+    // lexigraphically earlier should sort earlier
+    Put p1 = new Put(Bytes.toBytes("ro"), 10);
+    assertTrue("lexigraphically later sorting first, should be earlier first.",
+      comparator.compare(p, p1) > 0);
+    p1 = new Put(Bytes.toBytes("row1"), 10);
+    assertTrue("lexigraphically later sorting first, should be earlier first.",
+      comparator.compare(p1, p) > 0);
+
+    // larger ts sorts before smaller, for the same row
+    p1 = new Put(row, 11);
+    assertTrue("Smaller timestamp sorting first, should be larger first.",
+      comparator.compare(p, p1) > 0);
+    // still true, even for deletes
+    Delete d = new Delete(row, 11);
+    assertTrue("Smaller timestamp sorting first, should be larger first.",
+      comparator.compare(p, d) > 0);
+
+    // for the same row, t1, the delete should sort earlier
+    d = new Delete(row, 10);
+    assertTrue("Delete doesn't sort before put, for the same row and ts",
+      comparator.compare(p, d) > 0);
+
+    // but for different rows, we still respect the row sorting.
+    d = new Delete(Bytes.toBytes("row1"), 10);
+    assertTrue("Delete doesn't sort before put, for the same row and ts",
+      comparator.compare(p, d) < 0);
+  }
+
+  /**
+   * When making updates we need to cancel out {@link Delete} and {@link Put}s for the same row.
+   * @throws Exception on failure
+   */
+  @Test
+  public void testCancelingUpdates() throws Exception {
+    IndexUpdateManager manager = new IndexUpdateManager();
+
+    long ts1 = 10, ts2 = 11;
+    // at different timestamps, so both should be retained
+    Delete d = new Delete(row, ts1);
+    Put p = new Put(row, ts2);
+    manager.addIndexUpdate(table, d);
+    manager.addIndexUpdate(table, p);
+    List<Mutation> pending = new ArrayList<Mutation>();
+    pending.add(p);
+    pending.add(d);
+    validate(manager, pending);
+
+    // add a delete that should cancel out the put, leading to only one delete remaining
+    Delete d2 = new Delete(row, ts2);
+    manager.addIndexUpdate(table, d2);
+    pending.add(d);
+    validate(manager, pending);
+
+    // double-deletes of the same row only retain the existing one, which was already canceled out
+    // above
+    Delete d3 = new Delete(row, ts2);
+    manager.addIndexUpdate(table, d3);
+    pending.add(d);
+    validate(manager, pending);
+
+    // if there is just a put and a delete at the same ts, no pending updates should be returned
+    manager = new IndexUpdateManager();
+    manager.addIndexUpdate(table, d2);
+    manager.addIndexUpdate(table, p);
+    validate(manager, Collections.<Mutation> emptyList());
+
+    // different row insertions can be tricky too, if you don't get the base cases right
+    manager = new IndexUpdateManager();
+    manager.addIndexUpdate(table, p);
+    // this row definitely sorts after the current row
+    byte[] row1 = Bytes.toBytes("row1");
+    Put p1 = new Put(row1, ts1);
+    manager.addIndexUpdate(table, p1);
+    // this delete should completely cover the given put and both should be removed
+    Delete d4 = new Delete(row1, ts1);
+    manager.addIndexUpdate(table, d4);
+    pending.clear();
+    pending.add(p);
+    validate(manager, pending);
+  }
+
+  private void validate(IndexUpdateManager manager, List<Mutation> pending) {
+    for (Pair<Mutation, byte[]> entry : manager.toMap()) {
+      assertEquals("Table name didn't match for stored entry!", table, entry.getSecond());
+      Mutation m = pending.remove(0);
+      // test with == to match the exact entries, Mutation.equals just checks the row
+      assertTrue(
+        "Didn't get the expected mutation! Expected: " + m + ", but got: " + entry.getFirst(),
+        m == entry.getFirst());
+    }
+    assertTrue("Missing pending updates: " + pending, pending.isEmpty());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/parallel/TestThreadPoolBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/parallel/TestThreadPoolBuilder.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/parallel/TestThreadPoolBuilder.java
new file mode 100644
index 0000000..5ff7b8b
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/parallel/TestThreadPoolBuilder.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.hbase.index.parallel;
+
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.phoenix.hbase.index.TableName;
+import org.apache.phoenix.hbase.index.parallel.ThreadPoolBuilder;
+
+public class TestThreadPoolBuilder {
+
+  @Rule
+  public TableName name = new TableName();
+
+  @Test
+  public void testCoreThreadTimeoutNonZero() {
+    Configuration conf = new Configuration(false);
+    String key = name.getTableNameString()+"-key";
+    ThreadPoolBuilder builder = new ThreadPoolBuilder(name.getTableNameString(), conf);
+    assertTrue("core threads not set, but failed return", builder.getKeepAliveTime() > 0);
+    // set an negative value
+    builder.setCoreTimeout(key, -1);
+    assertTrue("core threads not set, but failed return", builder.getKeepAliveTime() > 0);
+    // set a positive value
+    builder.setCoreTimeout(key, 1234);
+    assertEquals("core threads not set, but failed return", 1234, builder.getKeepAliveTime());
+    // set an empty value
+    builder.setCoreTimeout(key);
+    assertTrue("core threads not set, but failed return", builder.getKeepAliveTime() > 0);
+  }
+  
+  @Test
+  public void testMaxThreadsNonZero() {
+    Configuration conf = new Configuration(false);
+    String key = name.getTableNameString()+"-key";
+    ThreadPoolBuilder builder = new ThreadPoolBuilder(name.getTableNameString(), conf);
+    assertTrue("core threads not set, but failed return", builder.getMaxThreads() > 0);
+    // set an negative value
+    builder.setMaxThread(key, -1);
+    assertTrue("core threads not set, but failed return", builder.getMaxThreads() > 0);
+    // set a positive value
+    builder.setMaxThread(key, 1234);
+    assertEquals("core threads not set, but failed return", 1234, builder.getMaxThreads());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/parallel/TestThreadPoolManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/parallel/TestThreadPoolManager.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/parallel/TestThreadPoolManager.java
new file mode 100644
index 0000000..24c30ac
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/parallel/TestThreadPoolManager.java
@@ -0,0 +1,95 @@
+/*
+ * 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.phoenix.hbase.index.parallel;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.phoenix.hbase.index.TableName;
+import org.apache.phoenix.hbase.index.parallel.ThreadPoolBuilder;
+import org.apache.phoenix.hbase.index.parallel.ThreadPoolManager;
+
+public class TestThreadPoolManager {
+
+  @Rule
+  public TableName name = new TableName();
+
+  @Test
+  public void testShutdownGetsNewThreadPool() throws Exception{
+    Map<String, Object> cache = new HashMap<String, Object>();
+    ThreadPoolBuilder builder = new ThreadPoolBuilder(name.getTableNameString(), new Configuration(false));
+    ThreadPoolExecutor exec = ThreadPoolManager.getExecutor(builder, cache);
+    assertNotNull("Got a null exector from the pool!", exec);
+    //shutdown the pool and ensure that it actually shutdown
+    exec.shutdown();
+    ThreadPoolExecutor exec2 = ThreadPoolManager.getExecutor(builder, cache);
+    assertFalse("Got the same exectuor, even though the original shutdown", exec2 == exec);
+  }
+
+  @Test
+  public void testShutdownWithReferencesDoesNotStopExecutor() throws Exception {
+    Map<String, Object> cache = new HashMap<String, Object>();
+    ThreadPoolBuilder builder =
+        new ThreadPoolBuilder(name.getTableNameString(), new Configuration(false));
+    ThreadPoolExecutor exec = ThreadPoolManager.getExecutor(builder, cache);
+    assertNotNull("Got a null exector from the pool!", exec);
+    ThreadPoolExecutor exec2 = ThreadPoolManager.getExecutor(builder, cache);
+    assertTrue("Should have gotten the same executor", exec2 == exec);
+    exec.shutdown();
+    assertFalse("Executor is shutting down, even though we have a live reference!",
+      exec.isShutdown() || exec.isTerminating());
+    exec2.shutdown();
+    // wait 5 minutes for thread pool to shutdown
+    assertTrue("Executor is NOT shutting down, after releasing live reference!",
+      exec.awaitTermination(300, TimeUnit.SECONDS));
+  }
+
+  @Test
+  public void testGetExpectedExecutorForName() throws Exception {
+    Map<String, Object> cache = new HashMap<String, Object>();
+    ThreadPoolBuilder builder =
+        new ThreadPoolBuilder(name.getTableNameString(), new Configuration(false));
+    ThreadPoolExecutor exec = ThreadPoolManager.getExecutor(builder, cache);
+    assertNotNull("Got a null exector from the pool!", exec);
+    ThreadPoolExecutor exec2 = ThreadPoolManager.getExecutor(builder, cache);
+    assertTrue("Got a different exectuor, even though they have the same name", exec2 == exec);
+    builder = new ThreadPoolBuilder(name.getTableNameString(), new Configuration(false));
+    exec2 = ThreadPoolManager.getExecutor(builder, cache);
+    assertTrue(
+      "Got a different exectuor, even though they have the same name, but different confs",
+      exec2 == exec);
+
+    builder =
+        new ThreadPoolBuilder(name.getTableNameString() + "-some-other-pool", new Configuration(
+            false));
+    exec2 = ThreadPoolManager.getExecutor(builder, cache);
+    assertFalse(
+      "Got a different exectuor, even though they have the same name, but different confs",
+      exec2 == exec);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/util/TestIndexManagementUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/util/TestIndexManagementUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/util/TestIndexManagementUtil.java
new file mode 100644
index 0000000..ef46ff4
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/util/TestIndexManagementUtil.java
@@ -0,0 +1,67 @@
+/*
+ * 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.phoenix.hbase.index.util;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.regionserver.wal.IndexedHLogReader;
+import org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec;
+import org.apache.hadoop.hbase.regionserver.wal.WALEditCodec;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
+import org.junit.Test;
+
+public class TestIndexManagementUtil {
+
+  @Test
+  public void testUncompressedWal() throws Exception {
+    Configuration conf = new Configuration(false);
+    // works with WALEditcodec
+    conf.set(WALEditCodec.WAL_EDIT_CODEC_CLASS_KEY, IndexedWALEditCodec.class.getName());
+    IndexManagementUtil.ensureMutableIndexingCorrectlyConfigured(conf);
+    // clear the codec and set the wal reader
+    conf = new Configuration(false);
+    conf.set(IndexManagementUtil.HLOG_READER_IMPL_KEY, IndexedHLogReader.class.getName());
+    IndexManagementUtil.ensureMutableIndexingCorrectlyConfigured(conf);
+  }
+
+  /**
+   * Compressed WALs are supported when we have the WALEditCodec installed
+   * @throws Exception
+   */
+  @Test
+  public void testCompressedWALWithCodec() throws Exception {
+    Configuration conf = new Configuration(false);
+    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
+    // works with WALEditcodec
+    conf.set(WALEditCodec.WAL_EDIT_CODEC_CLASS_KEY, IndexedWALEditCodec.class.getName());
+    IndexManagementUtil.ensureMutableIndexingCorrectlyConfigured(conf);
+  }
+
+  /**
+   * We cannot support WAL Compression with the IndexedHLogReader
+   * @throws Exception
+   */
+  @Test(expected = IllegalStateException.class)
+  public void testCompressedWALWithHLogReader() throws Exception {
+    Configuration conf = new Configuration(false);
+    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
+    // works with WALEditcodec
+    conf.set(IndexManagementUtil.HLOG_READER_IMPL_KEY, IndexedHLogReader.class.getName());
+    IndexManagementUtil.ensureMutableIndexingCorrectlyConfigured(conf);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/FakeTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/FakeTableFactory.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/FakeTableFactory.java
new file mode 100644
index 0000000..2b6be18
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/FakeTableFactory.java
@@ -0,0 +1,51 @@
+/*
+ * 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.phoenix.hbase.index.write;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.client.HTableInterface;
+
+import org.apache.phoenix.hbase.index.table.HTableFactory;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * Simple table factory that just looks up the tables based on name. Useful for mocking up
+ * {@link HTableInterface}s without having to mock up the factory too.
+ */
+class FakeTableFactory implements HTableFactory {
+
+  boolean shutdown = false;
+  private Map<ImmutableBytesPtr, HTableInterface> tables;
+
+  public FakeTableFactory(Map<ImmutableBytesPtr, HTableInterface> tables) {
+    this.tables = tables;
+  }
+
+  @Override
+  public HTableInterface getTable(ImmutableBytesPtr tablename) throws IOException {
+    return this.tables.get(tablename);
+  }
+
+  @Override
+  public void shutdown() {
+    shutdown = true;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestCachingHTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestCachingHTableFactory.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestCachingHTableFactory.java
new file mode 100644
index 0000000..adf82f3
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestCachingHTableFactory.java
@@ -0,0 +1,58 @@
+/*
+ * 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.phoenix.hbase.index.write;
+
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.phoenix.hbase.index.table.CachingHTableFactory;
+import org.apache.phoenix.hbase.index.table.HTableFactory;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+
+public class TestCachingHTableFactory {
+
+  @Test
+  public void testCacheCorrectlyExpiresTable() throws Exception {
+    // setup the mocks for the tables we will request
+    HTableFactory delegate = Mockito.mock(HTableFactory.class);
+    ImmutableBytesPtr t1 = new ImmutableBytesPtr(Bytes.toBytes("t1"));
+    ImmutableBytesPtr t2 = new ImmutableBytesPtr(Bytes.toBytes("t2"));
+    ImmutableBytesPtr t3 = new ImmutableBytesPtr(Bytes.toBytes("t3"));
+    HTableInterface table1 = Mockito.mock(HTableInterface.class);
+    HTableInterface table2 = Mockito.mock(HTableInterface.class);
+    HTableInterface table3 = Mockito.mock(HTableInterface.class);
+    Mockito.when(delegate.getTable(t1)).thenReturn(table1);
+    Mockito.when(delegate.getTable(t2)).thenReturn(table2);
+    Mockito.when(delegate.getTable(t3)).thenReturn(table3);
+    
+    // setup our factory with a cache size of 2
+    CachingHTableFactory factory = new CachingHTableFactory(delegate, 2);
+    factory.getTable(t1);
+    factory.getTable(t2);
+    factory.getTable(t3);
+    // get the same table a second time, after it has gone out of cache
+    factory.getTable(t1);
+    
+    Mockito.verify(delegate, Mockito.times(2)).getTable(t1);
+    Mockito.verify(delegate, Mockito.times(1)).getTable(t2);
+    Mockito.verify(delegate, Mockito.times(1)).getTable(t3);
+    Mockito.verify(table1).close();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestIndexWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestIndexWriter.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestIndexWriter.java
new file mode 100644
index 0000000..56a9ab8
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestIndexWriter.java
@@ -0,0 +1,287 @@
+/*
+ * 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.phoenix.hbase.index.write;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.Rule;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import org.apache.phoenix.hbase.index.StubAbortable;
+import org.apache.phoenix.hbase.index.TableName;
+import org.apache.phoenix.hbase.index.exception.IndexWriteException;
+import org.apache.phoenix.hbase.index.exception.SingleIndexWriteFailureException;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.write.IndexWriter;
+import org.apache.phoenix.hbase.index.write.KillServerOnFailurePolicy;
+import org.apache.phoenix.hbase.index.write.ParallelWriterIndexCommitter;
+
+public class TestIndexWriter {
+  private static final Log LOG = LogFactory.getLog(TestIndexWriter.class);
+  @Rule
+  public TableName testName = new TableName();
+  private final byte[] row = Bytes.toBytes("row");
+
+  @Test
+  public void getDefaultWriter() throws Exception {
+    Configuration conf = new Configuration(false);
+    RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
+    Mockito.when(env.getConfiguration()).thenReturn(conf);
+    assertNotNull(IndexWriter.getCommitter(env));
+  }
+
+  @Test
+  public void getDefaultFailurePolicy() throws Exception {
+    Configuration conf = new Configuration(false);
+    RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
+    Mockito.when(env.getConfiguration()).thenReturn(conf);
+    assertNotNull(IndexWriter.getFailurePolicy(env));
+  }
+
+  /**
+   * With the move to using a pool of threads to write, we need to ensure that we still block until
+   * all index writes for a mutation/batch are completed.
+   * @throws Exception on failure
+   */
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testSynchronouslyCompletesAllWrites() throws Exception {
+    LOG.info("Starting " + testName.getTableNameString());
+    LOG.info("Current thread is interrupted: " + Thread.interrupted());
+    Abortable abort = new StubAbortable();
+    Stoppable stop = Mockito.mock(Stoppable.class);
+    ExecutorService exec = Executors.newFixedThreadPool(1);
+    Map<ImmutableBytesPtr, HTableInterface> tables = new HashMap<ImmutableBytesPtr, HTableInterface>();
+    FakeTableFactory factory = new FakeTableFactory(tables);
+
+    byte[] tableName = this.testName.getTableName();
+    Put m = new Put(row);
+    m.add(Bytes.toBytes("family"), Bytes.toBytes("qual"), null);
+    Collection<Pair<Mutation, byte[]>> indexUpdates = Arrays.asList(new Pair<Mutation, byte[]>(m,
+        tableName));
+
+    HTableInterface table = Mockito.mock(HTableInterface.class);
+    final boolean[] completed = new boolean[] { false };
+    Mockito.when(table.batch(Mockito.anyList())).thenAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        // just keep track that it was called
+        completed[0] = true;
+        return null;
+      }
+    });
+    Mockito.when(table.getTableName()).thenReturn(testName.getTableName());
+    // add the table to the set of tables, so its returned to the writer
+    tables.put(new ImmutableBytesPtr(tableName), table);
+
+    // setup the writer and failure policy
+    ParallelWriterIndexCommitter committer = new ParallelWriterIndexCommitter();
+    committer.setup(factory, exec, abort, stop, 2);
+    KillServerOnFailurePolicy policy = new KillServerOnFailurePolicy();
+    policy.setup(stop, abort);
+    IndexWriter writer = new IndexWriter(committer, policy);
+    writer.write(indexUpdates);
+    assertTrue("Writer returned before the table batch completed! Likely a race condition tripped",
+      completed[0]);
+    writer.stop(this.testName.getTableNameString() + " finished");
+    assertTrue("Factory didn't get shutdown after writer#stop!", factory.shutdown);
+    assertTrue("ExectorService isn't terminated after writer#stop!", exec.isShutdown());
+  }
+
+  /**
+   * Index updates can potentially be queued up if there aren't enough writer threads. If a running
+   * index write fails, then we should early exit the pending indexupdate, when it comes up (if the
+   * pool isn't already shutdown).
+   * <p>
+   * This test is a little bit racey - we could actually have the failure of the first task before
+   * the third task is even submitted. However, we should never see the third task attempt to make
+   * the batch write, so we should never see a failure here.
+   * @throws Exception on failure
+   */
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testFailureOnRunningUpdateAbortsPending() throws Exception {
+    Abortable abort = new StubAbortable();
+    Stoppable stop = Mockito.mock(Stoppable.class);
+    // single thread factory so the older request gets queued
+    ExecutorService exec = Executors.newFixedThreadPool(1);
+    Map<ImmutableBytesPtr, HTableInterface> tables = new HashMap<ImmutableBytesPtr, HTableInterface>();
+    FakeTableFactory factory = new FakeTableFactory(tables);
+
+    // updates to two different tables
+    byte[] tableName = Bytes.add(this.testName.getTableName(), new byte[] { 1, 2, 3, 4 });
+    Put m = new Put(row);
+    m.add(Bytes.toBytes("family"), Bytes.toBytes("qual"), null);
+    byte[] tableName2 = this.testName.getTableName();// this will sort after the first tablename
+    List<Pair<Mutation, byte[]>> indexUpdates = new ArrayList<Pair<Mutation, byte[]>>();
+    indexUpdates.add(new Pair<Mutation, byte[]>(m, tableName));
+    indexUpdates.add(new Pair<Mutation, byte[]>(m, tableName2));
+    indexUpdates.add(new Pair<Mutation, byte[]>(m, tableName2));
+
+    // first table will fail
+    HTableInterface table = Mockito.mock(HTableInterface.class);
+    Mockito.when(table.batch(Mockito.anyList())).thenThrow(
+      new IOException("Intentional IOException for failed first write."));
+    Mockito.when(table.getTableName()).thenReturn(tableName);
+
+    // second table just blocks to make sure that the abort propagates to the third task
+    final CountDownLatch waitOnAbortedLatch = new CountDownLatch(1);
+    final boolean[] failed = new boolean[] { false };
+    HTableInterface table2 = Mockito.mock(HTableInterface.class);
+    Mockito.when(table2.getTableName()).thenReturn(tableName2);
+    Mockito.when(table2.batch(Mockito.anyList())).thenAnswer(new Answer<Void>() {
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        waitOnAbortedLatch.await();
+        return null;
+      }
+    }).thenAnswer(new Answer<Void>() {
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        failed[0] = true;
+        throw new RuntimeException(
+            "Unexpected exception - second index table shouldn't have been written to");
+      }
+    });
+
+    // add the tables to the set of tables, so its returned to the writer
+    tables.put(new ImmutableBytesPtr(tableName), table);
+    tables.put(new ImmutableBytesPtr(tableName2), table2);
+
+    ParallelWriterIndexCommitter committer = new ParallelWriterIndexCommitter();
+    committer.setup(factory, exec, abort, stop, 2);
+    KillServerOnFailurePolicy policy = new KillServerOnFailurePolicy();
+    policy.setup(stop, abort);
+    IndexWriter writer = new IndexWriter(committer, policy);
+    try {
+      writer.write(indexUpdates);
+      fail("Should not have successfully completed all index writes");
+    } catch (SingleIndexWriteFailureException e) {
+      LOG.info("Correctly got a failure to reach the index", e);
+      // should have correctly gotten the correct abort, so let the next task execute
+      waitOnAbortedLatch.countDown();
+    }
+    assertFalse(
+      "Third set of index writes never have been attempted - should have seen the abort before done!",
+      failed[0]);
+    writer.stop(this.testName.getTableNameString() + " finished");
+    assertTrue("Factory didn't get shutdown after writer#stop!", factory.shutdown);
+    assertTrue("ExectorService isn't terminated after writer#stop!", exec.isShutdown());
+  }
+
+  /**
+   * Test that if we get an interruption to to the thread while doing a batch (e.g. via shutdown),
+   * that we correctly end the task
+   * @throws Exception on failure
+   */
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testShutdownInterruptsAsExpected() throws Exception {
+    Stoppable stop = Mockito.mock(Stoppable.class);
+    Abortable abort = new StubAbortable();
+    // single thread factory so the older request gets queued
+    ExecutorService exec = Executors.newFixedThreadPool(1);
+    Map<ImmutableBytesPtr, HTableInterface> tables = new HashMap<ImmutableBytesPtr, HTableInterface>();
+    FakeTableFactory factory = new FakeTableFactory(tables);
+
+    byte[] tableName = this.testName.getTableName();
+    HTableInterface table = Mockito.mock(HTableInterface.class);
+    Mockito.when(table.getTableName()).thenReturn(tableName);
+    final CountDownLatch writeStartedLatch = new CountDownLatch(1);
+    // latch never gets counted down, so we wait forever
+    final CountDownLatch waitOnAbortedLatch = new CountDownLatch(1);
+    Mockito.when(table.batch(Mockito.anyList())).thenAnswer(new Answer<Void>() {
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        LOG.info("Write started");
+        writeStartedLatch.countDown();
+        // when we interrupt the thread for shutdown, we should see this throw an interrupt too
+        try {
+        waitOnAbortedLatch.await();
+        } catch (InterruptedException e) {
+          LOG.info("Correctly interrupted while writing!");
+          throw e;
+        }
+        return null;
+      }
+    });
+    // add the tables to the set of tables, so its returned to the writer
+    tables.put(new ImmutableBytesPtr(tableName), table);
+
+    // update a single table
+    Put m = new Put(row);
+    m.add(Bytes.toBytes("family"), Bytes.toBytes("qual"), null);
+    final List<Pair<Mutation, byte[]>> indexUpdates = new ArrayList<Pair<Mutation, byte[]>>();
+    indexUpdates.add(new Pair<Mutation, byte[]>(m, tableName));
+
+    // setup the writer
+    ParallelWriterIndexCommitter committer = new ParallelWriterIndexCommitter();
+    committer.setup(factory, exec, abort, stop, 2);
+    KillServerOnFailurePolicy policy = new KillServerOnFailurePolicy();
+    policy.setup(stop, abort);
+    final IndexWriter writer = new IndexWriter(committer, policy);
+
+    final boolean[] failedWrite = new boolean[] { false };
+    Thread primaryWriter = new Thread() {
+
+      @Override
+      public void run() {
+        try {
+          writer.write(indexUpdates);
+        } catch (IndexWriteException e) {
+          failedWrite[0] = true;
+        }
+      }
+    };
+    primaryWriter.start();
+    // wait for the write to start before intentionally shutdown the pool
+    writeStartedLatch.await();
+    writer.stop("Shutting down writer for test " + this.testName.getTableNameString());
+    primaryWriter.join();
+    assertTrue("Writer should have failed because of the stop we issued", failedWrite[0]);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleIndexWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleIndexWriter.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleIndexWriter.java
new file mode 100644
index 0000000..b6331cd
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleIndexWriter.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.hbase.index.write;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Rule;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+
+import org.apache.phoenix.hbase.index.StubAbortable;
+import org.apache.phoenix.hbase.index.TableName;
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.write.ParallelWriterIndexCommitter;
+
+public class TestParalleIndexWriter {
+
+  private static final Log LOG = LogFactory.getLog(TestParalleIndexWriter.class);
+  @Rule
+  public TableName test = new TableName();
+  private final byte[] row = Bytes.toBytes("row");
+
+  @Test
+  public void testCorrectlyCleansUpResources() throws Exception{
+    ExecutorService exec = Executors.newFixedThreadPool(1);
+    FakeTableFactory factory = new FakeTableFactory(
+        Collections.<ImmutableBytesPtr, HTableInterface> emptyMap());
+    ParallelWriterIndexCommitter writer = new ParallelWriterIndexCommitter();
+    Abortable mockAbort = Mockito.mock(Abortable.class);
+    Stoppable mockStop = Mockito.mock(Stoppable.class);
+    // create a simple writer
+    writer.setup(factory, exec, mockAbort, mockStop, 1);
+    // stop the writer
+    writer.stop(this.test.getTableNameString() + " finished");
+    assertTrue("Factory didn't get shutdown after writer#stop!", factory.shutdown);
+    assertTrue("ExectorService isn't terminated after writer#stop!", exec.isShutdown());
+    Mockito.verifyZeroInteractions(mockAbort, mockStop);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testSynchronouslyCompletesAllWrites() throws Exception {
+    LOG.info("Starting " + test.getTableNameString());
+    LOG.info("Current thread is interrupted: " + Thread.interrupted());
+    Abortable abort = new StubAbortable();
+    Stoppable stop = Mockito.mock(Stoppable.class);
+    ExecutorService exec = Executors.newFixedThreadPool(1);
+    Map<ImmutableBytesPtr, HTableInterface> tables =
+        new HashMap<ImmutableBytesPtr, HTableInterface>();
+    FakeTableFactory factory = new FakeTableFactory(tables);
+
+    ImmutableBytesPtr tableName = new ImmutableBytesPtr(this.test.getTableName());
+    Put m = new Put(row);
+    m.add(Bytes.toBytes("family"), Bytes.toBytes("qual"), null);
+    Multimap<HTableInterfaceReference, Mutation> indexUpdates =
+        ArrayListMultimap.<HTableInterfaceReference, Mutation> create();
+    indexUpdates.put(new HTableInterfaceReference(tableName), m);
+
+    HTableInterface table = Mockito.mock(HTableInterface.class);
+    final boolean[] completed = new boolean[] { false };
+    Mockito.when(table.batch(Mockito.anyList())).thenAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        // just keep track that it was called
+        completed[0] = true;
+        return null;
+      }
+    });
+    Mockito.when(table.getTableName()).thenReturn(test.getTableName());
+    // add the table to the set of tables, so its returned to the writer
+    tables.put(tableName, table);
+
+    // setup the writer and failure policy
+    ParallelWriterIndexCommitter writer = new ParallelWriterIndexCommitter();
+    writer.setup(factory, exec, abort, stop, 1);
+    writer.write(indexUpdates);
+    assertTrue("Writer returned before the table batch completed! Likely a race condition tripped",
+      completed[0]);
+    writer.stop(this.test.getTableNameString() + " finished");
+    assertTrue("Factory didn't get shutdown after writer#stop!", factory.shutdown);
+    assertTrue("ExectorService isn't terminated after writer#stop!", exec.isShutdown());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleWriterIndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleWriterIndexCommitter.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleWriterIndexCommitter.java
new file mode 100644
index 0000000..7a09d4a
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestParalleWriterIndexCommitter.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.hbase.index.write;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Rule;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+
+import org.apache.phoenix.hbase.index.StubAbortable;
+import org.apache.phoenix.hbase.index.TableName;
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.write.ParallelWriterIndexCommitter;
+
+public class TestParalleWriterIndexCommitter {
+
+  private static final Log LOG = LogFactory.getLog(TestParalleWriterIndexCommitter.class);
+  @Rule
+  public TableName test = new TableName();
+  private final byte[] row = Bytes.toBytes("row");
+
+  @Test
+  public void testCorrectlyCleansUpResources() throws Exception{
+    ExecutorService exec = Executors.newFixedThreadPool(1);
+    FakeTableFactory factory = new FakeTableFactory(
+        Collections.<ImmutableBytesPtr, HTableInterface> emptyMap());
+    ParallelWriterIndexCommitter writer = new ParallelWriterIndexCommitter();
+    Abortable mockAbort = Mockito.mock(Abortable.class);
+    Stoppable mockStop = Mockito.mock(Stoppable.class);
+    // create a simple writer
+    writer.setup(factory, exec, mockAbort, mockStop, 1);
+    // stop the writer
+    writer.stop(this.test.getTableNameString() + " finished");
+    assertTrue("Factory didn't get shutdown after writer#stop!", factory.shutdown);
+    assertTrue("ExectorService isn't terminated after writer#stop!", exec.isShutdown());
+    Mockito.verifyZeroInteractions(mockAbort, mockStop);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testSynchronouslyCompletesAllWrites() throws Exception {
+    LOG.info("Starting " + test.getTableNameString());
+    LOG.info("Current thread is interrupted: " + Thread.interrupted());
+    Abortable abort = new StubAbortable();
+    Stoppable stop = Mockito.mock(Stoppable.class);
+    ExecutorService exec = Executors.newFixedThreadPool(1);
+    Map<ImmutableBytesPtr, HTableInterface> tables =
+        new HashMap<ImmutableBytesPtr, HTableInterface>();
+    FakeTableFactory factory = new FakeTableFactory(tables);
+
+    ImmutableBytesPtr tableName = new ImmutableBytesPtr(this.test.getTableName());
+    Put m = new Put(row);
+    m.add(Bytes.toBytes("family"), Bytes.toBytes("qual"), null);
+    Multimap<HTableInterfaceReference, Mutation> indexUpdates =
+        ArrayListMultimap.<HTableInterfaceReference, Mutation> create();
+    indexUpdates.put(new HTableInterfaceReference(tableName), m);
+
+    HTableInterface table = Mockito.mock(HTableInterface.class);
+    final boolean[] completed = new boolean[] { false };
+    Mockito.when(table.batch(Mockito.anyList())).thenAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        // just keep track that it was called
+        completed[0] = true;
+        return null;
+      }
+    });
+    Mockito.when(table.getTableName()).thenReturn(test.getTableName());
+    // add the table to the set of tables, so its returned to the writer
+    tables.put(tableName, table);
+
+    // setup the writer and failure policy
+    ParallelWriterIndexCommitter writer = new ParallelWriterIndexCommitter();
+    writer.setup(factory, exec, abort, stop, 1);
+    writer.write(indexUpdates);
+    assertTrue("Writer returned before the table batch completed! Likely a race condition tripped",
+      completed[0]);
+    writer.stop(this.test.getTableNameString() + " finished");
+    assertTrue("Factory didn't get shutdown after writer#stop!", factory.shutdown);
+    assertTrue("ExectorService isn't terminated after writer#stop!", exec.isShutdown());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java
new file mode 100644
index 0000000..4d886e5
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java
@@ -0,0 +1,369 @@
+/*
+ * 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.phoenix.hbase.index.write;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+
+import com.google.common.collect.Multimap;
+
+import org.apache.phoenix.hbase.index.IndexTestingUtils;
+import org.apache.phoenix.hbase.index.Indexer;
+import org.apache.phoenix.hbase.index.TableName;
+import org.apache.phoenix.hbase.index.covered.example.ColumnGroup;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumn;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumnIndexSpecifierBuilder;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumnIndexer;
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
+import org.apache.phoenix.hbase.index.write.recovery.PerRegionIndexWriteCache;
+import org.apache.phoenix.hbase.index.write.recovery.StoreFailuresInCachePolicy;
+
+/**
+ * When a regionserver crashes, its WAL is split and then replayed to the server. If the index
+ * region was present on the same server, we have to make a best effort to not kill the server for
+ * not succeeding on index writes while the index region is coming up.
+ */
+public class TestWALRecoveryCaching {
+
+  private static final Log LOG = LogFactory.getLog(TestWALRecoveryCaching.class);
+  private static final long ONE_SEC = 1000;
+  private static final long ONE_MIN = 60 * ONE_SEC;
+  private static final long TIMEOUT = ONE_MIN;
+
+  @Rule
+  public TableName testTable = new TableName();
+
+  private String getIndexTableName() {
+    return this.testTable.getTableNameString() + "_index";
+  }
+
+  // -----------------------------------------------------------------------------------------------
+  // Warning! The classes here rely on this static. Adding multiple tests to this class and running
+  // them concurrently could have unexpected results (including, but not limited to, odd failures
+  // and flapping tests).
+  // -----------------------------------------------------------------------------------------------
+  private static CountDownLatch allowIndexTableToRecover;
+
+  public static class IndexTableBlockingReplayObserver extends BaseRegionObserver {
+
+    @Override
+    public void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info,
+        HLogKey logKey, WALEdit logEdit) throws IOException {
+      try {
+        LOG.debug("Restoring logs for index table");
+        if (allowIndexTableToRecover != null) {
+          allowIndexTableToRecover.await();
+          LOG.debug("Completed index table recovery wait latch");
+        }
+      } catch (InterruptedException e) {
+        Assert.fail("Should not be interrupted while waiting to allow the index to restore WALs.");
+      }
+    }
+  }
+
+  public static class ReleaseLatchOnFailurePolicy extends StoreFailuresInCachePolicy {
+
+    /**
+     * @param failedIndexEdits
+     */
+    public ReleaseLatchOnFailurePolicy(PerRegionIndexWriteCache failedIndexEdits) {
+      super(failedIndexEdits);
+    }
+
+    @Override
+    public void handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted,
+        Exception cause) throws IOException {
+      LOG.debug("Found index update failure!");
+      if (allowIndexTableToRecover != null) {
+        LOG.info("failed index write on WAL recovery - allowing index table to be restored.");
+        allowIndexTableToRecover.countDown();
+      }
+      super.handleFailure(attempted, cause);
+    }
+
+  }
+
+  //TODO: Jesse to fix
+  @Ignore("Configuration issue - valid test, just needs fixing")
+  @Test
+  public void testWaitsOnIndexRegionToReload() throws Exception {
+    HBaseTestingUtility util = new HBaseTestingUtility();
+    Configuration conf = util.getConfiguration();
+
+    // setup other useful stats
+    IndexTestingUtils.setupConfig(conf);
+    conf.setBoolean(Indexer.CHECK_VERSION_CONF_KEY, false);
+
+    // make sure everything is setup correctly
+    IndexManagementUtil.ensureMutableIndexingCorrectlyConfigured(conf);
+
+    // start the cluster with 2 rs
+    util.startMiniCluster(2);
+
+    HBaseAdmin admin = util.getHBaseAdmin();
+    // setup the index
+    byte[] family = Bytes.toBytes("family");
+    byte[] qual = Bytes.toBytes("qualifier");
+    byte[] nonIndexedFamily = Bytes.toBytes("nonIndexedFamily");
+    String indexedTableName = getIndexTableName();
+    ColumnGroup columns = new ColumnGroup(indexedTableName);
+    columns.add(new CoveredColumn(family, qual));
+    CoveredColumnIndexSpecifierBuilder builder = new CoveredColumnIndexSpecifierBuilder();
+    builder.addIndexGroup(columns);
+
+    // create the primary table w/ indexing enabled
+    HTableDescriptor primaryTable = new HTableDescriptor(testTable.getTableName());
+    primaryTable.addFamily(new HColumnDescriptor(family));
+    primaryTable.addFamily(new HColumnDescriptor(nonIndexedFamily));
+    builder.addArbitraryConfigForTesting(Indexer.RecoveryFailurePolicyKeyForTesting,
+      ReleaseLatchOnFailurePolicy.class.getName());
+    builder.build(primaryTable);
+    admin.createTable(primaryTable);
+
+    // create the index table
+    HTableDescriptor indexTableDesc = new HTableDescriptor(Bytes.toBytes(getIndexTableName()));
+    indexTableDesc.addCoprocessor(IndexTableBlockingReplayObserver.class.getName());
+    CoveredColumnIndexer.createIndexTable(admin, indexTableDesc);
+
+    // figure out where our tables live
+    ServerName shared =
+        ensureTablesLiveOnSameServer(util.getMiniHBaseCluster(), Bytes.toBytes(indexedTableName),
+          testTable.getTableName());
+
+    // load some data into the table
+    Put p = new Put(Bytes.toBytes("row"));
+    p.add(family, qual, Bytes.toBytes("value"));
+    HTable primary = new HTable(conf, testTable.getTableName());
+    primary.put(p);
+    primary.flushCommits();
+
+    // turn on the recovery latch
+    allowIndexTableToRecover = new CountDownLatch(1);
+
+    // kill the server where the tables live - this should trigger distributed log splitting
+    // find the regionserver that matches the passed server
+    List<HRegion> online = new ArrayList<HRegion>();
+    online.addAll(getRegionsFromServerForTable(util.getMiniHBaseCluster(), shared,
+      testTable.getTableName()));
+    online.addAll(getRegionsFromServerForTable(util.getMiniHBaseCluster(), shared,
+      Bytes.toBytes(indexedTableName)));
+
+    // log all the current state of the server
+    LOG.info("Current Server/Region paring: ");
+    for (RegionServerThread t : util.getMiniHBaseCluster().getRegionServerThreads()) {
+      // check all the conditions for the server to be done
+      HRegionServer server = t.getRegionServer();
+      if (server.isStopping() || server.isStopped() || server.isAborted()) {
+        LOG.info("\t== Offline: " + server.getServerName());
+        continue;
+      }
+      List<HRegionInfo> regions = server.getOnlineRegions();
+      LOG.info("\t" + server.getServerName() + " regions: " + regions);
+    }
+
+    LOG.debug("Killing server " + shared);
+    util.getMiniHBaseCluster().killRegionServer(shared);
+    LOG.debug("Waiting on server " + shared + "to die");
+    util.getMiniHBaseCluster().waitForRegionServerToStop(shared, TIMEOUT);
+    // force reassign the regions from the table
+    // LOG.debug("Forcing region reassignment from the killed server: " + shared);
+    // for (HRegion region : online) {
+    // util.getMiniHBaseCluster().getMaster().assign(region.getRegionName());
+    // }
+    System.out.println(" ====== Killed shared server ==== ");
+
+    // make a second put that (1), isn't indexed, so we can be sure of the index state and (2)
+    // ensures that our table is back up
+    Put p2 = new Put(p.getRow());
+    p2.add(nonIndexedFamily, Bytes.toBytes("Not indexed"), Bytes.toBytes("non-indexed value"));
+    primary.put(p2);
+    primary.flushCommits();
+
+    // make sure that we actually failed the write once (within a 5 minute window)
+    assertTrue("Didn't find an error writing to index table within timeout!",
+      allowIndexTableToRecover.await(ONE_MIN * 5, TimeUnit.MILLISECONDS));
+
+    // scan the index to make sure it has the one entry, (that had to be replayed from the WAL,
+    // since we hard killed the server)
+    Scan s = new Scan();
+    HTable index = new HTable(conf, getIndexTableName());
+    ResultScanner scanner = index.getScanner(s);
+    int count = 0;
+    for (Result r : scanner) {
+      LOG.info("Got index table result:" + r);
+      count++;
+    }
+    assertEquals("Got an unexpected found of index rows", 1, count);
+
+    // cleanup
+    scanner.close();
+    index.close();
+    primary.close();
+    util.shutdownMiniCluster();
+  }
+
+  /**
+   * @param miniHBaseCluster
+   * @param server
+   * @param bs
+   * @return
+   */
+  private List<HRegion> getRegionsFromServerForTable(MiniHBaseCluster cluster, ServerName server,
+      byte[] table) {
+    List<HRegion> online = Collections.emptyList();
+    for (RegionServerThread rst : cluster.getRegionServerThreads()) {
+      // if its the server we are going to kill, get the regions we want to reassign
+      if (rst.getRegionServer().getServerName().equals(server)) {
+        online = rst.getRegionServer().getOnlineRegions(table);
+        break;
+      }
+    }
+    return online;
+  }
+
+  /**
+   * @param miniHBaseCluster
+   * @param indexedTableName
+   * @param tableNameString
+   */
+  private ServerName ensureTablesLiveOnSameServer(MiniHBaseCluster cluster, byte[] indexTable,
+      byte[] primaryTable) throws Exception {
+
+    ServerName shared = getSharedServer(cluster, indexTable, primaryTable);
+    boolean tryIndex = true;
+    while (shared == null) {
+
+      // start killing servers until we get an overlap
+      Set<ServerName> servers;
+      byte[] table = null;
+      // switch which server we kill each time to get region movement
+      if (tryIndex) {
+        table = indexTable;
+      } else {
+        table = primaryTable;
+      }
+      servers = getServersForTable(cluster, table);
+      tryIndex = !tryIndex;
+      for (ServerName server : servers) {
+        // find the regionserver that matches the passed server
+        List<HRegion> online = getRegionsFromServerForTable(cluster, server, table);
+
+        LOG.info("Shutting down and reassigning regions from " + server);
+        cluster.stopRegionServer(server);
+        cluster.waitForRegionServerToStop(server, TIMEOUT);
+
+        // force reassign the regions from the table
+        for (HRegion region : online) {
+          cluster.getMaster().assign(region.getRegionName());
+        }
+
+        LOG.info("Starting region server:" + server.getHostname());
+        cluster.startRegionServer(server.getHostname());
+
+        cluster.waitForRegionServerToStart(server.getHostname(), TIMEOUT);
+
+        // start a server to get back to the base number of servers
+        LOG.info("STarting server to replace " + server);
+        cluster.startRegionServer();
+        break;
+      }
+
+      shared = getSharedServer(cluster, indexTable, primaryTable);
+    }
+    return shared;
+  }
+
+  /**
+   * @param cluster
+   * @param indexTable
+   * @param primaryTable
+   * @return
+   * @throws Exception
+   */
+  private ServerName getSharedServer(MiniHBaseCluster cluster, byte[] indexTable,
+      byte[] primaryTable) throws Exception {
+    Set<ServerName> indexServers = getServersForTable(cluster, indexTable);
+    Set<ServerName> primaryServers = getServersForTable(cluster, primaryTable);
+
+    Set<ServerName> joinSet = new HashSet<ServerName>(indexServers);
+    joinSet.addAll(primaryServers);
+    // if there is already an overlap, then find it and return it
+    if (joinSet.size() < indexServers.size() + primaryServers.size()) {
+      // find the first overlapping server
+      for (ServerName server : joinSet) {
+        if (indexServers.contains(server) && primaryServers.contains(server)) {
+          return server;
+        }
+      }
+      throw new RuntimeException(
+          "Couldn't find a matching server on which both the primary and index table live, "
+              + "even though they have overlapping server sets");
+    }
+    return null;
+  }
+
+  private Set<ServerName> getServersForTable(MiniHBaseCluster cluster, byte[] table)
+      throws Exception {
+    List<HRegion> indexRegions = cluster.getRegions(table);
+    Set<ServerName> indexServers = new HashSet<ServerName>();
+    for (HRegion region : indexRegions) {
+      indexServers.add(cluster.getServerHoldingRegion(region.getRegionName()));
+    }
+    return indexServers;
+  }
+}
\ No newline at end of file


[06/15] Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/PerRegionIndexWriteCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/PerRegionIndexWriteCache.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/PerRegionIndexWriteCache.java
new file mode 100644
index 0000000..4d5f667
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/PerRegionIndexWriteCache.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.hbase.index.write.recovery;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+
+
+public class PerRegionIndexWriteCache {
+
+  private Map<HRegion, Multimap<HTableInterfaceReference, Mutation>> cache =
+      new HashMap<HRegion, Multimap<HTableInterfaceReference, Mutation>>();
+
+
+  /**
+   * Get the edits for the current region. Removes the edits from the cache. To add them back, call
+   * {@link #addEdits(HRegion, HTableInterfaceReference, Collection)}.
+   * @param region
+   * @return Get the edits for the given region. Returns <tt>null</tt> if there are no pending edits
+   *         for the region
+   */
+  public Multimap<HTableInterfaceReference, Mutation> getEdits(HRegion region) {
+    return cache.remove(region);
+  }
+
+  /**
+   * @param region
+   * @param table
+   * @param collection
+   */
+  public void addEdits(HRegion region, HTableInterfaceReference table,
+      Collection<Mutation> collection) {
+    Multimap<HTableInterfaceReference, Mutation> edits = cache.get(region);
+    if (edits == null) {
+      edits = ArrayListMultimap.<HTableInterfaceReference, Mutation> create();
+      cache.put(region, edits);
+    }
+    edits.putAll(table, collection);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/StoreFailuresInCachePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/StoreFailuresInCachePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/StoreFailuresInCachePolicy.java
new file mode 100644
index 0000000..f36affb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/StoreFailuresInCachePolicy.java
@@ -0,0 +1,84 @@
+/**
+ * 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.phoenix.hbase.index.write.recovery;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+
+import com.google.common.collect.Multimap;
+import org.apache.phoenix.hbase.index.exception.MultiIndexWriteFailureException;
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+import org.apache.phoenix.hbase.index.write.IndexFailurePolicy;
+import org.apache.phoenix.hbase.index.write.KillServerOnFailurePolicy;
+
+/**
+ * Tracks any failed writes in The {@link PerRegionIndexWriteCache}, given a
+ * {@link MultiIndexWriteFailureException} (which is thrown from the
+ * {@link TrackingParallelWriterIndexCommitter}. Any other exception failure causes the a server
+ * abort via the usual {@link KillServerOnFailurePolicy}.
+ */
+public class StoreFailuresInCachePolicy implements IndexFailurePolicy {
+
+  private KillServerOnFailurePolicy delegate;
+  private PerRegionIndexWriteCache cache;
+  private HRegion region;
+
+  /**
+   * @param failedIndexEdits cache to update when we find a failure
+   */
+  public StoreFailuresInCachePolicy(PerRegionIndexWriteCache failedIndexEdits) {
+    this.cache = failedIndexEdits;
+  }
+
+  @Override
+  public void setup(Stoppable parent, RegionCoprocessorEnvironment env) {
+    this.region = env.getRegion();
+    this.delegate = new KillServerOnFailurePolicy();
+    this.delegate.setup(parent, env);
+
+  }
+
+  @Override
+  public void handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause) throws IOException {
+    // if its not an exception we can handle, let the delegate take care of it
+    if (!(cause instanceof MultiIndexWriteFailureException)) {
+      delegate.handleFailure(attempted, cause);
+    }
+    List<HTableInterfaceReference> failedTables =
+        ((MultiIndexWriteFailureException) cause).getFailedTables();
+    for (HTableInterfaceReference table : failedTables) {
+      cache.addEdits(this.region, table, attempted.get(table));
+    }
+  }
+
+
+  @Override
+  public void stop(String why) {
+    this.delegate.stop(why);
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.delegate.isStopped();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
new file mode 100644
index 0000000..2517b8f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
@@ -0,0 +1,226 @@
+/*
+ * 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.phoenix.hbase.index.write.recovery;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import com.google.common.collect.Multimap;
+import org.apache.phoenix.hbase.index.CapturingAbortable;
+import org.apache.phoenix.hbase.index.exception.MultiIndexWriteFailureException;
+import org.apache.phoenix.hbase.index.exception.SingleIndexWriteFailureException;
+import org.apache.phoenix.hbase.index.parallel.EarlyExitFailure;
+import org.apache.phoenix.hbase.index.parallel.Task;
+import org.apache.phoenix.hbase.index.parallel.TaskBatch;
+import org.apache.phoenix.hbase.index.parallel.TaskRunner;
+import org.apache.phoenix.hbase.index.parallel.ThreadPoolBuilder;
+import org.apache.phoenix.hbase.index.parallel.ThreadPoolManager;
+import org.apache.phoenix.hbase.index.parallel.WaitForCompletionTaskRunner;
+import org.apache.phoenix.hbase.index.table.CachingHTableFactory;
+import org.apache.phoenix.hbase.index.table.HTableFactory;
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+import org.apache.phoenix.hbase.index.write.IndexCommitter;
+import org.apache.phoenix.hbase.index.write.IndexWriter;
+import org.apache.phoenix.hbase.index.write.IndexWriterUtils;
+import org.apache.phoenix.hbase.index.write.ParallelWriterIndexCommitter;
+
+/**
+ * Like the {@link ParallelWriterIndexCommitter}, but blocks until all writes have attempted to
+ * allow the caller to retrieve the failed and succeeded index updates. Therefore, this class will
+ * be a lot slower, in the face of failures, when compared to the
+ * {@link ParallelWriterIndexCommitter} (though as fast for writes), so it should be used only when
+ * you need to at least attempt all writes and know their result; for instance, this is fine for
+ * doing WAL recovery - it's not a performance intensive situation and we want to limit the the
+ * edits we need to retry.
+ * <p>
+ * On failure to {@link #write(Multimap)}, we return a {@link MultiIndexWriteFailureException} that
+ * contains the list of {@link HTableInterfaceReference} that didn't complete successfully.
+ * <p>
+ * Failures to write to the index can happen several different ways:
+ * <ol>
+ * <li><tt>this</tt> is {@link #stop(String) stopped} or aborted (via the passed {@link Abortable}.
+ * This causing any pending tasks to fail whatever they are doing as fast as possible. Any writes
+ * that have not begun are not even attempted and marked as failures.</li>
+ * <li>A batch write fails. This is the generic HBase write failure - it may occur because the index
+ * table is not available, .META. or -ROOT- is unavailable, or any other (of many) possible HBase
+ * exceptions.</li>
+ * </ol>
+ * Regardless of how the write fails, we still wait for all writes to complete before passing the
+ * failure back to the client.
+ */
+public class TrackingParallelWriterIndexCommitter implements IndexCommitter {
+  private static final Log LOG = LogFactory.getLog(TrackingParallelWriterIndexCommitter.class);
+
+  public static final String NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY = "index.trackingwriter.threads.max";
+  private static final int DEFAULT_CONCURRENT_INDEX_WRITER_THREADS = 10;
+  private static final String INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY =
+      "index.trackingwriter.threads.keepalivetime";
+  
+  private TaskRunner pool;
+  private HTableFactory factory;
+  private CapturingAbortable abortable;
+  private Stoppable stopped;
+
+  @Override
+  public void setup(IndexWriter parent, RegionCoprocessorEnvironment env, String name) {
+    Configuration conf = env.getConfiguration();
+    setup(IndexWriterUtils.getDefaultDelegateHTableFactory(env),
+      ThreadPoolManager.getExecutor(
+        new ThreadPoolBuilder(name, conf).
+          setMaxThread(NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY,
+            DEFAULT_CONCURRENT_INDEX_WRITER_THREADS).
+          setCoreTimeout(INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY), env),
+      env.getRegionServerServices(), parent, CachingHTableFactory.getCacheSize(conf));
+  }
+
+  /**
+   * Setup <tt>this</tt>.
+   * <p>
+   * Exposed for TESTING
+   */
+  void setup(HTableFactory factory, ExecutorService pool, Abortable abortable, Stoppable stop,
+      int cacheSize) {
+    this.pool = new WaitForCompletionTaskRunner(pool);
+    this.factory = new CachingHTableFactory(factory, cacheSize);
+    this.abortable = new CapturingAbortable(abortable);
+    this.stopped = stop;
+  }
+
+  @Override
+  public void write(Multimap<HTableInterfaceReference, Mutation> toWrite)
+      throws MultiIndexWriteFailureException {
+    Set<Entry<HTableInterfaceReference, Collection<Mutation>>> entries = toWrite.asMap().entrySet();
+    TaskBatch<Boolean> tasks = new TaskBatch<Boolean>(entries.size());
+    List<HTableInterfaceReference> tables = new ArrayList<HTableInterfaceReference>(entries.size());
+    for (Entry<HTableInterfaceReference, Collection<Mutation>> entry : entries) {
+      // get the mutations for each table. We leak the implementation here a little bit to save
+      // doing a complete copy over of all the index update for each table.
+      final List<Mutation> mutations = (List<Mutation>) entry.getValue();
+      // track each reference so we can get at it easily later, when determing failures
+      final HTableInterfaceReference tableReference = entry.getKey();
+      tables.add(tableReference);
+
+      /*
+       * Write a batch of index updates to an index table. This operation stops (is cancelable) via
+       * two mechanisms: (1) setting aborted or stopped on the IndexWriter or, (2) interrupting the
+       * running thread. The former will only work if we are not in the midst of writing the current
+       * batch to the table, though we do check these status variables before starting and before
+       * writing the batch. The latter usage, interrupting the thread, will work in the previous
+       * situations as was at some points while writing the batch, depending on the underlying
+       * writer implementation (HTableInterface#batch is blocking, but doesn't elaborate when is
+       * supports an interrupt).
+       */
+      tasks.add(new Task<Boolean>() {
+
+        /**
+         * Do the actual write to the primary table. We don't need to worry about closing the table
+         * because that is handled the {@link CachingHTableFactory}.
+         */
+        @Override
+        public Boolean call() throws Exception {
+          try {
+            // this may have been queued, but there was an abort/stop so we try to early exit
+            throwFailureIfDone();
+
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Writing index update:" + mutations + " to table: " + tableReference);
+            }
+            HTableInterface table = factory.getTable(tableReference.get());
+            throwFailureIfDone();
+            table.batch(mutations);
+          } catch (InterruptedException e) {
+            // reset the interrupt status on the thread
+            Thread.currentThread().interrupt();
+            throw e;
+          } catch (Exception e) {
+            throw e;
+          }
+          return Boolean.TRUE;
+        }
+
+        private void throwFailureIfDone() throws SingleIndexWriteFailureException {
+          if (stopped.isStopped() || abortable.isAborted()
+              || Thread.currentThread().isInterrupted()) {
+            throw new SingleIndexWriteFailureException(
+                "Pool closed, not attempting to write to the index!", null);
+          }
+
+        }
+      });
+    }
+
+    List<Boolean> results = null;
+    try {
+      LOG.debug("Waiting on index update tasks to complete...");
+      results = this.pool.submitUninterruptible(tasks);
+    } catch (ExecutionException e) {
+      throw new RuntimeException(
+          "Should not fail on the results while using a WaitForCompletionTaskRunner", e);
+    } catch (EarlyExitFailure e) {
+      throw new RuntimeException("Stopped while waiting for batch, quiting!", e);
+    }
+    
+    // track the failures. We only ever access this on return from our calls, so no extra
+    // synchronization is needed. We could update all the failures as we find them, but that add a
+    // lot of locking overhead, and just doing the copy later is about as efficient.
+    List<HTableInterfaceReference> failures = new ArrayList<HTableInterfaceReference>();
+    int index = 0;
+    for (Boolean result : results) {
+      // there was a failure
+      if (result == null) {
+        // we know which table failed by the index of the result
+        failures.add(tables.get(index));
+      }
+      index++;
+    }
+
+    // if any of the tasks failed, then we need to propagate the failure
+    if (failures.size() > 0) {
+      // make the list unmodifiable to avoid any more synchronization concerns
+      throw new MultiIndexWriteFailureException(Collections.unmodifiableList(failures));
+    }
+    return;
+  }
+
+  @Override
+  public void stop(String why) {
+    LOG.info("Shutting down " + this.getClass().getSimpleName());
+    this.pool.stop(why);
+    this.factory.shutdown();
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped.isStopped();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/index/BaseIndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/BaseIndexCodec.java b/phoenix-core/src/main/java/org/apache/phoenix/index/BaseIndexCodec.java
index 5a57f0b..1c45cd3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/BaseIndexCodec.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/BaseIndexCodec.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 
-import org.apache.hadoop.hbase.index.covered.IndexCodec;
+import org.apache.phoenix.hbase.index.covered.IndexCodec;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
index f44e4fa..3b28382 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -35,14 +35,14 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.index.ValueGetter;
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.client.KeyValueBuilder;
+import org.apache.phoenix.hbase.index.ValueGetter;
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
index 6f2290d..6a0aae3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
@@ -32,9 +32,9 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Pair;
 
 import com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.index.covered.CoveredColumnsIndexBuilder;
-import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
 import org.apache.phoenix.compile.ScanRanges;
+import org.apache.phoenix.hbase.index.covered.CoveredColumnsIndexBuilder;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.util.SchemaUtil;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
index 02bb066..762ca7c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
@@ -32,14 +32,6 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Pair;
 
 import com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.index.ValueGetter;
-import org.apache.hadoop.hbase.index.covered.IndexCodec;
-import org.apache.hadoop.hbase.index.covered.IndexUpdate;
-import org.apache.hadoop.hbase.index.covered.TableState;
-import org.apache.hadoop.hbase.index.scanner.Scanner;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
-import org.apache.hadoop.hbase.index.write.IndexWriter;
 import org.apache.phoenix.cache.GlobalCache;
 import org.apache.phoenix.cache.IndexMetaDataCache;
 import org.apache.phoenix.cache.ServerCacheClient;
@@ -47,6 +39,14 @@ import org.apache.phoenix.cache.TenantCache;
 import org.apache.phoenix.client.KeyValueBuilder;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.hbase.index.ValueGetter;
+import org.apache.phoenix.hbase.index.covered.IndexCodec;
+import org.apache.phoenix.hbase.index.covered.IndexUpdate;
+import org.apache.phoenix.hbase.index.covered.TableState;
+import org.apache.phoenix.hbase.index.scanner.Scanner;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
+import org.apache.phoenix.hbase.index.write.IndexWriter;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.ServerUtil;
 

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
index f5b89f8..788adbf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
@@ -32,11 +32,11 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 
 import com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-import org.apache.hadoop.hbase.index.write.KillServerOnFailurePolicy;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+import org.apache.phoenix.hbase.index.write.KillServerOnFailurePolicy;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.util.SchemaUtil;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java
index 71b7f54..846e4cc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java
@@ -29,11 +29,11 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
 import org.xerial.snappy.Snappy;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.cache.HashCache;
 import org.apache.phoenix.coprocessor.ServerCachingProtocol.ServerCacheFactory;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.ExpressionType;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
 import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.schema.tuple.Tuple;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/join/HashJoinInfo.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/join/HashJoinInfo.java b/phoenix-core/src/main/java/org/apache/phoenix/join/HashJoinInfo.java
index afdafa1..3386cda 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/join/HashJoinInfo.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/join/HashJoinInfo.java
@@ -24,9 +24,9 @@ import java.util.List;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.io.WritableUtils;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.ExpressionType;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.parse.JoinTableNode.JoinType;
 import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.PColumn;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 46c0ddc..8b73e49 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -48,8 +48,6 @@ import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
-import org.apache.hadoop.hbase.index.Indexer;
-import org.apache.hadoop.hbase.index.covered.CoveredColumnsIndexBuilder;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -71,6 +69,8 @@ import org.apache.phoenix.exception.PhoenixIOException;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.hbase.index.Indexer;
+import org.apache.phoenix.hbase.index.covered.CoveredColumnsIndexBuilder;
 import org.apache.phoenix.index.PhoenixIndexBuilder;
 import org.apache.phoenix.index.PhoenixIndexCodec;
 import org.apache.phoenix.jdbc.PhoenixConnection;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 1bd14d3..b62c3e9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -74,9 +74,9 @@ import java.math.BigDecimal;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.schema.MetaDataSplitPolicy;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNormalizedName;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
index 7853cc2..361ac36 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
@@ -17,7 +17,7 @@
  */
 package org.apache.phoenix.schema;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.util.ByteUtil;
 

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/schema/PNameImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PNameImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PNameImpl.java
index 8feca89..9d4a48e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PNameImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PNameImpl.java
@@ -20,7 +20,7 @@ package org.apache.phoenix.schema;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.http.annotation.Immutable;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 
 @Immutable
 public class PNameImpl implements PName {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 88de71d..ee95f46 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -38,11 +38,11 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.client.KeyValueBuilder;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
index 02cd70b..fd66d8a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
@@ -27,10 +27,10 @@ import java.util.Arrays;
 import java.util.Comparator;
 
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.SortOrder;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index 745f7dd..7075980 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -25,14 +25,14 @@ import java.util.Map;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.index.ValueGetter;
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.client.KeyValueBuilder;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.hbase.index.ValueGetter;
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnFamilyNotFoundException;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index 9eb3907..a14e36a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -27,11 +27,11 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Properties;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.KeyRange;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/util/TupleUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/TupleUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/TupleUtil.java
index 2c29c20..3ebbeae 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/TupleUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/TupleUtil.java
@@ -30,8 +30,8 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
 
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.tuple.Tuple;
 

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/IndexTestingUtils.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/IndexTestingUtils.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/IndexTestingUtils.java
deleted file mode 100644
index f4efe35..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/IndexTestingUtils.java
+++ /dev/null
@@ -1,94 +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.hadoop.hbase.index;
-
-import static org.junit.Assert.assertEquals;
-
-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.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec;
-import org.apache.hadoop.hbase.regionserver.wal.WALEditCodec;
-import org.apache.hadoop.hbase.util.Bytes;
-
-
-
-/**
- * Utility class for testing indexing
- */
-public class IndexTestingUtils {
-
-  private static final Log LOG = LogFactory.getLog(IndexTestingUtils.class);
-  private static final String MASTER_INFO_PORT_KEY = "hbase.master.info.port";
-  private static final String RS_INFO_PORT_KEY = "hbase.regionserver.info.port";
-  
-  private IndexTestingUtils() {
-    // private ctor for util class
-  }
-
-  public static void setupConfig(Configuration conf) {
-      conf.setInt(MASTER_INFO_PORT_KEY, -1);
-      conf.setInt(RS_INFO_PORT_KEY, -1);
-    // setup our codec, so we get proper replay/write
-      conf.set(WALEditCodec.WAL_EDIT_CODEC_CLASS_KEY, IndexedWALEditCodec.class.getName());
-  }
-  /**
-   * Verify the state of the index table between the given key and time ranges against the list of
-   * expected keyvalues.
-   * @throws IOException
-   */
-  @SuppressWarnings("javadoc")
-  public static void verifyIndexTableAtTimestamp(HTable index1, List<KeyValue> expected,
-      long start, long end, byte[] startKey, byte[] endKey) throws IOException {
-    LOG.debug("Scanning " + Bytes.toString(index1.getTableName()) + " between times (" + start
-        + ", " + end + "] and keys: [" + Bytes.toString(startKey) + ", " + Bytes.toString(endKey)
-        + "].");
-    Scan s = new Scan(startKey, endKey);
-    // s.setRaw(true);
-    s.setMaxVersions();
-    s.setTimeRange(start, end);
-    List<KeyValue> received = new ArrayList<KeyValue>();
-    ResultScanner scanner = index1.getScanner(s);
-    for (Result r : scanner) {
-      received.addAll(r.list());
-      LOG.debug("Received: " + r.list());
-    }
-    scanner.close();
-    assertEquals("Didn't get the expected kvs from the index table!", expected, received);
-  }
-
-  public static void verifyIndexTableAtTimestamp(HTable index1, List<KeyValue> expected, long ts,
-      byte[] startKey) throws IOException {
-    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, startKey, HConstants.EMPTY_END_ROW);
-  }
-
-  public static void verifyIndexTableAtTimestamp(HTable index1, List<KeyValue> expected, long start,
-      byte[] startKey, byte[] endKey) throws IOException {
-    verifyIndexTableAtTimestamp(index1, expected, start, start + 1, startKey, endKey);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/StubAbortable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/StubAbortable.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/StubAbortable.java
deleted file mode 100644
index b99c36d..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/StubAbortable.java
+++ /dev/null
@@ -1,41 +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.hadoop.hbase.index;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.Abortable;
-
-/**
- * TEst helper to stub out an {@link Abortable} when needed.
- */
-public class StubAbortable implements Abortable {
-  private static final Log LOG = LogFactory.getLog(StubAbortable.class);
-  private boolean abort;
-
-  @Override
-  public void abort(String reason, Throwable e) {
-    LOG.info("Aborting: " + reason, e);
-    abort = true;
-  }
-
-  @Override
-  public boolean isAborted() {
-    return abort;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/TableName.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/TableName.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/TableName.java
deleted file mode 100644
index dc48659..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/TableName.java
+++ /dev/null
@@ -1,45 +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.hadoop.hbase.index;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.rules.TestWatcher;
-import org.junit.runner.Description;
-
-/**
- * Returns a {@code byte[]} containing the name of the currently running test method.
- */
-public class TableName extends TestWatcher {
-  private String tableName;
-
-  /**
-   * Invoked when a test is about to start
-   */
-  @Override
-  protected void starting(Description description) {
-    tableName = description.getMethodName();
-  }
-
-  public byte[] getTableName() {
-    return Bytes.toBytes(tableName);
-  }
-
-  public String getTableNameString() {
-    return this.tableName;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/TestFailForUnsupportedHBaseVersions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/TestFailForUnsupportedHBaseVersions.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/TestFailForUnsupportedHBaseVersions.java
deleted file mode 100644
index 31fbe49..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/TestFailForUnsupportedHBaseVersions.java
+++ /dev/null
@@ -1,155 +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.hadoop.hbase.index;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.VersionInfo;
-import org.junit.Test;
-
-import org.apache.hadoop.hbase.index.covered.example.ColumnGroup;
-import org.apache.hadoop.hbase.index.covered.example.CoveredColumn;
-import org.apache.hadoop.hbase.index.covered.example.CoveredColumnIndexSpecifierBuilder;
-
-/**
- * Test that we correctly fail for versions of HBase that don't support current properties
- */
-public class TestFailForUnsupportedHBaseVersions {
-  private static final Log LOG = LogFactory.getLog(TestFailForUnsupportedHBaseVersions.class);
-
-  /**
-   * We don't support WAL Compression for HBase &lt; 0.94.9, so we shouldn't even allow the server
-   * to start if both indexing and WAL Compression are enabled for the wrong versions.
-   */
-  @Test
-  public void testDoesNotSupportCompressedWAL() {
-    Configuration conf = HBaseConfiguration.create();
-    IndexTestingUtils.setupConfig(conf);
-    // get the current version
-    String version = VersionInfo.getVersion();
-    
-    // ensure WAL Compression not enabled
-    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
-    
-    //we support all versions without WAL Compression
-    String supported = Indexer.validateVersion(version, conf);
-    assertNull(
-      "WAL Compression wasn't enabled, but version "+version+" of HBase wasn't supported! All versions should"
-          + " support writing without a compressed WAL. Message: "+supported, supported);
-
-    // enable WAL Compression
-    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
-
-    // set the version to something we know isn't supported
-    version = "0.94.4";
-    supported = Indexer.validateVersion(version, conf);
-    assertNotNull("WAL Compression was enabled, but incorrectly marked version as supported",
-      supported);
-    
-    //make sure the first version of 0.94 that supports Indexing + WAL Compression works
-    version = "0.94.9";
-    supported = Indexer.validateVersion(version, conf);
-    assertNull(
-      "WAL Compression wasn't enabled, but version "+version+" of HBase wasn't supported! Message: "+supported, supported);
-    
-    //make sure we support snapshot builds too
-    version = "0.94.9-SNAPSHOT";
-    supported = Indexer.validateVersion(version, conf);
-    assertNull(
-      "WAL Compression wasn't enabled, but version "+version+" of HBase wasn't supported! Message: "+supported, supported);
-  }
-
-  /**
-   * Test that we correctly abort a RegionServer when we run tests with an unsupported HBase
-   * version. The 'completeness' of this test requires that we run the test with both a version of
-   * HBase that wouldn't be supported with WAL Compression. Currently, this is the default version
-   * (0.94.4) so just running 'mvn test' will run the full test. However, this test will not fail
-   * when running against a version of HBase with WALCompression enabled. Therefore, to fully test
-   * this functionality, we need to run the test against both a supported and an unsupported version
-   * of HBase (as long as we want to support an version of HBase that doesn't support custom WAL
-   * Codecs).
-   * @throws Exception on failure
-   */
-  @Test(timeout = 300000 /* 5 mins */)
-  public void testDoesNotStartRegionServerForUnsupportedCompressionAndVersion() throws Exception {
-    Configuration conf = HBaseConfiguration.create();
-    IndexTestingUtils.setupConfig(conf);
-    // enable WAL Compression
-    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
-
-    // check the version to see if it isn't supported
-    String version = VersionInfo.getVersion();
-    boolean supported = false;
-    if (Indexer.validateVersion(version, conf) == null) {
-      supported = true;
-    }
-
-    // start the minicluster
-    HBaseTestingUtility util = new HBaseTestingUtility(conf);
-    util.startMiniCluster();
-
-    // setup the primary table
-    HTableDescriptor desc = new HTableDescriptor(
-        "testDoesNotStartRegionServerForUnsupportedCompressionAndVersion");
-    byte[] family = Bytes.toBytes("f");
-    desc.addFamily(new HColumnDescriptor(family));
-
-    // enable indexing to a non-existant index table
-    String indexTableName = "INDEX_TABLE";
-    ColumnGroup fam1 = new ColumnGroup(indexTableName);
-    fam1.add(new CoveredColumn(family, CoveredColumn.ALL_QUALIFIERS));
-    CoveredColumnIndexSpecifierBuilder builder = new CoveredColumnIndexSpecifierBuilder();
-    builder.addIndexGroup(fam1);
-    builder.build(desc);
-
-    // get a reference to the regionserver, so we can ensure it aborts
-    HRegionServer server = util.getMiniHBaseCluster().getRegionServer(0);
-
-    // create the primary table
-    HBaseAdmin admin = util.getHBaseAdmin();
-    if (supported) {
-      admin.createTable(desc);
-      assertFalse("Hosting regeion server failed, even the HBase version (" + version
-          + ") supports WAL Compression.", server.isAborted());
-    } else {
-      admin.createTableAsync(desc, null);
-
-      // wait for the regionserver to abort - if this doesn't occur in the timeout, assume its
-      // broken.
-      while (!server.isAborted()) {
-        LOG.debug("Waiting on regionserver to abort..");
-      }
-    }
-
-    // cleanup
-    util.shutdownMiniCluster();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/CoveredIndexCodecForTesting.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/CoveredIndexCodecForTesting.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/CoveredIndexCodecForTesting.java
deleted file mode 100644
index ccbd202..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/CoveredIndexCodecForTesting.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.index.covered;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-
-import org.apache.phoenix.index.BaseIndexCodec;
-
-/**
- * An {@link IndexCodec} for testing that allow you to specify the index updates/deletes, regardless
- * of the current tables' state.
- */
-public class CoveredIndexCodecForTesting extends BaseIndexCodec {
-
-  private List<IndexUpdate> deletes = new ArrayList<IndexUpdate>();
-  private List<IndexUpdate> updates = new ArrayList<IndexUpdate>();
-
-  public void addIndexDelete(IndexUpdate... deletes) {
-    this.deletes.addAll(Arrays.asList(deletes));
-  }
-  
-  public void addIndexUpserts(IndexUpdate... updates) {
-    this.updates.addAll(Arrays.asList(updates));
-  }
-
-  public void clear() {
-    this.deletes.clear();
-    this.updates.clear();
-  }
-  
-  @Override
-  public Iterable<IndexUpdate> getIndexDeletes(TableState state) {
-    return this.deletes;
-  }
-
-  @Override
-  public Iterable<IndexUpdate> getIndexUpserts(TableState state) {
-    return this.updates;
-  }
-
-  @Override
-  public void initialize(RegionCoprocessorEnvironment env) throws IOException {
-    // noop
-  }
-
-  @Override
-  public boolean isEnabled(Mutation m) {
-    return true;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/TestCoveredColumns.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/TestCoveredColumns.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/TestCoveredColumns.java
deleted file mode 100644
index 5af0ad3..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/TestCoveredColumns.java
+++ /dev/null
@@ -1,45 +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.hadoop.hbase.index.covered;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.Arrays;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-
-public class TestCoveredColumns {
-
-  private static final byte[] fam = Bytes.toBytes("fam");
-  private static final byte[] qual = Bytes.toBytes("qual");
-
-  @Test
-  public void testCovering() {
-    ColumnReference ref = new ColumnReference(fam, qual);
-    CoveredColumns columns = new CoveredColumns();
-    assertEquals("Should have only found a single column to cover", 1, columns
-        .findNonCoveredColumns(Arrays.asList(ref)).size());
-
-    columns.addColumn(ref);
-    assertEquals("Shouldn't have any columns to cover", 0,
-      columns.findNonCoveredColumns(Arrays.asList(ref)).size());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/TestEndToEndCoveredColumnsIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/TestEndToEndCoveredColumnsIndexBuilder.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/TestEndToEndCoveredColumnsIndexBuilder.java
deleted file mode 100644
index 026937f..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/TestEndToEndCoveredColumnsIndexBuilder.java
+++ /dev/null
@@ -1,339 +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.hadoop.hbase.index.covered;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdge;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
-import org.apache.hadoop.hbase.index.IndexTestingUtils;
-import org.apache.hadoop.hbase.index.Indexer;
-import org.apache.hadoop.hbase.index.TableName;
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-import org.apache.hadoop.hbase.index.scanner.Scanner;
-
-/**
- * End-to-End test of just the {@link CoveredColumnsIndexBuilder}, but with a simple
- * {@link IndexCodec} and BatchCache implementation.
- */
-public class TestEndToEndCoveredColumnsIndexBuilder {
-
-  public class TestState {
-
-    private HTable table;
-    private long ts;
-    private VerifyingIndexCodec codec;
-
-    /**
-     * @param primary
-     * @param codec
-     * @param ts
-     */
-    public TestState(HTable primary, VerifyingIndexCodec codec, long ts) {
-      this.table = primary;
-      this.ts = ts;
-      this.codec = codec;
-    }
-
-  }
-
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-  private static final byte[] row = Bytes.toBytes("row");
-  private static final byte[] family = Bytes.toBytes("FAM");
-  private static final byte[] qual = Bytes.toBytes("qual");
-  private static final HColumnDescriptor FAM1 = new HColumnDescriptor(family);
-
-  @Rule
-  public TableName TestTable = new TableName();
-
-  private TestState state;
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    Configuration conf = UTIL.getConfiguration();
-    IndexTestingUtils.setupConfig(conf);
-    // disable version checking, so we can test against whatever version of HBase happens to be
-    // installed (right now, its generally going to be SNAPSHOT versions).
-    conf.setBoolean(Indexer.CHECK_VERSION_CONF_KEY, false);
-    UTIL.startMiniCluster();
-  }
-
-  @AfterClass
-  public static void shutdownCluster() throws Exception {
-    UTIL.shutdownMiniCluster();
-  }
-
-  @Before
-  public void setup() throws Exception {
-    this.state = setupTest(TestTable.getTableNameString());
-  }
-    
-  private interface TableStateVerifier {
-
-    /**
-     * Verify that the state of the table is correct. Should fail the unit test if it isn't as
-     * expected.
-     * @param state
-     */
-    public void verify(TableState state);
-
-  }
-
-  /**
-   * {@link TableStateVerifier} that ensures the kvs returned from the table match the passed
-   * {@link KeyValue}s when querying on the given columns.
-   */
-  private class ListMatchingVerifier implements TableStateVerifier {
-
-    private List<KeyValue> expectedKvs;
-    private ColumnReference[] columns;
-    private String msg;
-
-    public ListMatchingVerifier(String msg, List<KeyValue> kvs, ColumnReference... columns) {
-      this.expectedKvs = kvs;
-      this.columns = columns;
-      this.msg = msg;
-    }
-
-    @Override
-    public void verify(TableState state) {
-      try {
-        Scanner kvs =
-            ((LocalTableState) state).getIndexedColumnsTableState(Arrays.asList(columns)).getFirst();
-
-        int count = 0;
-        KeyValue kv;
-        while ((kv = kvs.next()) != null) {
-          KeyValue next = expectedKvs.get(count++);
-          assertEquals(
-            msg + ": Unexpected kv in table state!\nexpected v1: "
-                + Bytes.toString(next.getValue()) + "\nactual v1:" + Bytes.toString(kv.getValue()),
-            next, kv);
-        }
-
-        assertEquals(msg + ": Didn't find enough kvs in table state!", expectedKvs.size(), count);
-      } catch (IOException e) {
-        fail(msg + ": Got an exception while reading local table state! " + e.getMessage());
-      }
-    }
-  }
-
-  private class VerifyingIndexCodec extends CoveredIndexCodecForTesting {
-
-    private Queue<TableStateVerifier> verifiers = new ArrayDeque<TableStateVerifier>();
-
-    @Override
-    public Iterable<IndexUpdate> getIndexDeletes(TableState state) {
-      verify(state);
-      return super.getIndexDeletes(state);
-    }
-
-    @Override
-    public Iterable<IndexUpdate> getIndexUpserts(TableState state) {
-      verify(state);
-      return super.getIndexUpserts(state);
-    }
-
-    private void verify(TableState state) {
-      TableStateVerifier verifier = verifiers.poll();
-      if (verifier == null) return;
-      verifier.verify(state);
-    }
-  }
-  
-  /**
-   * Test that we see the expected values in a {@link TableState} when doing single puts against a
-   * region.
-   * @throws Exception on failure
-   */
-  @Test
-  public void testExpectedResultsInTableStateForSinglePut() throws Exception {
-    //just do a simple Put to start with
-    long ts = state.ts;
-    Put p = new Put(row, ts);
-    p.add(family, qual, Bytes.toBytes("v1"));
-    
-    // get all the underlying kvs for the put
-    final List<KeyValue> expectedKvs = new ArrayList<KeyValue>();
-    final List<KeyValue> allKvs = new ArrayList<KeyValue>();
-    allKvs.addAll(p.getFamilyMap().get(family));
-
-    // setup the verifier for the data we expect to write
-    // first call shouldn't have anything in the table
-    final ColumnReference familyRef =
-        new ColumnReference(TestEndToEndCoveredColumnsIndexBuilder.family, ColumnReference.ALL_QUALIFIERS);
-
-    VerifyingIndexCodec codec = state.codec;
-    codec.verifiers.add(new ListMatchingVerifier("cleanup state 1", expectedKvs, familyRef));
-    codec.verifiers.add(new ListMatchingVerifier("put state 1", allKvs, familyRef));
-
-    // do the actual put (no indexing will actually be done)
-    HTable primary = state.table;
-    primary.put(p);
-    primary.flushCommits();
-
-    // now we do another put to the same row. We should see just the old row state, followed by the
-    // new + old
-    p = new Put(row, ts + 1);
-    p.add(family, qual, Bytes.toBytes("v2"));
-    expectedKvs.addAll(allKvs);
-    // add them first b/c the ts is newer
-    allKvs.addAll(0, p.get(family, qual));
-    codec.verifiers.add(new ListMatchingVerifier("cleanup state 2", expectedKvs, familyRef));
-    codec.verifiers.add(new ListMatchingVerifier("put state 2", allKvs, familyRef));
-    
-    // do the actual put
-    primary.put(p);
-    primary.flushCommits();
-
-    // cleanup after ourselves
-    cleanup(state);
-  }
-
-  /**
-   * Similar to {@link #testExpectedResultsInTableStateForSinglePut()}, but against batches of puts.
-   * Previous implementations managed batches by playing current state against each element in the
-   * batch, rather than combining all the per-row updates into a single mutation for the batch. This
-   * test ensures that we see the correct expected state.
-   * @throws Exception on failure
-   */
-  @Test
-  public void testExpectedResultsInTableStateForBatchPuts() throws Exception {
-    long ts = state.ts;
-    // build up a list of puts to make, all on the same row
-    Put p1 = new Put(row, ts);
-    p1.add(family, qual, Bytes.toBytes("v1"));
-    Put p2 = new Put(row, ts + 1);
-    p2.add(family, qual, Bytes.toBytes("v2"));
-
-    // setup all the verifiers we need. This is just the same as above, but will be called twice
-    // since we need to iterate the batch.
-
-    // get all the underlying kvs for the put
-    final List<KeyValue> allKvs = new ArrayList<KeyValue>(2);
-    allKvs.addAll(p2.getFamilyMap().get(family));
-    allKvs.addAll(p1.getFamilyMap().get(family));
-
-    // setup the verifier for the data we expect to write
-    // both puts should be put into a single batch
-    final ColumnReference familyRef =
-        new ColumnReference(TestEndToEndCoveredColumnsIndexBuilder.family, ColumnReference.ALL_QUALIFIERS);
-    VerifyingIndexCodec codec = state.codec;
-    // no previous state in the table
-    codec.verifiers.add(new ListMatchingVerifier("cleanup state 1", Collections
-        .<KeyValue> emptyList(), familyRef));
-    codec.verifiers.add(new ListMatchingVerifier("put state 1", p1.getFamilyMap().get(family),
-        familyRef));
-
-    codec.verifiers.add(new ListMatchingVerifier("cleanup state 2", p1.getFamilyMap().get(family),
-        familyRef));
-    // kvs from both puts should be in the table now
-    codec.verifiers.add(new ListMatchingVerifier("put state 2", allKvs, familyRef));
-
-    // do the actual put (no indexing will actually be done)
-    HTable primary = state.table;
-    primary.setAutoFlush(false);
-    primary.put(Arrays.asList(p1, p2));
-    primary.flushCommits();
-
-    // cleanup after ourselves
-    cleanup(state);
-  }
-
-  /**
-   * @param tableName name of the table to create for the test
-   * @return the supporting state for the test
-   */
-  private TestState setupTest(String tableName) throws IOException {
-    byte[] tableNameBytes = Bytes.toBytes(tableName);
-    HTableDescriptor desc = new HTableDescriptor(tableNameBytes);
-    desc.addFamily(FAM1);
-    // add the necessary simple options to create the builder
-    Map<String, String> indexerOpts = new HashMap<String, String>();
-    // just need to set the codec - we are going to set it later, but we need something here or the
-    // initializer blows up.
-    indexerOpts.put(CoveredColumnsIndexBuilder.CODEC_CLASS_NAME_KEY,
-      CoveredIndexCodecForTesting.class.getName());
-    Indexer.enableIndexing(desc, CoveredColumnsIndexBuilder.class, indexerOpts);
-
-    // create the table
-    HBaseAdmin admin = UTIL.getHBaseAdmin();
-    admin.createTable(desc);
-    HTable primary = new HTable(UTIL.getConfiguration(), tableNameBytes);
-
-    // overwrite the codec so we can verify the current state
-    HRegion region = UTIL.getMiniHBaseCluster().getRegions(tableNameBytes).get(0);
-    Indexer indexer =
-        (Indexer) region.getCoprocessorHost().findCoprocessor(Indexer.class.getName());
-    CoveredColumnsIndexBuilder builder =
-        (CoveredColumnsIndexBuilder) indexer.getBuilderForTesting();
-    VerifyingIndexCodec codec = new VerifyingIndexCodec();
-    builder.setIndexCodecForTesting(codec);
-
-    // setup the Puts we want to write
-    final long ts = System.currentTimeMillis();
-    EnvironmentEdge edge = new EnvironmentEdge() {
-
-      @Override
-      public long currentTimeMillis() {
-        return ts;
-      }
-    };
-    EnvironmentEdgeManager.injectEdge(edge);
-
-    return new TestState(primary, codec, ts);
-  }
-
-  /**
-   * Cleanup the test based on the passed state.
-   * @param state
-   */
-  private void cleanup(TestState state) throws IOException {
-    EnvironmentEdgeManager.reset();
-    state.table.close();
-    UTIL.deleteTable(state.table.getTableName());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/TestLocalTableState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/TestLocalTableState.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/TestLocalTableState.java
deleted file mode 100644
index e20b8bb..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/TestLocalTableState.java
+++ /dev/null
@@ -1,196 +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.hadoop.hbase.index.covered;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.RegionScanner;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import org.apache.hadoop.hbase.index.covered.data.LocalHBaseState;
-import org.apache.hadoop.hbase.index.covered.data.LocalTable;
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-import org.apache.hadoop.hbase.index.scanner.Scanner;
-
-/**
- *
- */
-public class TestLocalTableState {
-
-  private static final byte[] row = Bytes.toBytes("row");
-  private static final byte[] fam = Bytes.toBytes("fam");
-  private static final byte[] qual = Bytes.toBytes("qual");
-  private static final byte[] val = Bytes.toBytes("val");
-  private static final long ts = 10;
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testCorrectOrderingWithLazyLoadingColumns() throws Exception {
-    Put m = new Put(row);
-    m.add(fam, qual, ts, val);
-    // setup mocks
-    Configuration conf = new Configuration(false);
-    RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
-    Mockito.when(env.getConfiguration()).thenReturn(conf);
-
-    HRegion region = Mockito.mock(HRegion.class);
-    Mockito.when(env.getRegion()).thenReturn(region);
-    RegionScanner scanner = Mockito.mock(RegionScanner.class);
-    Mockito.when(region.getScanner(Mockito.any(Scan.class))).thenReturn(scanner);
-    final byte[] stored = Bytes.toBytes("stored-value");
-    Mockito.when(scanner.next(Mockito.any(List.class))).thenAnswer(new Answer<Boolean>() {
-      @Override
-      public Boolean answer(InvocationOnMock invocation) throws Throwable {
-        List<KeyValue> list = (List<KeyValue>) invocation.getArguments()[0];
-        KeyValue kv = new KeyValue(row, fam, qual, ts, Type.Put, stored);
-        kv.setMemstoreTS(0);
-        list.add(kv);
-        return false;
-      }
-    });
-
-
-    LocalHBaseState state = new LocalTable(env);
-    LocalTableState table = new LocalTableState(env, state, m);
-    //add the kvs from the mutation
-    table.addPendingUpdates(m.get(fam, qual));
-
-    // setup the lookup
-    ColumnReference col = new ColumnReference(fam, qual);
-    table.setCurrentTimestamp(ts);
-    //check that our value still shows up first on scan, even though this is a lazy load
-    Pair<Scanner, IndexUpdate> p = table.getIndexedColumnsTableState(Arrays.asList(col));
-    Scanner s = p.getFirst();
-    assertEquals("Didn't get the pending mutation's value first", m.get(fam, qual).get(0), s.next());
-  }
-
-  /**
-   * Test that we correctly rollback the state of keyvalue
-   * @throws Exception
-   */
-  @Test
-  @SuppressWarnings("unchecked")
-  public void testCorrectRollback() throws Exception {
-    Put m = new Put(row);
-    m.add(fam, qual, ts, val);
-    // setup mocks
-    RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
-
-    HRegion region = Mockito.mock(HRegion.class);
-    Mockito.when(env.getRegion()).thenReturn(region);
-    RegionScanner scanner = Mockito.mock(RegionScanner.class);
-    Mockito.when(region.getScanner(Mockito.any(Scan.class))).thenReturn(scanner);
-    final byte[] stored = Bytes.toBytes("stored-value");
-    final KeyValue storedKv = new KeyValue(row, fam, qual, ts, Type.Put, stored);
-    storedKv.setMemstoreTS(2);
-    Mockito.when(scanner.next(Mockito.any(List.class))).thenAnswer(new Answer<Boolean>() {
-      @Override
-      public Boolean answer(InvocationOnMock invocation) throws Throwable {
-        List<KeyValue> list = (List<KeyValue>) invocation.getArguments()[0];
-
-        list.add(storedKv);
-        return false;
-      }
-    });
-    LocalHBaseState state = new LocalTable(env);
-    LocalTableState table = new LocalTableState(env, state, m);
-    // add the kvs from the mutation
-    KeyValue kv = m.get(fam, qual).get(0);
-    kv.setMemstoreTS(0);
-    table.addPendingUpdates(kv);
-
-    // setup the lookup
-    ColumnReference col = new ColumnReference(fam, qual);
-    table.setCurrentTimestamp(ts);
-    // check that the value is there
-    Pair<Scanner, IndexUpdate> p = table.getIndexedColumnsTableState(Arrays.asList(col));
-    Scanner s = p.getFirst();
-    assertEquals("Didn't get the pending mutation's value first", kv, s.next());
-
-    // rollback that value
-    table.rollback(Arrays.asList(kv));
-    p = table.getIndexedColumnsTableState(Arrays.asList(col));
-    s = p.getFirst();
-    assertEquals("Didn't correctly rollback the row - still found it!", null, s.next());
-    Mockito.verify(env, Mockito.times(1)).getRegion();
-    Mockito.verify(region, Mockito.times(1)).getScanner(Mockito.any(Scan.class));
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testOnlyLoadsRequestedColumns() throws Exception {
-    // setup mocks
-    RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
-
-    HRegion region = Mockito.mock(HRegion.class);
-    Mockito.when(env.getRegion()).thenReturn(region);
-    RegionScanner scanner = Mockito.mock(RegionScanner.class);
-    Mockito.when(region.getScanner(Mockito.any(Scan.class))).thenReturn(scanner);
-    final KeyValue storedKv =
-        new KeyValue(row, fam, qual, ts, Type.Put, Bytes.toBytes("stored-value"));
-    storedKv.setMemstoreTS(2);
-    Mockito.when(scanner.next(Mockito.any(List.class))).thenAnswer(new Answer<Boolean>() {
-      @Override
-      public Boolean answer(InvocationOnMock invocation) throws Throwable {
-        List<KeyValue> list = (List<KeyValue>) invocation.getArguments()[0];
-
-        list.add(storedKv);
-        return false;
-      }
-    });
-    LocalHBaseState state = new LocalTable(env);
-    Put pendingUpdate = new Put(row);
-    pendingUpdate.add(fam, qual, ts, val);
-    LocalTableState table = new LocalTableState(env, state, pendingUpdate);
-
-    // do the lookup for the given column
-    ColumnReference col = new ColumnReference(fam, qual);
-    table.setCurrentTimestamp(ts);
-    // check that the value is there
-    Pair<Scanner, IndexUpdate> p = table.getIndexedColumnsTableState(Arrays.asList(col));
-    Scanner s = p.getFirst();
-    // make sure it read the table the one time
-    assertEquals("Didn't get the stored keyvalue!", storedKv, s.next());
-
-    // on the second lookup it shouldn't access the underlying table again - the cached columns
-    // should know they are done
-    p = table.getIndexedColumnsTableState(Arrays.asList(col));
-    s = p.getFirst();
-    assertEquals("Lost already loaded update!", storedKv, s.next());
-    Mockito.verify(env, Mockito.times(1)).getRegion();
-    Mockito.verify(region, Mockito.times(1)).getScanner(Mockito.any(Scan.class));
-  }
-
-  // TODO add test here for making sure multiple column references with the same column family don't
-  // cause an infinite loop
-}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/data/TestIndexMemStore.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/data/TestIndexMemStore.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/data/TestIndexMemStore.java
deleted file mode 100644
index ad9293e..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/data/TestIndexMemStore.java
+++ /dev/null
@@ -1,93 +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.hadoop.hbase.index.covered.data;
-
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-
-public class TestIndexMemStore {
-
-  private static final byte[] row = Bytes.toBytes("row");
-  private static final byte[] family = Bytes.toBytes("family");
-  private static final byte[] qual = Bytes.toBytes("qual");
-  private static final byte[] val = Bytes.toBytes("val");
-  private static final byte[] val2 = Bytes.toBytes("val2");
-
-  @Test
-  public void testCorrectOverwritting() throws Exception {
-    IndexMemStore store = new IndexMemStore(IndexMemStore.COMPARATOR);
-    long ts = 10;
-    KeyValue kv = new KeyValue(row, family, qual, ts, Type.Put, val);
-    kv.setMemstoreTS(2);
-    KeyValue kv2 = new KeyValue(row, family, qual, ts, Type.Put, val2);
-    kv2.setMemstoreTS(0);
-    store.add(kv, true);
-    // adding the exact same kv shouldn't change anything stored if not overwritting
-    store.add(kv2, false);
-    KeyValueScanner scanner = store.getScanner();
-    KeyValue first = KeyValue.createFirstOnRow(row);
-    scanner.seek(first);
-    assertTrue("Overwrote kv when specifically not!", kv == scanner.next());
-    scanner.close();
-
-    // now when we overwrite, we should get the newer one
-    store.add(kv2, true);
-    scanner = store.getScanner();
-    scanner.seek(first);
-    assertTrue("Didn't overwrite kv when specifically requested!", kv2 == scanner.next());
-    scanner.close();
-  }
-
-  /**
-   * We don't expect custom KeyValue creation, so we can't get into weird situations, where a
-   * {@link Type#DeleteFamily} has a column qualifier specified.
-   * @throws Exception
-   */
-  @Test
-  public void testExpectedOrdering() throws Exception {
-    IndexMemStore store = new IndexMemStore();
-    KeyValue kv = new KeyValue(row, family, qual, 12, Type.Put, val);
-    store.add(kv, true);
-    KeyValue kv2 = new KeyValue(row, family, qual, 10, Type.Put, val2);
-    store.add(kv2, true);
-    KeyValue df = new KeyValue(row, family, null, 11, Type.DeleteFamily, null);
-    store.add(df, true);
-    KeyValue dc = new KeyValue(row, family, qual, 11, Type.DeleteColumn, null);
-    store.add(dc, true);
-    KeyValue d = new KeyValue(row, family, qual, 12, Type.Delete, null);
-    store.add(d, true);
-
-    // null qualifiers should always sort before the non-null cases
-    KeyValueScanner scanner = store.getScanner();
-    KeyValue first = KeyValue.createFirstOnRow(row);
-    assertTrue("Didn't have any data in the scanner", scanner.seek(first));
-    assertTrue("Didn't get delete family first (no qualifier == sort first)", df == scanner.next());
-    assertTrue("Didn't get point delete before corresponding put", d == scanner.next());
-    assertTrue("Didn't get larger ts Put", kv == scanner.next());
-    assertTrue("Didn't get delete column before corresponding put(delete sorts first)",
-      dc == scanner.next());
-    assertTrue("Didn't get smaller ts Put", kv2 == scanner.next());
-    assertNull("Have more data in the scanner", scanner.next());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestColumnTracker.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestColumnTracker.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestColumnTracker.java
deleted file mode 100644
index a349149..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/covered/example/TestColumnTracker.java
+++ /dev/null
@@ -1,61 +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.hadoop.hbase.index.covered.example;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.Collection;
-
-import org.junit.Test;
-
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-import org.apache.hadoop.hbase.index.covered.update.ColumnTracker;
-
-public class TestColumnTracker {
-
-  @Test
-  public void testEnsureGuarranteedMinValid() {
-    assertFalse("Guarranted min wasn't recognized as having newer timestamps!",
-      ColumnTracker.isNewestTime(ColumnTracker.GUARANTEED_NEWER_UPDATES));
-  }
-
-  @Test
-  public void testOnlyKeepsOlderTimestamps() {
-    Collection<ColumnReference> columns = new ArrayList<ColumnReference>();
-    ColumnTracker tracker = new ColumnTracker(columns);
-    tracker.setTs(10);
-    assertEquals("Column tracker didn't set original TS", 10, tracker.getTS());
-    tracker.setTs(12);
-    assertEquals("Column tracker allowed newer timestamp to be set.", 10, tracker.getTS());
-    tracker.setTs(9);
-    assertEquals("Column tracker didn't decrease set timestamp for smaller value", 9,
-      tracker.getTS());
-  }
-
-  @Test
-  public void testHasNewerTimestamps() throws Exception {
-    Collection<ColumnReference> columns = new ArrayList<ColumnReference>();
-    ColumnTracker tracker = new ColumnTracker(columns);
-    assertFalse("Tracker has newer timestamps when no ts set", tracker.hasNewerTimestamps());
-    tracker.setTs(10);
-    assertTrue("Tracker doesn't have newer timetamps with set ts", tracker.hasNewerTimestamps());
-  }
-}
\ No newline at end of file


[10/15] Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java
new file mode 100644
index 0000000..a035da5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java
@@ -0,0 +1,704 @@
+/*
+ * 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.phoenix.hbase.index;
+
+import static org.apache.phoenix.hbase.index.util.IndexManagementUtil.rethrowIndexingException;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.wal.HLog;
+import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.Multimap;
+import org.apache.phoenix.hbase.index.builder.IndexBuildManager;
+import org.apache.phoenix.hbase.index.builder.IndexBuilder;
+import org.apache.phoenix.hbase.index.builder.IndexBuildingFailureException;
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
+import org.apache.phoenix.hbase.index.wal.IndexedKeyValue;
+import org.apache.phoenix.hbase.index.write.IndexFailurePolicy;
+import org.apache.phoenix.hbase.index.write.IndexWriter;
+import org.apache.phoenix.hbase.index.write.recovery.PerRegionIndexWriteCache;
+import org.apache.phoenix.hbase.index.write.recovery.StoreFailuresInCachePolicy;
+import org.apache.phoenix.hbase.index.write.recovery.TrackingParallelWriterIndexCommitter;
+import org.apache.phoenix.util.MetaDataUtil;
+
+/**
+ * Do all the work of managing index updates from a single coprocessor. All Puts/Delets are passed
+ * to an {@link IndexBuilder} to determine the actual updates to make.
+ * <p>
+ * If the WAL is enabled, these updates are then added to the WALEdit and attempted to be written to
+ * the WAL after the WALEdit has been saved. If any of the index updates fail, this server is
+ * immediately terminated and we rely on WAL replay to attempt the index updates again (see
+ * {@link #preWALRestore(ObserverContext, HRegionInfo, HLogKey, WALEdit)}).
+ * <p>
+ * If the WAL is disabled, the updates are attempted immediately. No consistency guarantees are made
+ * if the WAL is disabled - some or none of the index updates may be successful. All updates in a
+ * single batch must have the same durability level - either everything gets written to the WAL or
+ * nothing does. Currently, we do not support mixed-durability updates within a single batch. If you
+ * want to have different durability levels, you only need to split the updates into two different
+ * batches.
+ */
+public class Indexer extends BaseRegionObserver {
+
+  private static final Log LOG = LogFactory.getLog(Indexer.class);
+
+  /** WAL on this server */
+  private HLog log;
+  protected IndexWriter writer;
+  protected IndexBuildManager builder;
+
+  /** Configuration key for the {@link IndexBuilder} to use */
+  public static final String INDEX_BUILDER_CONF_KEY = "index.builder";
+
+  // Setup out locking on the index edits/WAL so we can be sure that we don't lose a roll a WAL edit
+  // before an edit is applied to the index tables
+  private static final ReentrantReadWriteLock INDEX_READ_WRITE_LOCK = new ReentrantReadWriteLock(
+      true);
+  public static final ReadLock INDEX_UPDATE_LOCK = INDEX_READ_WRITE_LOCK.readLock();
+
+  /**
+   * Configuration key for if the indexer should check the version of HBase is running. Generally,
+   * you only want to ignore this for testing or for custom versions of HBase.
+   */
+  public static final String CHECK_VERSION_CONF_KEY = "com.saleforce.hbase.index.checkversion";
+
+  private static final String INDEX_RECOVERY_FAILURE_POLICY_KEY = "org.apache.hadoop.hbase.index.recovery.failurepolicy";
+
+  /**
+   * Marker {@link KeyValue} to indicate that we are doing a batch operation. Needed because the
+   * coprocessor framework throws away the WALEdit from the prePut/preDelete hooks when checking a
+   * batch if there were no {@link KeyValue}s attached to the {@link WALEdit}. When you get down to
+   * the preBatch hook, there won't be any WALEdits to which to add the index updates.
+   */
+  private static KeyValue BATCH_MARKER = new KeyValue();
+
+  /**
+   * cache the failed updates to the various regions. Used for making the WAL recovery mechanisms
+   * more robust in the face of recoverying index regions that were on the same server as the
+   * primary table region
+   */
+  private PerRegionIndexWriteCache failedIndexEdits = new PerRegionIndexWriteCache();
+
+  /**
+   * IndexWriter for writing the recovered index edits. Separate from the main indexer since we need
+   * different write/failure policies
+   */
+  private IndexWriter recoveryWriter;
+
+  private boolean stopped;
+  private boolean disabled;
+
+  public static final String RecoveryFailurePolicyKeyForTesting = INDEX_RECOVERY_FAILURE_POLICY_KEY;
+
+    public static final int INDEXING_SUPPORTED_MAJOR_VERSION = MetaDataUtil
+            .encodeMaxPatchVersion(0, 94);
+    public static final int INDEXING_SUPPORTED__MIN_MAJOR_VERSION = MetaDataUtil
+            .encodeVersion("0.94.0");
+    private static final int INDEX_WAL_COMPRESSION_MINIMUM_SUPPORTED_VERSION = MetaDataUtil
+            .encodeVersion("0.94.9");
+
+  @Override
+  public void start(CoprocessorEnvironment e) throws IOException {
+      try {
+        final RegionCoprocessorEnvironment env = (RegionCoprocessorEnvironment) e;
+        String serverName = env.getRegionServerServices().getServerName().getServerName();
+        if (env.getConfiguration().getBoolean(CHECK_VERSION_CONF_KEY, true)) {
+          // make sure the right version <-> combinations are allowed.
+          String errormsg = Indexer.validateVersion(env.getHBaseVersion(), env.getConfiguration());
+          if (errormsg != null) {
+            IOException ioe = new IOException(errormsg);
+            env.getRegionServerServices().abort(errormsg, ioe);
+            throw ioe;
+          }
+        }
+    
+        this.builder = new IndexBuildManager(env);
+    
+        // get a reference to the WAL
+        log = env.getRegionServerServices().getWAL();
+        // add a synchronizer so we don't archive a WAL that we need
+        log.registerWALActionsListener(new IndexLogRollSynchronizer(INDEX_READ_WRITE_LOCK.writeLock()));
+    
+        // setup the actual index writer
+        this.writer = new IndexWriter(env, serverName + "-index-writer");
+    
+        // setup the recovery writer that does retries on the failed edits
+        TrackingParallelWriterIndexCommitter recoveryCommmiter =
+            new TrackingParallelWriterIndexCommitter();
+    
+        try {
+          // get the specified failure policy. We only ever override it in tests, but we need to do it
+          // here
+          Class<? extends IndexFailurePolicy> policyClass =
+              env.getConfiguration().getClass(INDEX_RECOVERY_FAILURE_POLICY_KEY,
+                StoreFailuresInCachePolicy.class, IndexFailurePolicy.class);
+          IndexFailurePolicy policy =
+              policyClass.getConstructor(PerRegionIndexWriteCache.class).newInstance(failedIndexEdits);
+          LOG.debug("Setting up recovery writter with committer: " + recoveryCommmiter.getClass()
+              + " and failure policy: " + policy.getClass());
+          recoveryWriter =
+              new IndexWriter(recoveryCommmiter, policy, env, serverName + "-recovery-writer");
+        } catch (Exception ex) {
+          throw new IOException("Could not instantiate recovery failure policy!", ex);
+        }
+      } catch (NoSuchMethodError ex) {
+          disabled = true;
+          super.start(e);
+          LOG.error("Must be too early a version of HBase. Disabled coprocessor ", ex);
+      }
+  }
+
+  @Override
+  public void stop(CoprocessorEnvironment e) throws IOException {
+    if (this.stopped) {
+      return;
+    }
+    if (this.disabled) {
+        super.stop(e);
+        return;
+      }
+    this.stopped = true;
+    String msg = "Indexer is being stopped";
+    this.builder.stop(msg);
+    this.writer.stop(msg);
+    this.recoveryWriter.stop(msg);
+  }
+
+  @Override
+  public void prePut(final ObserverContext<RegionCoprocessorEnvironment> c, final Put put,
+      final WALEdit edit, final boolean writeToWAL) throws IOException {
+      if (this.disabled) {
+          super.prePut(c, put, edit, writeToWAL);
+          return;
+        }
+    // just have to add a batch marker to the WALEdit so we get the edit again in the batch
+    // processing step. We let it throw an exception here because something terrible has happened.
+    edit.add(BATCH_MARKER);
+  }
+
+  @Override
+  public void preDelete(ObserverContext<RegionCoprocessorEnvironment> e, Delete delete,
+      WALEdit edit, boolean writeToWAL) throws IOException {
+      if (this.disabled) {
+          super.preDelete(e, delete, edit, writeToWAL);
+          return;
+        }
+    try {
+      preDeleteWithExceptions(e, delete, edit, writeToWAL);
+      return;
+    } catch (Throwable t) {
+      rethrowIndexingException(t);
+    }
+    throw new RuntimeException(
+        "Somehow didn't return an index update but also didn't propagate the failure to the client!");
+  }
+
+  public void preDeleteWithExceptions(ObserverContext<RegionCoprocessorEnvironment> e,
+      Delete delete, WALEdit edit, boolean writeToWAL) throws Exception {
+    // if we are making the update as part of a batch, we need to add in a batch marker so the WAL
+    // is retained
+    if (this.builder.getBatchId(delete) != null) {
+      edit.add(BATCH_MARKER);
+      return;
+    }
+
+    // get the mapping for index column -> target index table
+    Collection<Pair<Mutation, byte[]>> indexUpdates = this.builder.getIndexUpdate(delete);
+
+    if (doPre(indexUpdates, edit, writeToWAL)) {
+      takeUpdateLock("delete");
+    }
+  }
+
+  @Override
+  public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+      MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
+      if (this.disabled) {
+          super.preBatchMutate(c, miniBatchOp);
+          return;
+        }
+    try {
+      preBatchMutateWithExceptions(c, miniBatchOp);
+      return;
+    } catch (Throwable t) {
+      rethrowIndexingException(t);
+    }
+    throw new RuntimeException(
+        "Somehow didn't return an index update but also didn't propagate the failure to the client!");
+  }
+
+  @SuppressWarnings("deprecation")
+  public void preBatchMutateWithExceptions(ObserverContext<RegionCoprocessorEnvironment> c,
+      MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws Throwable {
+
+    // first group all the updates for a single row into a single update to be processed
+    Map<ImmutableBytesPtr, MultiMutation> mutations =
+        new HashMap<ImmutableBytesPtr, MultiMutation>();
+    boolean durable = false;
+    for (int i = 0; i < miniBatchOp.size(); i++) {
+      // remove the batch keyvalue marker - its added for all puts
+      WALEdit edit = miniBatchOp.getWalEdit(i);
+      // we don't have a WALEdit for immutable index cases, which still see this path
+      // we could check is indexing is enable for the mutation in prePut and then just skip this
+      // after checking here, but this saves us the checking again.
+      if (edit != null) {
+        KeyValue kv = edit.getKeyValues().remove(0);
+        assert kv == BATCH_MARKER : "Expected batch marker from the WALEdit, but got: " + kv;
+      }
+      Pair<Mutation, Integer> op = miniBatchOp.getOperation(i);
+      Mutation m = op.getFirst();
+      // skip this mutation if we aren't enabling indexing
+      // unfortunately, we really should ask if the raw mutation (rather than the combined mutation)
+      // should be indexed, which means we need to expose another method on the builder. Such is the
+      // way optimization go though.
+      if (!this.builder.isEnabled(m)) {
+        continue;
+      }
+      
+      // figure out if this is batch is durable or not
+      if(!durable){
+        durable = m.getDurability() != Durability.SKIP_WAL;
+      }
+
+      // add the mutation to the batch set
+      ImmutableBytesPtr row = new ImmutableBytesPtr(m.getRow());
+      MultiMutation stored = mutations.get(row);
+      // we haven't seen this row before, so add it
+      if (stored == null) {
+        stored = new MultiMutation(row, m.getWriteToWAL());
+        mutations.put(row, stored);
+      }
+      stored.addAll(m);
+    }
+    
+    // early exit if it turns out we don't have any edits
+    if (mutations.entrySet().size() == 0) {
+      return;
+    }
+
+    // dump all the index updates into a single WAL. They will get combined in the end anyways, so
+    // don't worry which one we get
+    WALEdit edit = miniBatchOp.getWalEdit(0);
+
+    // get the index updates for all elements in this batch
+    Collection<Pair<Mutation, byte[]>> indexUpdates =
+        this.builder.getIndexUpdate(miniBatchOp, mutations.values());
+    // write them
+    if (doPre(indexUpdates, edit, durable)) {
+      takeUpdateLock("batch mutation");
+    }
+  }
+
+  private void takeUpdateLock(String opDesc) throws IndexBuildingFailureException {
+    boolean interrupted = false;
+    // lock the log, so we are sure that index write gets atomically committed
+    LOG.debug("Taking INDEX_UPDATE readlock for " + opDesc);
+    // wait for the update lock
+    while (!this.stopped) {
+      try {
+        INDEX_UPDATE_LOCK.lockInterruptibly();
+        LOG.debug("Got the INDEX_UPDATE readlock for " + opDesc);
+        // unlock the lock so the server can shutdown, if we find that we have stopped since getting
+        // the lock
+        if (this.stopped) {
+          INDEX_UPDATE_LOCK.unlock();
+          throw new IndexBuildingFailureException(
+              "Found server stop after obtaining the update lock, killing update attempt");
+        }
+        break;
+      } catch (InterruptedException e) {
+        LOG.info("Interrupted while waiting for update lock. Ignoring unless stopped");
+        interrupted = true;
+      }
+    }
+    if (interrupted) {
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  private class MultiMutation extends Mutation {
+
+    private ImmutableBytesPtr rowKey;
+
+    public MultiMutation(ImmutableBytesPtr rowkey, boolean writeToWal) {
+      this.rowKey = rowkey;
+      this.writeToWAL = writeToWal;
+    }
+
+    /**
+     * @param stored
+     */
+    @SuppressWarnings("deprecation")
+    public void addAll(Mutation stored) {
+      // add all the kvs
+      for (Entry<byte[], List<KeyValue>> kvs : stored.getFamilyMap().entrySet()) {
+        byte[] family = kvs.getKey();
+        List<KeyValue> list = getKeyValueList(family, kvs.getValue().size());
+        list.addAll(kvs.getValue());
+        familyMap.put(family, list);
+      }
+
+      // add all the attributes, not overriding already stored ones
+      for (Entry<String, byte[]> attrib : stored.getAttributesMap().entrySet()) {
+        if (this.getAttribute(attrib.getKey()) == null) {
+          this.setAttribute(attrib.getKey(), attrib.getValue());
+        }
+      }
+      if (stored.getWriteToWAL()) {
+        this.writeToWAL = true;
+      }
+    }
+
+    private List<KeyValue> getKeyValueList(byte[] family, int hint) {
+      List<KeyValue> list = familyMap.get(family);
+      if (list == null) {
+        list = new ArrayList<KeyValue>(hint);
+      }
+      return list;
+    }
+
+    @Override
+    public byte[] getRow(){
+      return this.rowKey.copyBytesIfNecessary();
+    }
+
+    @Override
+    public int hashCode() {
+      return this.rowKey.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      return o == null ? false : o.hashCode() == this.hashCode();
+    }
+
+    @Override
+    public void readFields(DataInput arg0) throws IOException {
+      throw new UnsupportedOperationException("MultiMutations cannot be read/written");
+    }
+
+    @Override
+    public void write(DataOutput arg0) throws IOException {
+      throw new UnsupportedOperationException("MultiMutations cannot be read/written");
+    }
+  }
+
+  /**
+   * Add the index updates to the WAL, or write to the index table, if the WAL has been disabled
+   * @return <tt>true</tt> if the WAL has been updated.
+   * @throws IOException
+   */
+  private boolean doPre(Collection<Pair<Mutation, byte[]>> indexUpdates, final WALEdit edit,
+      final boolean writeToWAL) throws IOException {
+    // no index updates, so we are done
+    if (indexUpdates == null || indexUpdates.size() == 0) {
+      return false;
+    }
+
+    // if writing to wal is disabled, we never see the WALEdit updates down the way, so do the index
+    // update right away
+    if (!writeToWAL) {
+      try {
+        this.writer.write(indexUpdates);
+        return false;
+      } catch (Throwable e) {
+        LOG.error("Failed to update index with entries:" + indexUpdates, e);
+        IndexManagementUtil.rethrowIndexingException(e);
+      }
+    }
+
+    // we have all the WAL durability, so we just update the WAL entry and move on
+    for (Pair<Mutation, byte[]> entry : indexUpdates) {
+      edit.add(new IndexedKeyValue(entry.getSecond(), entry.getFirst()));
+    }
+
+    return true;
+  }
+
+  @Override
+  public void postPut(ObserverContext<RegionCoprocessorEnvironment> e, Put put, WALEdit edit,
+      boolean writeToWAL) throws IOException {
+      if (this.disabled) {
+          super.postPut(e, put, edit, writeToWAL);
+          return;
+        }
+    doPost(edit, put, writeToWAL);
+  }
+
+  @Override
+  public void postDelete(ObserverContext<RegionCoprocessorEnvironment> e, Delete delete,
+      WALEdit edit, boolean writeToWAL) throws IOException {
+      if (this.disabled) {
+          super.postDelete(e, delete, edit, writeToWAL);
+          return;
+        }
+    doPost(edit,delete, writeToWAL);
+  }
+
+  @Override
+  public void postBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+      MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
+      if (this.disabled) {
+          super.postBatchMutate(c, miniBatchOp);
+          return;
+        }
+    this.builder.batchCompleted(miniBatchOp);
+    // noop for the rest of the indexer - its handled by the first call to put/delete
+  }
+
+  private void doPost(WALEdit edit, Mutation m, boolean writeToWAL) throws IOException {
+    try {
+      doPostWithExceptions(edit, m, writeToWAL);
+      return;
+    } catch (Throwable e) {
+      rethrowIndexingException(e);
+    }
+    throw new RuntimeException(
+        "Somehow didn't complete the index update, but didn't return succesfully either!");
+  }
+
+  private void doPostWithExceptions(WALEdit edit, Mutation m, boolean writeToWAL) throws Exception {
+    //short circuit, if we don't need to do any work
+    if (!writeToWAL || !this.builder.isEnabled(m)) {
+      // already did the index update in prePut, so we are done
+      return;
+    }
+
+    // there is a little bit of excess here- we iterate all the non-indexed kvs for this check first
+    // and then do it again later when getting out the index updates. This should be pretty minor
+    // though, compared to the rest of the runtime
+    IndexedKeyValue ikv = getFirstIndexedKeyValue(edit);
+    /*
+     * early exit - we have nothing to write, so we don't need to do anything else. NOTE: we don't
+     * release the WAL Rolling lock (INDEX_UPDATE_LOCK) since we never take it in doPre if there are
+     * no index updates.
+     */
+    if (ikv == null) {
+      return;
+    }
+
+    /*
+     * only write the update if we haven't already seen this batch. We only want to write the batch
+     * once (this hook gets called with the same WALEdit for each Put/Delete in a batch, which can
+     * lead to writing all the index updates for each Put/Delete).
+     */
+    if (!ikv.getBatchFinished()) {
+      Collection<Pair<Mutation, byte[]>> indexUpdates = extractIndexUpdate(edit);
+
+      // the WAL edit is kept in memory and we already specified the factory when we created the
+      // references originally - therefore, we just pass in a null factory here and use the ones
+      // already specified on each reference
+      try {
+          writer.writeAndKillYourselfOnFailure(indexUpdates);
+      } finally {
+        // With a custom kill policy, we may throw instead of kill the server.
+        // Without doing this in a finally block (at least with the mini cluster),
+        // the region server never goes down.
+
+        // mark the batch as having been written. In the single-update case, this never gets check
+        // again, but in the batch case, we will check it again (see above).
+        ikv.markBatchFinished();
+      
+        // release the lock on the index, we wrote everything properly
+        // we took the lock for each Put/Delete, so we have to release it a matching number of times
+        // batch cases only take the lock once, so we need to make sure we don't over-release the
+        // lock.
+        LOG.debug("Releasing INDEX_UPDATE readlock");
+        INDEX_UPDATE_LOCK.unlock();
+      }
+    }
+  }
+
+  /**
+   * Search the {@link WALEdit} for the first {@link IndexedKeyValue} present
+   * @param edit {@link WALEdit}
+   * @return the first {@link IndexedKeyValue} in the {@link WALEdit} or <tt>null</tt> if not
+   *         present
+   */
+  private IndexedKeyValue getFirstIndexedKeyValue(WALEdit edit) {
+    for (KeyValue kv : edit.getKeyValues()) {
+      if (kv instanceof IndexedKeyValue) {
+        return (IndexedKeyValue) kv;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Extract the index updates from the WAL Edit
+   * @param edit to search for index updates
+   * @return the mutations to apply to the index tables
+   */
+  private Collection<Pair<Mutation, byte[]>> extractIndexUpdate(WALEdit edit) {
+    Collection<Pair<Mutation, byte[]>> indexUpdates = new ArrayList<Pair<Mutation, byte[]>>();
+    for (KeyValue kv : edit.getKeyValues()) {
+      if (kv instanceof IndexedKeyValue) {
+        IndexedKeyValue ikv = (IndexedKeyValue) kv;
+        indexUpdates.add(new Pair<Mutation, byte[]>(ikv.getMutation(), ikv.getIndexTable()));
+      }
+    }
+
+    return indexUpdates;
+  }
+
+  @Override
+  public void postOpen(final ObserverContext<RegionCoprocessorEnvironment> c) {
+    Multimap<HTableInterfaceReference, Mutation> updates = failedIndexEdits.getEdits(c.getEnvironment().getRegion());
+    
+    if (this.disabled) {
+        super.postOpen(c);
+        return;
+      }
+    LOG.info("Found some outstanding index updates that didn't succeed during"
+        + " WAL replay - attempting to replay now.");
+    //if we have no pending edits to complete, then we are done
+    if (updates == null || updates.size() == 0) {
+      return;
+    }
+    
+    // do the usual writer stuff, killing the server again, if we can't manage to make the index
+    // writes succeed again
+    try {
+        writer.writeAndKillYourselfOnFailure(updates);
+    } catch (IOException e) {
+        LOG.error("Exception thrown instead of killing server during index writing", e);
+    }
+  }
+
+  @Override
+  public void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info,
+      HLogKey logKey, WALEdit logEdit) throws IOException {
+      if (this.disabled) {
+          super.preWALRestore(env, info, logKey, logEdit);
+          return;
+        }
+    // TODO check the regions in transition. If the server on which the region lives is this one,
+    // then we should rety that write later in postOpen.
+    // we might be able to get even smarter here and pre-split the edits that are server-local
+    // into their own recovered.edits file. This then lets us do a straightforward recovery of each
+    // region (and more efficiently as we aren't writing quite as hectically from this one place).
+
+    /*
+     * Basically, we let the index regions recover for a little while long before retrying in the
+     * hopes they come up before the primary table finishes.
+     */
+    Collection<Pair<Mutation, byte[]>> indexUpdates = extractIndexUpdate(logEdit);
+    recoveryWriter.writeAndKillYourselfOnFailure(indexUpdates);
+  }
+
+  /**
+   * Create a custom {@link InternalScanner} for a compaction that tracks the versions of rows that
+   * are removed so we can clean then up from the the index table(s).
+   * <p>
+   * This is not yet implemented - its not clear if we should even mess around with the Index table
+   * for these rows as those points still existed. TODO: v2 of indexing
+   */
+  @Override
+  public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
+      Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
+      InternalScanner s) throws IOException {
+    return super.preCompactScannerOpen(c, store, scanners, scanType, earliestPutTs, s);
+  }
+
+  /**
+   * Exposed for testing!
+   * @return the currently instantiated index builder
+   */
+  public IndexBuilder getBuilderForTesting() {
+    return this.builder.getBuilderForTesting();
+  }
+
+    /**
+     * Validate that the version and configuration parameters are supported
+     * @param hbaseVersion current version of HBase on which <tt>this</tt> coprocessor is installed
+     * @param conf configuration to check for allowed parameters (e.g. WAL Compression only if >=
+     *            0.94.9)
+     * @return <tt>null</tt> if the version is supported, the error message to display otherwise
+     */
+    public static String validateVersion(String hbaseVersion, Configuration conf) {
+        int encodedVersion = MetaDataUtil.encodeVersion(hbaseVersion);
+        // above 0.94 everything should be supported
+        if (encodedVersion > INDEXING_SUPPORTED_MAJOR_VERSION) {
+            return null;
+        }
+        // check to see if its at least 0.94
+        if (encodedVersion < INDEXING_SUPPORTED__MIN_MAJOR_VERSION) {
+            return "Indexing not supported for versions older than 0.94.X";
+        }
+        // if less than 0.94.9, we need to check if WAL Compression is enabled
+        if (encodedVersion < INDEX_WAL_COMPRESSION_MINIMUM_SUPPORTED_VERSION) {
+            if (conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false)) {
+                return "Indexing not supported with WAL Compression for versions of HBase older than 0.94.9 - found version:"
+                        + hbaseVersion;
+            }
+        }
+        return null;
+    }
+
+  /**
+   * Enable indexing on the given table
+   * @param desc {@link HTableDescriptor} for the table on which indexing should be enabled
+   * @param builder class to use when building the index for this table
+   * @param properties map of custom configuration options to make available to your
+   *          {@link IndexBuilder} on the server-side
+   * @throws IOException the Indexer coprocessor cannot be added
+   */
+  public static void enableIndexing(HTableDescriptor desc, Class<? extends IndexBuilder> builder,
+      Map<String, String> properties) throws IOException {
+    if (properties == null) {
+      properties = new HashMap<String, String>();
+    }
+    properties.put(Indexer.INDEX_BUILDER_CONF_KEY, builder.getName());
+    desc.addCoprocessor(Indexer.class.getName(), null, Coprocessor.PRIORITY_USER, properties);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/ValueGetter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/ValueGetter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/ValueGetter.java
new file mode 100644
index 0000000..0e321a7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/ValueGetter.java
@@ -0,0 +1,36 @@
+/*
+ * 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.phoenix.hbase.index;
+
+import java.io.IOException;
+
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+
+public interface ValueGetter {
+
+  /**
+   * Get the most recent (largest timestamp) for the given column reference
+   * @param ref to match against an underlying key value. Uses the passed object to match the
+   *          keyValue via {@link ColumnReference#matches}
+   * @return the stored value for the given {@link ColumnReference}, or <tt>null</tt> if no value is
+   *         present.
+   * @throws IOException if there is an error accessing the underlying data storage
+   */
+  public ImmutableBytesPtr getLatestValue(ColumnReference ref) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/BaseIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/BaseIndexBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/BaseIndexBuilder.java
new file mode 100644
index 0000000..d008d59
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/BaseIndexBuilder.java
@@ -0,0 +1,97 @@
+/*
+ * 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.phoenix.hbase.index.builder;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.phoenix.hbase.index.covered.CoveredColumnsIndexBuilder;
+
+/**
+ * Basic implementation of the {@link IndexBuilder} that doesn't do any actual work of indexing.
+ * <p>
+ * You should extend this class, rather than implementing IndexBuilder directly to maintain
+ * compatability going forward.
+ * <p>
+ * Generally, you should consider using one of the implemented IndexBuilders (e.g
+ * {@link CoveredColumnsIndexBuilder}) as there is a lot of work required to keep an index table
+ * up-to-date.
+ */
+public abstract class BaseIndexBuilder implements IndexBuilder {
+
+  private static final Log LOG = LogFactory.getLog(BaseIndexBuilder.class);
+  protected boolean stopped;
+
+  @Override
+  public void extendBaseIndexBuilderInstead() { }
+  
+  @Override
+  public void setup(RegionCoprocessorEnvironment conf) throws IOException {
+    // noop
+  }
+
+  @Override
+  public void batchStarted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
+    // noop
+  }
+
+  @Override
+  public void batchCompleted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) {
+    // noop
+  }
+  
+  /**
+   * By default, we always attempt to index the mutation. Commonly this can be slow (because the
+   * framework spends the time to do the indexing, only to realize that you don't need it) or not
+   * ideal (if you want to turn on/off indexing on a table without completely reloading it).
+ * @throws IOException 
+   */
+  @Override
+  public boolean isEnabled(Mutation m) throws IOException {
+    return true; 
+  }
+
+  /**
+   * {@inheritDoc}
+   * <p>
+   * By default, assumes that all mutations should <b>not be batched</b>. That is to say, each
+   * mutation always applies to different rows, even if they are in the same batch, or are
+   * independent updates.
+   */
+  @Override
+  public byte[] getBatchId(Mutation m) {
+    return null;
+  }
+
+  @Override
+  public void stop(String why) {
+    LOG.debug("Stopping because: " + why);
+    this.stopped = true;
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuildManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuildManager.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuildManager.java
new file mode 100644
index 0000000..2149209
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuildManager.java
@@ -0,0 +1,214 @@
+/*
+ * 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.phoenix.hbase.index.builder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.phoenix.hbase.index.Indexer;
+import org.apache.phoenix.hbase.index.parallel.QuickFailingTaskRunner;
+import org.apache.phoenix.hbase.index.parallel.Task;
+import org.apache.phoenix.hbase.index.parallel.TaskBatch;
+import org.apache.phoenix.hbase.index.parallel.ThreadPoolBuilder;
+import org.apache.phoenix.hbase.index.parallel.ThreadPoolManager;
+
+/**
+ * Manage the building of index updates from primary table updates.
+ * <p>
+ * Internally, parallelizes updates through a thread-pool to a delegate index builder. Underlying
+ * {@link IndexBuilder} <b>must be thread safe</b> for each index update.
+ */
+public class IndexBuildManager implements Stoppable {
+
+  private static final Log LOG = LogFactory.getLog(IndexBuildManager.class);
+  private final IndexBuilder delegate;
+  private QuickFailingTaskRunner pool;
+  private boolean stopped;
+
+  /**
+   * Set the number of threads with which we can concurrently build index updates. Unused threads
+   * will be released, but setting the number of threads too high could cause frequent swapping and
+   * resource contention on the server - <i>tune with care</i>. However, if you are spending a lot
+   * of time building index updates, it could be worthwhile to spend the time to tune this parameter
+   * as it could lead to dramatic increases in speed.
+   */
+  public static final String NUM_CONCURRENT_INDEX_BUILDER_THREADS_CONF_KEY = "index.builder.threads.max";
+  /** Default to a single thread. This is the safest course of action, but the slowest as well */
+  private static final int DEFAULT_CONCURRENT_INDEX_BUILDER_THREADS = 10;
+  /**
+   * Amount of time to keep idle threads in the pool. After this time (seconds) we expire the
+   * threads and will re-create them as needed, up to the configured max
+   */
+  private static final String INDEX_BUILDER_KEEP_ALIVE_TIME_CONF_KEY =
+      "index.builder.threads.keepalivetime";
+
+  /**
+   * @param env environment in which <tt>this</tt> is running. Used to setup the
+   *          {@link IndexBuilder} and executor
+   * @throws IOException if an {@link IndexBuilder} cannot be correctly steup
+   */
+  public IndexBuildManager(RegionCoprocessorEnvironment env) throws IOException {
+    this(getIndexBuilder(env), new QuickFailingTaskRunner(ThreadPoolManager.getExecutor(
+      getPoolBuilder(env), env)));
+  }
+
+  private static IndexBuilder getIndexBuilder(RegionCoprocessorEnvironment e) throws IOException {
+    Configuration conf = e.getConfiguration();
+    Class<? extends IndexBuilder> builderClass =
+        conf.getClass(Indexer.INDEX_BUILDER_CONF_KEY, null, IndexBuilder.class);
+    try {
+      IndexBuilder builder = builderClass.newInstance();
+      builder.setup(e);
+      return builder;
+    } catch (InstantiationException e1) {
+      throw new IOException("Couldn't instantiate index builder:" + builderClass
+          + ", disabling indexing on table " + e.getRegion().getTableDesc().getNameAsString());
+    } catch (IllegalAccessException e1) {
+      throw new IOException("Couldn't instantiate index builder:" + builderClass
+          + ", disabling indexing on table " + e.getRegion().getTableDesc().getNameAsString());
+    }
+  }
+
+  private static ThreadPoolBuilder getPoolBuilder(RegionCoprocessorEnvironment env) {
+    String serverName = env.getRegionServerServices().getServerName().getServerName();
+    return new ThreadPoolBuilder(serverName + "-index-builder", env.getConfiguration()).
+        setCoreTimeout(INDEX_BUILDER_KEEP_ALIVE_TIME_CONF_KEY).
+        setMaxThread(NUM_CONCURRENT_INDEX_BUILDER_THREADS_CONF_KEY,
+          DEFAULT_CONCURRENT_INDEX_BUILDER_THREADS);
+  }
+
+  public IndexBuildManager(IndexBuilder builder, QuickFailingTaskRunner pool) {
+    this.delegate = builder;
+    this.pool = pool;
+  }
+
+
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(
+      MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp,
+      Collection<? extends Mutation> mutations) throws Throwable {
+    // notify the delegate that we have started processing a batch
+    this.delegate.batchStarted(miniBatchOp);
+
+    // parallelize each mutation into its own task
+    // each task is cancelable via two mechanisms: (1) underlying HRegion is closing (which would
+    // fail lookups/scanning) and (2) by stopping this via the #stop method. Interrupts will only be
+    // acknowledged on each thread before doing the actual lookup, but after that depends on the
+    // underlying builder to look for the closed flag.
+    TaskBatch<Collection<Pair<Mutation, byte[]>>> tasks =
+        new TaskBatch<Collection<Pair<Mutation, byte[]>>>(mutations.size());
+    for (final Mutation m : mutations) {
+      tasks.add(new Task<Collection<Pair<Mutation, byte[]>>>() {
+
+        @Override
+        public Collection<Pair<Mutation, byte[]>> call() throws IOException {
+          return delegate.getIndexUpdate(m);
+        }
+
+      });
+    }
+    List<Collection<Pair<Mutation, byte[]>>> allResults = null;
+    try {
+      allResults = pool.submitUninterruptible(tasks);
+    } catch (CancellationException e) {
+      throw e;
+    } catch (ExecutionException e) {
+      LOG.error("Found a failed index update!");
+      throw e.getCause();
+    }
+
+    // we can only get here if we get successes from each of the tasks, so each of these must have a
+    // correct result
+    Collection<Pair<Mutation, byte[]>> results = new ArrayList<Pair<Mutation, byte[]>>();
+    for (Collection<Pair<Mutation, byte[]>> result : allResults) {
+      assert result != null : "Found an unsuccessful result, but didn't propagate a failure earlier";
+      results.addAll(result);
+    }
+
+    return results;
+  }
+
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Delete delete) throws IOException {
+    // all we get is a single update, so it would probably just go slower if we needed to queue it
+    // up. It will increase underlying resource contention a little bit, but the mutation case is
+    // far more common, so let's not worry about it for now.
+    // short circuit so we don't waste time.
+    if (!this.delegate.isEnabled(delete)) {
+      return null;
+    }
+
+    return delegate.getIndexUpdate(delete);
+
+  }
+
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdateForFilteredRows(
+      Collection<KeyValue> filtered) throws IOException {
+    // this is run async, so we can take our time here
+    return delegate.getIndexUpdateForFilteredRows(filtered);
+  }
+
+  public void batchCompleted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) {
+    delegate.batchCompleted(miniBatchOp);
+  }
+
+  public void batchStarted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp)
+      throws IOException {
+    delegate.batchStarted(miniBatchOp);
+  }
+
+  public boolean isEnabled(Mutation m) throws IOException {
+    return delegate.isEnabled(m);
+  }
+
+  public byte[] getBatchId(Mutation m) {
+    return delegate.getBatchId(m);
+  }
+
+  @Override
+  public void stop(String why) {
+    if (stopped) {
+      return;
+    }
+    this.stopped = true;
+    this.delegate.stop(why);
+    this.pool.stop(why);
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped;
+  }
+
+  public IndexBuilder getBuilderForTesting() {
+    return this.delegate;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuilder.java
new file mode 100644
index 0000000..50e5aef
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuilder.java
@@ -0,0 +1,142 @@
+/*
+ * 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.phoenix.hbase.index.builder;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.phoenix.hbase.index.Indexer;
+
+/**
+ * Interface to build updates ({@link Mutation}s) to the index tables, based on the primary table
+ * updates.
+ * <p>
+ * Either all the index updates will be applied to all tables or the primary table will kill itself
+ * and will attempt to replay the index edits through the WAL replay mechanism.
+ */
+public interface IndexBuilder extends Stoppable {
+
+  /** Helper method signature to ensure people don't attempt to extend this class directly */
+  public void extendBaseIndexBuilderInstead();
+
+  /**
+   * This is always called exactly once on install of {@link Indexer}, before any calls
+   * {@link #getIndexUpdate} on
+   * @param env in which the builder is running
+   * @throws IOException on failure to setup the builder
+   */
+  public void setup(RegionCoprocessorEnvironment env) throws IOException;
+
+  /**
+   * Your opportunity to update any/all index tables based on the update of the primary table row.
+   * Its up to your implementation to ensure that timestamps match between the primary and index
+   * tables.
+   * <p>
+   * The mutation is a generic mutation (not a {@link Put} or a {@link Delete}), as it actually
+   * corresponds to a batch update. Its important to note that {@link Put}s always go through the
+   * batch update code path, so a single {@link Put} will come through here and update the primary
+   * table as the only update in the mutation.
+   * <p>
+   * Implementers must ensure that this method is thread-safe - it could (and probably will) be
+   * called concurrently for different mutations, which may or may not be part of the same batch.
+   * @param mutation update to the primary table to be indexed.
+   * @return a Map of the mutations to make -> target index table name
+   * @throws IOException on failure
+   */
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Mutation mutation) throws IOException;
+
+  /**
+   * The counter-part to {@link #getIndexUpdate(Mutation)} - your opportunity to update any/all
+   * index tables based on the delete of the primary table row. This is only called for cases where
+   * the client sends a single delete ({@link HTable#delete}). We separate this method from
+   * {@link #getIndexUpdate(Mutation)} only for the ease of implementation as the delete path has
+   * subtly different semantics for updating the families/timestamps from the generic batch path.
+   * <p>
+   * Its up to your implementation to ensure that timestamps match between the primary and index
+   * tables.
+   * <p>
+   * Implementers must ensure that this method is thread-safe - it could (and probably will) be
+   * called concurrently for different mutations, which may or may not be part of the same batch.
+   * @param delete {@link Delete} to the primary table that may be indexed
+   * @return a {@link Map} of the mutations to make -> target index table name
+   * @throws IOException on failure
+   */
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Delete delete) throws IOException;
+
+  /**
+   * Build an index update to cleanup the index when we remove {@link KeyValue}s via the normal
+   * flush or compaction mechanisms.
+   * @param filtered {@link KeyValue}s that previously existed, but won't be included in further
+   *          output from HBase.
+   * @return a {@link Map} of the mutations to make -> target index table name
+   * @throws IOException on failure
+   */
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdateForFilteredRows(
+      Collection<KeyValue> filtered)
+      throws IOException;
+
+  /**
+   * Notification that a batch of updates has successfully been written.
+   * @param miniBatchOp the full batch operation that was written
+   */
+  public void batchCompleted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp);
+
+  /**
+   * Notification that a batch has been started.
+   * <p>
+   * Unfortunately, the way HBase has the coprocessor hooks setup, this is actually called
+   * <i>after</i> the {@link #getIndexUpdate} methods. Therefore, you will likely need an attribute
+   * on your {@link Put}/{@link Delete} to indicate it is a batch operation.
+   * @param miniBatchOp the full batch operation to be written
+ * @throws IOException 
+   */
+  public void batchStarted(MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException;
+
+  /**
+   * This allows the codec to dynamically change whether or not indexing should take place for a
+   * table. If it doesn't take place, we can save a lot of time on the regular Put patch. By making
+   * it dynamic, we can save offlining and then onlining a table just to turn indexing on.
+   * <p>
+   * We can also be smart about even indexing a given update here too - if the update doesn't
+   * contain any columns that we care about indexing, we can save the effort of analyzing the put
+   * and further.
+   * @param m mutation that should be indexed.
+   * @return <tt>true</tt> if indexing is enabled for the given table. This should be on a per-table
+   *         basis, as each codec is instantiated per-region.
+ * @throws IOException 
+   */
+  public boolean isEnabled(Mutation m) throws IOException;
+
+  /**
+   * @param m mutation that has been received by the indexer and is waiting to be indexed
+   * @return the ID of batch to which the Mutation belongs, or <tt>null</tt> if the mutation is not
+   *         part of a batch.
+   */
+  public byte[] getBatchId(Mutation m);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuildingFailureException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuildingFailureException.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuildingFailureException.java
new file mode 100644
index 0000000..cc7cc35
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuildingFailureException.java
@@ -0,0 +1,48 @@
+/*
+ * 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.phoenix.hbase.index.builder;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+
+/**
+ * Unexpected failure while building index updates that wasn't caused by an {@link IOException}.
+ * This should be used if there is some basic issue with indexing - and no matter of retries will
+ * fix it.
+ */
+@SuppressWarnings("serial")
+public class IndexBuildingFailureException extends DoNotRetryIOException {
+
+  /**
+   * Constructor for over the wire propagation. Generally, shouldn't be used since index failure
+   * should have an underlying cause to propagate.
+   * @param msg reason for the failure
+   */
+  public IndexBuildingFailureException(String msg) {
+    super(msg);
+  }
+
+  /**
+   * @param msg reason
+   * @param cause underlying cause for the failure
+   */
+  public IndexBuildingFailureException(String msg, Throwable cause) {
+    super(msg, cause);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/Batch.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/Batch.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/Batch.java
new file mode 100644
index 0000000..e707ea2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/Batch.java
@@ -0,0 +1,60 @@
+/*
+ * 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.phoenix.hbase.index.covered;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+/**
+ * A collection of {@link KeyValue KeyValues} to the primary table
+ */
+public class Batch {
+
+  private static final long pointDeleteCode = KeyValue.Type.Delete.getCode();
+  private final long timestamp;
+  private List<KeyValue> batch = new ArrayList<KeyValue>();
+  private boolean allPointDeletes = true;
+
+  /**
+   * @param ts
+   */
+  public Batch(long ts) {
+    this.timestamp = ts;
+  }
+
+  public void add(KeyValue kv){
+    if (pointDeleteCode != kv.getType()) {
+      allPointDeletes = false;
+    }
+    batch.add(kv);
+  }
+
+  public boolean isAllPointDeletes() {
+    return allPointDeletes;
+  }
+
+  public long getTimestamp() {
+    return this.timestamp;
+  }
+
+  public List<KeyValue> getKvs() {
+    return this.batch;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/CoveredColumns.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/CoveredColumns.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/CoveredColumns.java
new file mode 100644
index 0000000..c6474e1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/CoveredColumns.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.phoenix.hbase.index.covered;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+
+/**
+ * Manage a set of {@link ColumnReference}s for the {@link LocalTableState}.
+ */
+public class CoveredColumns {
+
+  Set<ColumnReference> columns = new HashSet<ColumnReference>();
+
+  public Collection<? extends ColumnReference> findNonCoveredColumns(
+      Collection<? extends ColumnReference> columns2) {
+    List<ColumnReference> uncovered = new ArrayList<ColumnReference>();
+    for (ColumnReference column : columns2) {
+      if (!columns.contains(column)) {
+        uncovered.add(column);
+      }
+    }
+    return uncovered;
+  }
+
+  public void addColumn(ColumnReference column) {
+    this.columns.add(column);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/CoveredColumnsIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/CoveredColumnsIndexBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/CoveredColumnsIndexBuilder.java
new file mode 100644
index 0000000..0667467
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/CoveredColumnsIndexBuilder.java
@@ -0,0 +1,488 @@
+/*
+ * 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.phoenix.hbase.index.covered;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Longs;
+import org.apache.phoenix.hbase.index.builder.BaseIndexBuilder;
+import org.apache.phoenix.hbase.index.covered.data.LocalHBaseState;
+import org.apache.phoenix.hbase.index.covered.data.LocalTable;
+import org.apache.phoenix.hbase.index.covered.update.ColumnTracker;
+import org.apache.phoenix.hbase.index.covered.update.IndexUpdateManager;
+import org.apache.phoenix.hbase.index.covered.update.IndexedColumnGroup;
+
+/**
+ * Build covered indexes for phoenix updates.
+ * <p>
+ * Before any call to prePut/preDelete, the row has already been locked. This ensures that we don't
+ * need to do any extra synchronization in the IndexBuilder.
+ * <p>
+ * NOTE: This implementation doesn't cleanup the index when we remove a key-value on compaction or
+ * flush, leading to a bloated index that needs to be cleaned up by a background process.
+ */
+public class CoveredColumnsIndexBuilder extends BaseIndexBuilder {
+
+  private static final Log LOG = LogFactory.getLog(CoveredColumnsIndexBuilder.class);
+  public static final String CODEC_CLASS_NAME_KEY = "org.apache.hadoop.hbase.index.codec.class";
+
+  protected RegionCoprocessorEnvironment env;
+  protected IndexCodec codec;
+  protected LocalHBaseState localTable;
+
+  @Override
+  public void setup(RegionCoprocessorEnvironment env) throws IOException {
+    this.env = env;
+    // setup the phoenix codec. Generally, this will just be in standard one, but abstracting here
+    // so we can use it later when generalizing covered indexes
+    Configuration conf = env.getConfiguration();
+    Class<? extends IndexCodec> codecClass =
+        conf.getClass(CODEC_CLASS_NAME_KEY, null, IndexCodec.class);
+    try {
+      Constructor<? extends IndexCodec> meth = codecClass.getDeclaredConstructor(new Class[0]);
+      meth.setAccessible(true);
+      this.codec = meth.newInstance();
+      this.codec.initialize(env);
+    } catch (IOException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+    
+    this.localTable = new LocalTable(env);
+  }
+
+  @Override
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Mutation mutation) throws IOException {
+    // build the index updates for each group
+    IndexUpdateManager updateMap = new IndexUpdateManager();
+
+    batchMutationAndAddUpdates(updateMap, mutation);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Found index updates for Mutation: " + mutation + "\n" + updateMap);
+    }
+
+    return updateMap.toMap();
+  }
+
+  /**
+   * Split the mutation into batches based on the timestamps of each keyvalue. We need to check each
+   * key-value in the update to see if it matches the others. Generally, this will be the case, but
+   * you can add kvs to a mutation that don't all have the timestamp, so we need to manage
+   * everything in batches based on timestamp.
+   * <p>
+   * Adds all the updates in the {@link Mutation} to the state, as a side-effect.
+   * @param updateMap index updates into which to add new updates. Modified as a side-effect.
+   * @param state current state of the row for the mutation.
+   * @param m mutation to batch
+ * @throws IOException 
+   */
+  private void batchMutationAndAddUpdates(IndexUpdateManager manager, Mutation m) throws IOException {
+    // split the mutation into timestamp-based batches
+    Collection<Batch> batches = createTimestampBatchesFromMutation(m);
+
+    // create a state manager, so we can manage each batch
+    LocalTableState state = new LocalTableState(env, localTable, m);
+
+    // go through each batch of keyvalues and build separate index entries for each
+    boolean cleanupCurrentState = true;
+    for (Batch batch : batches) {
+      /*
+       * We have to split the work between the cleanup and the update for each group because when we
+       * update the current state of the row for the current batch (appending the mutations for the
+       * current batch) the next group will see that as the current state, which will can cause the
+       * a delete and a put to be created for the next group.
+       */
+      if (addMutationsForBatch(manager, batch, state, cleanupCurrentState)) {
+        cleanupCurrentState = false;
+      }
+    }
+  }
+
+  /**
+   * Batch all the {@link KeyValue}s in a {@link Mutation} by timestamp. Updates any
+   * {@link KeyValue} with a timestamp == {@link HConstants#LATEST_TIMESTAMP} to the timestamp at
+   * the time the method is called.
+   * @param m {@link Mutation} from which to extract the {@link KeyValue}s
+   * @return the mutation, broken into batches and sorted in ascending order (smallest first)
+   */
+  protected Collection<Batch> createTimestampBatchesFromMutation(Mutation m) {
+    Map<Long, Batch> batches = new HashMap<Long, Batch>();
+    for (List<KeyValue> family : m.getFamilyMap().values()) {
+      createTimestampBatchesFromKeyValues(family, batches);
+    }
+    // sort the batches
+    List<Batch> sorted = new ArrayList<Batch>(batches.values());
+    Collections.sort(sorted, new Comparator<Batch>() {
+      @Override
+      public int compare(Batch o1, Batch o2) {
+        return Longs.compare(o1.getTimestamp(), o2.getTimestamp());
+      }
+    });
+    return sorted;
+  }
+
+  /**
+   * Batch all the {@link KeyValue}s in a collection of kvs by timestamp. Updates any
+   * {@link KeyValue} with a timestamp == {@link HConstants#LATEST_TIMESTAMP} to the timestamp at
+   * the time the method is called.
+   * @param kvs {@link KeyValue}s to break into batches
+   * @param batches to update with the given kvs
+   */
+  protected void createTimestampBatchesFromKeyValues(Collection<KeyValue> kvs,
+      Map<Long, Batch> batches) {
+    long now = EnvironmentEdgeManager.currentTimeMillis();
+    byte[] nowBytes = Bytes.toBytes(now);
+
+    // batch kvs by timestamp
+    for (KeyValue kv : kvs) {
+      long ts = kv.getTimestamp();
+      // override the timestamp to the current time, so the index and primary tables match
+      // all the keys with LATEST_TIMESTAMP will then be put into the same batch
+      if (kv.updateLatestStamp(nowBytes)) {
+        ts = now;
+      }
+      Batch batch = batches.get(ts);
+      if (batch == null) {
+        batch = new Batch(ts);
+        batches.put(ts, batch);
+      }
+      batch.add(kv);
+    }
+  }
+
+  /**
+   * For a single batch, get all the index updates and add them to the updateMap
+   * <p>
+   * This method manages cleaning up the entire history of the row from the given timestamp forward
+   * for out-of-order (e.g. 'back in time') updates.
+   * <p>
+   * If things arrive out of order (client is using custom timestamps) we should still see the index
+   * in the correct order (assuming we scan after the out-of-order update in finished). Therefore,
+   * we when we aren't the most recent update to the index, we need to delete the state at the
+   * current timestamp (similar to above), but also issue a delete for the added index updates at
+   * the next newest timestamp of any of the columns in the update; we need to cleanup the insert so
+   * it looks like it was also deleted at that next newest timestamp. However, its not enough to
+   * just update the one in front of us - that column will likely be applied to index entries up the
+   * entire history in front of us, which also needs to be fixed up.
+   * <p>
+   * However, the current update usually will be the most recent thing to be added. In that case,
+   * all we need to is issue a delete for the previous index row (the state of the row, without the
+   * update applied) at the current timestamp. This gets rid of anything currently in the index for
+   * the current state of the row (at the timestamp). Then we can just follow that by applying the
+   * pending update and building the index update based on the new row state.
+   * @param updateMap map to update with new index elements
+   * @param batch timestamp-based batch of edits
+   * @param state local state to update and pass to the codec
+   * @param requireCurrentStateCleanup <tt>true</tt> if we should should attempt to cleanup the
+   *          current state of the table, in the event of a 'back in time' batch. <tt>false</tt>
+   *          indicates we should not attempt the cleanup, e.g. an earlier batch already did the
+   *          cleanup.
+   * @return <tt>true</tt> if we cleaned up the current state forward (had a back-in-time put),
+   *         <tt>false</tt> otherwise
+ * @throws IOException 
+   */
+  private boolean addMutationsForBatch(IndexUpdateManager updateMap, Batch batch,
+      LocalTableState state, boolean requireCurrentStateCleanup) throws IOException {
+
+    // need a temporary manager for the current batch. It should resolve any conflicts for the
+    // current batch. Essentially, we can get the case where a batch doesn't change the current
+    // state of the index (all Puts are covered by deletes), in which case we don't want to add
+    // anything
+    // A. Get the correct values for the pending state in the batch
+    // A.1 start by cleaning up the current state - as long as there are key-values in the batch
+    // that are indexed, we need to change the current state of the index. Its up to the codec to
+    // determine if we need to make any cleanup given the pending update.
+    long batchTs = batch.getTimestamp();
+    state.setPendingUpdates(batch.getKvs());
+    addCleanupForCurrentBatch(updateMap, batchTs, state);
+
+    // A.2 do a single pass first for the updates to the current state
+    state.applyPendingUpdates();
+    long minTs = addUpdateForGivenTimestamp(batchTs, state, updateMap);
+    // if all the updates are the latest thing in the index, we are done - don't go and fix history
+    if (ColumnTracker.isNewestTime(minTs)) {
+      return false;
+    }
+
+    // A.3 otherwise, we need to roll up through the current state and get the 'correct' view of the
+    // index. after this, we have the correct view of the index, from the batch up to the index
+    while(!ColumnTracker.isNewestTime(minTs) ){
+      minTs = addUpdateForGivenTimestamp(minTs, state, updateMap);
+    }
+
+    // B. only cleanup the current state if we need to - its a huge waste of effort otherwise.
+   if (requireCurrentStateCleanup) {
+      // roll back the pending update. This is needed so we can remove all the 'old' index entries.
+      // We don't need to do the puts here, but just the deletes at the given timestamps since we
+      // just want to completely hide the incorrect entries.
+      state.rollback(batch.getKvs());
+      // setup state
+      state.setPendingUpdates(batch.getKvs());
+
+      // cleanup the pending batch. If anything in the correct history is covered by Deletes used to
+      // 'fix' history (same row key and ts), we just drop the delete (we don't want to drop both
+      // because the update may have a different set of columns or value based on the update).
+      cleanupIndexStateFromBatchOnward(updateMap, batchTs, state);
+
+      // have to roll the state forward again, so the current state is correct
+      state.applyPendingUpdates();
+      return true;
+    }
+    return false;
+  }
+
+  private long addUpdateForGivenTimestamp(long ts, LocalTableState state,
+      IndexUpdateManager updateMap) throws IOException {
+    state.setCurrentTimestamp(ts);
+    ts = addCurrentStateMutationsForBatch(updateMap, state);
+    return ts;
+  }
+
+  private void addCleanupForCurrentBatch(IndexUpdateManager updateMap, long batchTs,
+      LocalTableState state) throws IOException {
+    // get the cleanup for the current state
+    state.setCurrentTimestamp(batchTs);
+    addDeleteUpdatesToMap(updateMap, state, batchTs);
+    // ignore any index tracking from the delete
+    state.resetTrackedColumns();
+  }
+  
+  /**
+   * Add the necessary mutations for the pending batch on the local state. Handles rolling up
+   * through history to determine the index changes after applying the batch (for the case where the
+   * batch is back in time).
+   * @param updateMap to update with index mutations
+   * @param batch to apply to the current state
+   * @param state current state of the table
+   * @return the minimum timestamp across all index columns requested. If
+   *         {@link ColumnTracker#isNewestTime(long)} returns <tt>true</tt> on the returned
+   *         timestamp, we know that this <i>was not a back-in-time update</i>.
+ * @throws IOException 
+   */
+  private long
+      addCurrentStateMutationsForBatch(IndexUpdateManager updateMap, LocalTableState state) throws IOException {
+
+    // get the index updates for this current batch
+    Iterable<IndexUpdate> upserts = codec.getIndexUpserts(state);
+    state.resetTrackedColumns();
+
+    /*
+     * go through all the pending updates. If we are sure that all the entries are the latest
+     * timestamp, we can just add the index updates and move on. However, if there are columns that
+     * we skip past (based on the timestamp of the batch), we need to roll back up the history.
+     * Regardless of whether or not they are the latest timestamp, the entries here are going to be
+     * correct for the current batch timestamp, so we add them to the updates. The only thing we
+     * really care about it if we need to roll up the history and fix it as we go.
+     */
+    // timestamp of the next update we need to track
+    long minTs = ColumnTracker.NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP;
+    List<IndexedColumnGroup> columnHints = new ArrayList<IndexedColumnGroup>();
+    for (IndexUpdate update : upserts) {
+      // this is the one bit where we check the timestamps
+      final ColumnTracker tracker = update.getIndexedColumns();
+      long trackerTs = tracker.getTS();
+      // update the next min TS we need to track
+      if (trackerTs < minTs) {
+        minTs = tracker.getTS();
+      }
+      // track index hints for the next round. Hint if we need an update for that column for the
+      // next timestamp. These columns clearly won't need to update as we go through time as they
+      // already match the most recent possible thing.
+      boolean needsCleanup = false;
+      if (tracker.hasNewerTimestamps()) {
+        columnHints.add(tracker);
+        // this update also needs to be cleaned up at the next timestamp because it not the latest.
+        needsCleanup = true;
+      }
+
+
+      // only make the put if the index update has been setup
+      if (update.isValid()) {
+        byte[] table = update.getTableName();
+        Mutation mutation = update.getUpdate();
+        updateMap.addIndexUpdate(table, mutation);
+
+        // only make the cleanup if we made a put and need cleanup
+        if (needsCleanup) {
+          // there is a TS for the interested columns that is greater than the columns in the
+          // put. Therefore, we need to issue a delete at the same timestamp
+          Delete d = new Delete(mutation.getRow());
+          d.setTimestamp(tracker.getTS());
+          updateMap.addIndexUpdate(table, d);
+        }
+      }
+    }
+    return minTs;
+  }
+
+  /**
+   * Cleanup the index based on the current state from the given batch. Iterates over each timestamp
+   * (for the indexed rows) for the current state of the table and cleans up all the existing
+   * entries generated by the codec.
+   * <p>
+   * Adds all pending updates to the updateMap
+   * @param updateMap updated with the pending index updates from the codec
+   * @param batchTs timestamp from which we should cleanup
+   * @param state current state of the primary table. Should already by setup to the correct state
+   *          from which we want to cleanup.
+ * @throws IOException 
+   */
+  private void cleanupIndexStateFromBatchOnward(IndexUpdateManager updateMap,
+      long batchTs, LocalTableState state) throws IOException {
+    // get the cleanup for the current state
+    state.setCurrentTimestamp(batchTs);
+    addDeleteUpdatesToMap(updateMap, state, batchTs);
+    Set<ColumnTracker> trackers = state.getTrackedColumns();
+    long minTs = ColumnTracker.NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP;
+    for (ColumnTracker tracker : trackers) {
+      if (tracker.getTS() < minTs) {
+        minTs = tracker.getTS();
+      }
+    }
+    state.resetTrackedColumns();
+    if (!ColumnTracker.isNewestTime(minTs)) {
+      state.setHints(Lists.newArrayList(trackers));
+      cleanupIndexStateFromBatchOnward(updateMap, minTs, state);
+    }
+  }
+
+
+  /**
+   * Get the index deletes from the codec {@link IndexCodec#getIndexDeletes(TableState)} and then
+   * add them to the update map.
+   * <p>
+   * Expects the {@link LocalTableState} to already be correctly setup (correct timestamp, updates
+   * applied, etc).
+ * @throws IOException 
+   */
+  protected void
+      addDeleteUpdatesToMap(IndexUpdateManager updateMap,
+      LocalTableState state, long ts) throws IOException {
+    Iterable<IndexUpdate> cleanup = codec.getIndexDeletes(state);
+    if (cleanup != null) {
+      for (IndexUpdate d : cleanup) {
+        if (!d.isValid()) {
+          continue;
+        }
+        // override the timestamps in the delete to match the current batch.
+        Delete remove = (Delete)d.getUpdate();
+        remove.setTimestamp(ts);
+        updateMap.addIndexUpdate(d.getTableName(), remove);
+      }
+    }
+  }
+
+  @Override
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Delete d) throws IOException {
+    // stores all the return values
+    IndexUpdateManager updateMap = new IndexUpdateManager();
+
+    // We have to figure out which kind of delete it is, since we need to do different things if its
+    // a general (row) delete, versus a delete of just a single column or family
+    Map<byte[], List<KeyValue>> families = d.getFamilyMap();
+
+    /*
+     * Option 1: its a row delete marker, so we just need to delete the most recent state for each
+     * group, as of the specified timestamp in the delete. This can happen if we have a single row
+     * update and it is part of a batch mutation (prepare doesn't happen until later... maybe a
+     * bug?). In a single delete, this delete gets all the column families appended, so the family
+     * map won't be empty by the time it gets here.
+     */
+    if (families.size() == 0) {
+      LocalTableState state = new LocalTableState(env, localTable, d);
+      // get a consistent view of name
+      long now = d.getTimeStamp();
+      if (now == HConstants.LATEST_TIMESTAMP) {
+        now = EnvironmentEdgeManager.currentTimeMillis();
+        // update the delete's idea of 'now' to be consistent with the index
+        d.setTimestamp(now);
+      }
+      // get deletes from the codec
+      // we only need to get deletes and not add puts because this delete covers all columns
+      addDeleteUpdatesToMap(updateMap, state, now);
+
+      /*
+       * Update the current state for all the kvs in the delete. Generally, we would just iterate
+       * the family map, but since we go here, the family map is empty! Therefore, we need to fake a
+       * bunch of family deletes (just like hos HRegion#prepareDelete works). This is just needed
+       * for current version of HBase that has an issue where the batch update doesn't update the
+       * deletes before calling the hook.
+       */
+      byte[] deleteRow = d.getRow();
+      for (byte[] family : this.env.getRegion().getTableDesc().getFamiliesKeys()) {
+        state.addPendingUpdates(new KeyValue(deleteRow, family, null, now,
+            KeyValue.Type.DeleteFamily));
+      }
+    } else {
+      // Option 2: Its actually a bunch single updates, which can have different timestamps.
+      // Therefore, we need to do something similar to the put case and batch by timestamp
+      batchMutationAndAddUpdates(updateMap, d);
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Found index updates for Delete: " + d + "\n" + updateMap);
+    }
+
+    return updateMap.toMap();
+  }
+
+  @Override
+  public Collection<Pair<Mutation, byte[]>> getIndexUpdateForFilteredRows(
+      Collection<KeyValue> filtered) throws IOException {
+    // TODO Implement IndexBuilder.getIndexUpdateForFilteredRows
+    return null;
+  }
+
+  /**
+   * Exposed for testing!
+   * @param codec codec to use for this instance of the builder
+   */
+  public void setIndexCodecForTesting(IndexCodec codec) {
+    this.codec = codec;
+  }
+
+  @Override
+  public boolean isEnabled(Mutation m) throws IOException {
+    // ask the codec to see if we should even attempt indexing
+    return this.codec.isEnabled(m);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexCodec.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexCodec.java
new file mode 100644
index 0000000..daa631b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexCodec.java
@@ -0,0 +1,110 @@
+/*
+ * 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.phoenix.hbase.index.covered;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import org.apache.phoenix.index.BaseIndexCodec;
+
+
+/**
+ * Codec for creating index updates from the current state of a table.
+ * <p>
+ * Generally, you should extend {@link BaseIndexCodec} instead, so help maintain compatibility as
+ * features need to be added to the codec, as well as potentially not haivng to implement some
+ * methods.
+ */
+public interface IndexCodec {
+
+  /**
+   * Do any code initialization necessary
+   * @param env environment in which the codec is operating
+   * @throws IOException if the codec cannot be initalized correctly
+   */
+  public void initialize(RegionCoprocessorEnvironment env) throws IOException;
+
+  /**
+   * Get the index cleanup entries. Currently, this must return just single row deletes (where just
+   * the row-key is specified and no columns are returned) mapped to the table name. For instance,
+   * to you have an index 'myIndex' with row :
+   * 
+   * <pre>
+   * v1,v2,v3 | CF:CQ0  | rowkey
+   *          | CF:CQ1  | rowkey
+   * </pre>
+   * 
+   * To then cleanup this entry, you would just return 'v1,v2,v3', 'myIndex'.
+   * @param state the current state of the table that needs to be cleaned up. Generally, you only
+   *          care about the latest column values, for each column you are indexing for each index
+   *          table.
+   * @return the pairs of (deletes, index table name) that should be applied.
+ * @throws IOException 
+   */
+  public Iterable<IndexUpdate> getIndexDeletes(TableState state) throws IOException;
+
+  // table state has the pending update already applied, before calling
+  // get the new index entries
+  /**
+   * Get the index updates for the primary table state, for each index table. The returned
+   * {@link Put}s need to be fully specified (including timestamp) to minimize passes over the same
+   * key-values multiple times.
+   * <p>
+   * You must specify the same timestamps on the Put as {@link TableState#getCurrentTimestamp()} so
+   * the index entries match the primary table row. This could be managed at a higher level, but
+   * would require iterating all the kvs in the Put again - very inefficient when compared to the
+   * current interface where you must provide a timestamp anyways (so you might as well provide the
+   * right one).
+   * @param state the current state of the table that needs to an index update Generally, you only
+   *          care about the latest column values, for each column you are indexing for each index
+   *          table.
+   * @return the pairs of (updates,index table name) that should be applied.
+ * @throws IOException 
+   */
+  public Iterable<IndexUpdate> getIndexUpserts(TableState state) throws IOException;
+
+  /**
+   * This allows the codec to dynamically change whether or not indexing should take place for a
+   * table. If it doesn't take place, we can save a lot of time on the regular Put patch. By making
+   * it dynamic, we can save offlining and then onlining a table just to turn indexing on.
+   * <p>
+   * We can also be smart about even indexing a given update here too - if the update doesn't
+   * contain any columns that we care about indexing, we can save the effort of analyzing the put
+   * and further.
+   * @param m mutation that should be indexed.
+   * @return <tt>true</tt> if indexing is enabled for the given table. This should be on a per-table
+   *         basis, as each codec is instantiated per-region.
+ * @throws IOException 
+   */
+  public boolean isEnabled(Mutation m) throws IOException;
+
+  /**
+   * Get the batch identifier of the given mutation. Generally, updates to the table will take place
+   * in a batch of updates; if we know that the mutation is part of a batch, we can build the state
+   * much more intelligently.
+   * <p>
+   * <b>If you have batches that have multiple updates to the same row state, you must specify a
+   * batch id for each batch. Otherwise, we cannot guarantee index correctness</b>
+   * @param m mutation that may or may not be part of the batch
+   * @return <tt>null</tt> if the mutation is not part of a batch or an id for the batch.
+   */
+  public byte[] getBatchId(Mutation m);
+}
\ No newline at end of file


[08/15] Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/ApplyAndFilterDeletesFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/ApplyAndFilterDeletesFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/ApplyAndFilterDeletesFilter.java
new file mode 100644
index 0000000..dbf13aa
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/ApplyAndFilterDeletesFilter.java
@@ -0,0 +1,308 @@
+/*
+ * 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.phoenix.hbase.index.covered.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.filter.FilterBase;
+
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * Only allow the 'latest' timestamp of each family:qualifier pair, ensuring that they aren't
+ * covered by a previous delete. This is similar to some of the work the ScanQueryMatcher does to
+ * ensure correct visibility of keys based on deletes.
+ * <p>
+ * No actual delete {@link KeyValue}s are allowed to pass through this filter - they are always
+ * skipped.
+ * <p>
+ * Note there is a little bit of conceptually odd behavior (though it matches the HBase
+ * specifications) around point deletes ({@link KeyValue} of type {@link Type#Delete}. These deletes
+ * only apply to a single {@link KeyValue} at a single point in time - they essentially completely
+ * 'cover' the existing {@link Put} at that timestamp. However, they don't 'cover' any other
+ * keyvalues at older timestamps. Therefore, if there is a point-delete at ts = 5, and puts at ts =
+ * 4, and ts = 5, we will only allow the put at ts = 4.
+ * <p>
+ * Expects {@link KeyValue}s to arrive in sorted order, with 'Delete' {@link Type} {@link KeyValue}s
+ * ({@link Type#DeleteColumn}, {@link Type#DeleteFamily}, {@link Type#Delete})) before their regular
+ * {@link Type#Put} counterparts.
+ */
+public class ApplyAndFilterDeletesFilter extends FilterBase {
+
+  private boolean done = false;
+  List<ImmutableBytesPtr> families;
+  private final DeleteTracker coveringDelete = new DeleteTracker();
+  private Hinter currentHint;
+  private DeleteColumnHinter columnHint = new DeleteColumnHinter();
+  private DeleteFamilyHinter familyHint = new DeleteFamilyHinter();
+  
+  /**
+   * Setup the filter to only include the given families. This allows us to seek intelligently pass
+   * families we don't care about.
+   * @param families
+   */
+  public ApplyAndFilterDeletesFilter(Set<ImmutableBytesPtr> families) {
+    this.families = new ArrayList<ImmutableBytesPtr>(families);
+    Collections.sort(this.families);
+  }
+      
+  
+  private ImmutableBytesPtr getNextFamily(ImmutableBytesPtr family) {
+    int index = Collections.binarySearch(families, family);
+    //doesn't match exactly, be we can find the right next match
+    //this is pretty unlikely, but just incase
+    if(index < 0){
+      //the actual location of the next match
+      index = -index -1;
+    }else{
+      //its an exact match for a family, so we get the next entry
+      index = index +1;
+    }
+    //now we have the location of the next entry
+    if(index >= families.size()){
+      return null;
+    }
+    return  families.get(index);
+  }
+  
+  @Override
+  public void reset(){
+    this.coveringDelete.reset();
+    this.done = false;
+  }
+  
+  
+  @Override
+  public KeyValue getNextKeyHint(KeyValue peeked){
+    return currentHint.getHint(peeked);
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue next) {
+    // we marked ourselves done, but the END_ROW_KEY didn't manage to seek to the very last key
+    if (this.done) {
+      return ReturnCode.SKIP;
+    }
+
+    switch (KeyValue.Type.codeToType(next.getType())) {
+    /*
+     * DeleteFamily will always sort first because those KVs (we assume) don't have qualifiers (or
+     * rather are null). Therefore, we have to keep a hold of all the delete families until we get
+     * to a Put entry that is covered by that delete (in which case, we are done with the family).
+     */
+    case DeleteFamily:
+      // track the family to delete. If we are updating the delete, that means we have passed all
+      // kvs in the last column, so we can safely ignore the last deleteFamily, and just use this
+      // one. In fact, it means that all the previous deletes can be ignored because the family must
+      // not match anymore.
+      this.coveringDelete.reset();
+      this.coveringDelete.deleteFamily = next;
+      return ReturnCode.SKIP;
+    case DeleteColumn:
+      // similar to deleteFamily, all the newer deletes/puts would have been seen at this point, so
+      // we can safely replace the more recent delete column with the more recent one
+      this.coveringDelete.pointDelete = null;
+      this.coveringDelete.deleteColumn = next;
+      return ReturnCode.SKIP;
+    case Delete:
+      // we are just deleting the single column value at this point.
+      // therefore we just skip this entry and go onto the next one. The only caveat is that
+      // we should still cover the next entry if this delete applies to the next entry, so we
+      // have to keep around a reference to the KV to compare against the next valid entry
+      this.coveringDelete.pointDelete = next;
+      return ReturnCode.SKIP;
+    default:
+      // no covering deletes
+      if (coveringDelete.empty()) {
+        return ReturnCode.INCLUDE;
+      }
+
+      if (coveringDelete.matchesFamily(next)) {
+        this.currentHint = familyHint;
+        return ReturnCode.SEEK_NEXT_USING_HINT;
+      }
+
+      if (coveringDelete.matchesColumn(next)) {
+        // hint to the next column
+        this.currentHint = columnHint;
+        return ReturnCode.SEEK_NEXT_USING_HINT;
+      }
+
+      if (coveringDelete.matchesPoint(next)) {
+        return ReturnCode.SKIP;
+      }
+
+    }
+
+    // none of the deletes matches, we are done
+    return ReturnCode.INCLUDE;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    throw new UnsupportedOperationException("Server-side only filter, cannot be serialized!");
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    throw new UnsupportedOperationException("Server-side only filter, cannot be deserialized!");
+  }
+
+  /**
+   * Get the next hint for a given peeked keyvalue
+   */
+  interface Hinter {
+    public abstract KeyValue getHint(KeyValue peek);
+  }
+
+  /**
+   * Entire family has been deleted, so either seek to the next family, or if none are present in
+   * the original set of families to include, seek to the "last possible key"(or rather our best
+   * guess) and be done.
+   */
+  class DeleteFamilyHinter implements Hinter {
+
+    @Override
+    public KeyValue getHint(KeyValue peeked) {
+      // check to see if we have another column to seek
+      ImmutableBytesPtr nextFamily =
+          getNextFamily(new ImmutableBytesPtr(peeked.getBuffer(), peeked.getFamilyOffset(),
+              peeked.getFamilyLength()));
+      if (nextFamily == null) {
+        // no known next family, so we can be completely done
+        done = true;
+        return KeyValue.LOWESTKEY;
+      }
+        // there is a valid family, so we should seek to that
+      return KeyValue.createFirstOnRow(peeked.getRow(), nextFamily.copyBytesIfNecessary(),
+        HConstants.EMPTY_BYTE_ARRAY);
+    }
+
+  }
+
+  /**
+   * Hint the next column-qualifier after the given keyvalue. We can't be smart like in the
+   * ScanQueryMatcher since we don't know the columns ahead of time.
+   */
+  class DeleteColumnHinter implements Hinter {
+
+    @Override
+    public KeyValue getHint(KeyValue kv) {
+      return KeyValue.createLastOnRow(kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(),
+        kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(), kv.getBuffer(),
+        kv.getQualifierOffset(), kv.getQualifierLength());
+    }
+  }
+
+  class DeleteTracker {
+
+    public KeyValue deleteFamily;
+    public KeyValue deleteColumn;
+    public KeyValue pointDelete;
+
+    public void reset() {
+      this.deleteFamily = null;
+      this.deleteColumn = null;
+      this.pointDelete = null;
+
+    }
+
+    /**
+     * Check to see if we should skip this {@link KeyValue} based on the family.
+     * <p>
+     * Internally, also resets the currently tracked "Delete Family" marker we are tracking if the
+     * keyvalue is into another family (since CFs sort lexicographically, we can discard the current
+     * marker since it must not be applicable to any more kvs in a linear scan).
+     * @param next
+     * @return <tt>true</tt> if this {@link KeyValue} matches a delete.
+     */
+    public boolean matchesFamily(KeyValue next) {
+      if (deleteFamily == null) {
+        return false;
+      }
+      if (deleteFamily.matchingFamily(next)) {
+        // falls within the timestamp range
+        if (deleteFamily.getTimestamp() >= next.getTimestamp()) {
+          return true;
+        }
+      } else {
+        // only can reset the delete family because we are on to another family
+        deleteFamily = null;
+      }
+
+      return false;
+    }
+
+
+    /**
+     * @param next
+     * @return
+     */
+    public boolean matchesColumn(KeyValue next) {
+      if (deleteColumn == null) {
+        return false;
+      }
+      if (deleteColumn.matchingFamily(next) && deleteColumn.matchingQualifier(next)) {
+        // falls within the timestamp range
+        if (deleteColumn.getTimestamp() >= next.getTimestamp()) {
+          return true;
+        }
+      } else {
+        deleteColumn = null;
+      }
+      return false;
+    }
+
+    /**
+     * @param next
+     * @return
+     */
+    public boolean matchesPoint(KeyValue next) {
+      // point deletes only apply to the exact KV that they reference, so we only need to ensure
+      // that the timestamp matches exactly. Because we sort by timestamp first, either the next
+      // keyvalue has the exact timestamp or is an older (smaller) timestamp, and we can allow that
+      // one.
+      if (pointDelete != null && pointDelete.matchingFamily(next)
+          && pointDelete.matchingQualifier(next)) {
+        if (pointDelete.getTimestamp() == next.getTimestamp()) {
+          return true;
+        }
+        // clear the point delete since the TS must not be matching
+        coveringDelete.pointDelete = null;
+      }
+      return false;
+    }
+
+    /**
+     * @return <tt>true</tt> if no delete has been set
+     */
+    public boolean empty() {
+      return deleteFamily == null && deleteColumn == null && pointDelete == null;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/ColumnTrackingNextLargestTimestampFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/ColumnTrackingNextLargestTimestampFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/ColumnTrackingNextLargestTimestampFilter.java
new file mode 100644
index 0000000..2ad04ff
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/ColumnTrackingNextLargestTimestampFilter.java
@@ -0,0 +1,72 @@
+/*
+ * 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.phoenix.hbase.index.covered.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.FilterBase;
+
+import org.apache.phoenix.hbase.index.covered.update.ColumnTracker;
+
+/**
+ * Similar to the {@link MaxTimestampFilter}, but also updates the 'next largest' timestamp seen
+ * that is not skipped by the below criteria. Note that it isn't as quick as the
+ * {@link MaxTimestampFilter} as we can't just seek ahead to a key with the matching timestamp, but
+ * have to iterate each kv until we find the right one with an allowed timestamp.
+ * <p>
+ * Inclusively filter on the maximum timestamp allowed. Excludes all elements greater than (but not
+ * equal to) the given timestamp, so given ts = 5, a {@link KeyValue} with ts 6 is excluded, but not
+ * one with ts = 5.
+ * <p>
+ * This filter generally doesn't make sense on its own - it should follow a per-column filter and
+ * possible a per-delete filter to only track the most recent (but not exposed to the user)
+ * timestamp.
+ */
+public class ColumnTrackingNextLargestTimestampFilter extends FilterBase {
+
+  private long ts;
+  private ColumnTracker column;
+
+  public ColumnTrackingNextLargestTimestampFilter(long maxTime, ColumnTracker toTrack) {
+    this.ts = maxTime;
+    this.column = toTrack;
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue v) {
+    long timestamp = v.getTimestamp();
+    if (timestamp > ts) {
+      this.column.setTs(timestamp);
+      return ReturnCode.SKIP;
+    }
+    return ReturnCode.INCLUDE;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    throw new UnsupportedOperationException("Server-side only filter, cannot be serialized!");
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    throw new UnsupportedOperationException("Server-side only filter, cannot be deserialized!");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/FamilyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/FamilyOnlyFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/FamilyOnlyFilter.java
new file mode 100644
index 0000000..080bf92
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/FamilyOnlyFilter.java
@@ -0,0 +1,80 @@
+/**
+ * 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.phoenix.hbase.index.covered.filter;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.FamilyFilter;
+import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
+
+/**
+ * Similar to the {@link FamilyFilter} but stops when the end of the family is reached and only
+ * supports equality
+ */
+public class FamilyOnlyFilter extends FamilyFilter {
+
+  boolean done = false;
+  private boolean previousMatchFound;
+
+  /**
+   * Filter on exact binary matches to the passed family
+   * @param family to compare against
+   */
+  public FamilyOnlyFilter(final byte[] family) {
+    this(new BinaryComparator(family));
+  }
+
+  public FamilyOnlyFilter(final WritableByteArrayComparable familyComparator) {
+    super(CompareOp.EQUAL, familyComparator);
+  }
+
+
+  @Override
+  public boolean filterAllRemaining() {
+    return done;
+  }
+
+  @Override
+  public void reset() {
+    done = false;
+    previousMatchFound = false;
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue v) {
+    if (done) {
+      return ReturnCode.SKIP;
+    }
+    ReturnCode code = super.filterKeyValue(v);
+    if (previousMatchFound) {
+      // we found a match before, and now we are skipping the key because of the family, therefore
+      // we are done (no more of the family).
+      if (code.equals(ReturnCode.SKIP)) {
+      done = true;
+      }
+    } else {
+      // if we haven't seen a match before, then it doesn't matter what we see now, except to mark
+      // if we've seen a match
+      if (code.equals(ReturnCode.INCLUDE)) {
+        previousMatchFound = true;
+      }
+    }
+    return code;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/MaxTimestampFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/MaxTimestampFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/MaxTimestampFilter.java
new file mode 100644
index 0000000..e8e6347
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/MaxTimestampFilter.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.phoenix.hbase.index.covered.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Inclusive filter on the maximum timestamp allowed. Excludes all elements greater than (but not
+ * equal to) the given timestamp, so given ts = 5, a {@link KeyValue} with ts 6 is excluded, but not
+ * one with ts = 5.
+ */
+public class MaxTimestampFilter extends FilterBase {
+
+  private long ts;
+
+  public MaxTimestampFilter(long maxTime) {
+    this.ts = maxTime;
+  }
+
+  @Override
+  public KeyValue getNextKeyHint(KeyValue currentKV) {
+    // this might be a little excessive right now - better safe than sorry though, so we don't mess
+    // with other filters too much.
+    KeyValue kv = currentKV.deepCopy();
+    int offset =kv.getTimestampOffset();
+    //set the timestamp in the buffer
+    byte[] buffer = kv.getBuffer();
+    byte[] ts = Bytes.toBytes(this.ts);
+    System.arraycopy(ts, 0, buffer, offset, ts.length);
+
+    return kv;
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue v) {
+    long timestamp = v.getTimestamp();
+    if (timestamp > ts) {
+      return ReturnCode.SEEK_NEXT_USING_HINT;
+    }
+    return ReturnCode.INCLUDE;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    throw new UnsupportedOperationException("Server-side only filter, cannot be serialized!");
+
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    throw new UnsupportedOperationException("Server-side only filter, cannot be deserialized!");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/NewerTimestampFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/NewerTimestampFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/NewerTimestampFilter.java
new file mode 100644
index 0000000..7209ee2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/NewerTimestampFilter.java
@@ -0,0 +1,55 @@
+/*
+ * 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.phoenix.hbase.index.covered.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.FilterBase;
+
+/**
+ * Server-side only class used in the indexer to filter out keyvalues newer than a given timestamp
+ * (so allows anything <code><=</code> timestamp through).
+ * <p>
+ * Note,<tt>this</tt> doesn't support {@link #write(DataOutput)} or {@link #readFields(DataInput)}.
+ */
+public class NewerTimestampFilter extends FilterBase {
+
+  private long timestamp;
+
+  public NewerTimestampFilter(long timestamp) {
+    this.timestamp = timestamp;
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue ignored) {
+    return ignored.getTimestamp() > timestamp ? ReturnCode.SKIP : ReturnCode.INCLUDE;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    throw new UnsupportedOperationException("TimestampFilter is server-side only!");
+  }
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    throw new UnsupportedOperationException("TimestampFilter is server-side only!");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/ColumnReference.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/ColumnReference.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/ColumnReference.java
new file mode 100644
index 0000000..ae0421d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/ColumnReference.java
@@ -0,0 +1,166 @@
+/*
+ * 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.phoenix.hbase.index.covered.update;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * 
+ */
+public class ColumnReference implements Comparable<ColumnReference> {
+    
+  public static final byte[] ALL_QUALIFIERS = new byte[0];
+  
+  private static int calcHashCode(byte[] family, byte[] qualifier) {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + Bytes.hashCode(family);
+    result = prime * result + Bytes.hashCode(qualifier);
+    return result;
+  }
+
+  private final int hashCode;
+  protected final byte[] family;
+  protected final byte[] qualifier;
+    private volatile ImmutableBytesWritable familyPtr;
+    private volatile ImmutableBytesWritable qualifierPtr;
+
+  public ColumnReference(byte[] family, byte[] qualifier) {
+    this.family = family;
+    this.qualifier = qualifier;
+    this.hashCode = calcHashCode(family, qualifier);
+  }
+
+  public byte[] getFamily() {
+    return this.family;
+  }
+
+  public byte[] getQualifier() {
+    return this.qualifier;
+  }
+  
+    public ImmutableBytesWritable getFamilyWritable() {
+        if (this.familyPtr == null) {
+            synchronized (this.family) {
+                if (this.familyPtr == null) {
+                    this.familyPtr = new ImmutableBytesPtr(this.family);
+                }
+            }
+        }
+        return this.familyPtr;
+    }
+
+    public ImmutableBytesWritable getQualifierWritable() {
+        if (this.qualifierPtr == null) {
+            synchronized (this.qualifier) {
+                if (this.qualifierPtr == null) {
+                    this.qualifierPtr = new ImmutableBytesPtr(this.qualifier);
+                }
+            }
+        }
+        return this.qualifierPtr;
+    }
+
+  public boolean matches(KeyValue kv) {
+    if (matchesFamily(kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength())) {
+      return matchesQualifier(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
+    }
+    return false;
+  }
+
+  /**
+   * @param qual to check against
+   * @return <tt>true</tt> if this column covers the given qualifier.
+   */
+  public boolean matchesQualifier(byte[] qual) {
+    return matchesQualifier(qual, 0, qual.length);
+  }
+
+  public boolean matchesQualifier(byte[] bytes, int offset, int length) {
+    return allColumns() ? true : match(bytes, offset, length, qualifier);
+  }
+
+  /**
+   * @param family to check against
+   * @return <tt>true</tt> if this column covers the given family.
+   */
+  public boolean matchesFamily(byte[] family) {
+    return matchesFamily(family, 0, family.length);
+  }
+
+  public boolean matchesFamily(byte[] bytes, int offset, int length) {
+    return match(bytes, offset, length, family);
+  }
+
+  /**
+   * @return <tt>true</tt> if this should include all column qualifiers, <tt>false</tt> otherwise
+   */
+  public boolean allColumns() {
+    return this.qualifier == ALL_QUALIFIERS;
+  }
+
+  /**
+   * Check to see if the passed bytes match the stored bytes
+   * @param first
+   * @param storedKey the stored byte[], should never be <tt>null</tt>
+   * @return <tt>true</tt> if they are byte-equal
+   */
+  private boolean match(byte[] first, int offset, int length, byte[] storedKey) {
+    return first == null ? false : Bytes.equals(first, offset, length, storedKey, 0,
+      storedKey.length);
+  }
+
+  public KeyValue getFirstKeyValueForRow(byte[] row) {
+    return KeyValue.createFirstOnRow(row, family, qualifier == ALL_QUALIFIERS ? null : qualifier);
+  }
+
+  @Override
+  public int compareTo(ColumnReference o) {
+    int c = Bytes.compareTo(family, o.family);
+    if (c == 0) {
+      // matching families, compare qualifiers
+      c = Bytes.compareTo(qualifier, o.qualifier);
+    }
+    return c;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof ColumnReference) {
+      ColumnReference other = (ColumnReference) o;
+      if (hashCode == other.hashCode && Bytes.equals(family, other.family)) {
+        return Bytes.equals(qualifier, other.qualifier);
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return hashCode;
+  }
+
+  @Override
+  public String toString() {
+    return "ColumnReference - " + Bytes.toString(family) + ":" + Bytes.toString(qualifier);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/ColumnTracker.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/ColumnTracker.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/ColumnTracker.java
new file mode 100644
index 0000000..b9f3858
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/ColumnTracker.java
@@ -0,0 +1,115 @@
+/*
+ * 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.phoenix.hbase.index.covered.update;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+
+/**
+ * Simple POJO for tracking a bunch of column references and the next-newest timestamp for those
+ * columns
+ * <p>
+ * Two {@link ColumnTracker}s are considered equal if they track the same columns, even if their
+ * timestamps are different.
+ */
+public class ColumnTracker implements IndexedColumnGroup {
+
+  public static final long NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP = Long.MAX_VALUE;
+  public static final long GUARANTEED_NEWER_UPDATES = Long.MIN_VALUE;
+  private final List<ColumnReference> columns;
+  private long ts = NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP;
+  private final int hashCode;
+
+  private static int calcHashCode(List<ColumnReference> columns) {
+      return columns.hashCode();
+    }
+
+  public ColumnTracker(Collection<? extends ColumnReference> columns) {
+    this.columns = new ArrayList<ColumnReference>(columns);
+    // sort the columns
+    Collections.sort(this.columns);
+    this.hashCode = calcHashCode(this.columns);
+  }
+
+  /**
+   * Set the current timestamp, only if the passed timestamp is strictly less than the currently
+   * stored timestamp
+   * @param ts the timestmap to potentially store.
+   * @return the currently stored timestamp.
+   */
+  public long setTs(long ts) {
+    this.ts = this.ts > ts ? ts : this.ts;
+    return this.ts;
+  }
+
+  public long getTS() {
+    return this.ts;
+  }
+
+  @Override
+  public int hashCode() {
+    return hashCode;
+  }
+
+  @Override
+  public boolean equals(Object o){
+    if(!(o instanceof ColumnTracker)){
+      return false;
+    }
+    ColumnTracker other = (ColumnTracker)o;
+    if (hashCode != other.hashCode) {
+        return false;
+    }
+    if (other.columns.size() != columns.size()) {
+      return false;
+    }
+
+    // check each column to see if they match
+    for (int i = 0; i < columns.size(); i++) {
+      if (!columns.get(i).equals(other.columns.get(i))) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  @Override
+  public List<ColumnReference> getColumns() {
+    return this.columns;
+  }
+
+  /**
+   * @return <tt>true</tt> if this set of columns has seen a column with a timestamp newer than the
+   *         requested timestamp, <tt>false</tt> otherwise.
+   */
+  public boolean hasNewerTimestamps() {
+    return !isNewestTime(this.ts);
+  }
+
+  /**
+   * @param ts timestamp to check
+   * @return <tt>true</tt> if the timestamp is at the most recent timestamp for a column
+   */
+  public static boolean isNewestTime(long ts) {
+    return ts == NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/IndexUpdateManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/IndexUpdateManager.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/IndexUpdateManager.java
new file mode 100644
index 0000000..c8a188d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/IndexUpdateManager.java
@@ -0,0 +1,239 @@
+/*
+ * 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.phoenix.hbase.index.covered.update;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Longs;
+
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+
+/**
+ * Keeps track of the index updates
+ */
+public class IndexUpdateManager {
+
+  public Comparator<Mutation> COMPARATOR = new MutationComparator();
+  class MutationComparator implements Comparator<Mutation> {
+
+    @Override
+    public int compare(Mutation o1, Mutation o2) {
+      // always sort rows first
+      int compare = o1.compareTo(o2);
+      if (compare != 0) {
+        return compare;
+      }
+
+      // if same row, sort by reverse timestamp (larger first)
+      compare = Longs.compare(o2.getTimeStamp(), o1.getTimeStamp());
+      if (compare != 0) {
+        return compare;
+      }
+      // deletes always sort before puts for the same row
+      if (o1 instanceof Delete) {
+        // same row, same ts == same delete since we only delete rows
+        if (o2 instanceof Delete) {
+          return 0;
+        } else {
+          // o2 has to be a put
+          return -1;
+        }
+      }
+      // o1 must be a put
+      if (o2 instanceof Delete) {
+        return 1;
+      } else if (o2 instanceof Put) {
+        return comparePuts((Put) o1, (Put) o2);
+      }
+
+      throw new RuntimeException(
+          "Got unexpected mutation types! Can only be Put or Delete, but got: " + o1 + ", and "
+              + o2);
+    }
+
+    private int comparePuts(Put p1, Put p2) {
+      int p1Size = p1.size();
+      int p2Size = p2.size();
+      int compare = p1Size - p2Size;
+      if (compare == 0) {
+        // TODO: make this a real comparison
+        // this is a little cheating, but we don't really need to worry too much about this being
+        // the same - chances are that exact matches here are really the same update.
+        return Longs.compare(p1.heapSize(), p2.heapSize());
+      }
+      return compare;
+    }
+
+  }
+
+  private static final String PHOENIX_HBASE_TEMP_DELETE_MARKER = "phoenix.hbase.temp.delete.marker";
+  private static final byte[] TRUE_MARKER = new byte[] { 1 };
+
+  protected final Map<ImmutableBytesPtr, Collection<Mutation>> map =
+      new HashMap<ImmutableBytesPtr, Collection<Mutation>>();
+
+  /**
+   * Add an index update. Keeps the latest {@link Put} for a given timestamp
+   * @param tableName
+   * @param m
+   */
+  public void addIndexUpdate(byte[] tableName, Mutation m) {
+    // we only keep the most recent update
+    ImmutableBytesPtr key = new ImmutableBytesPtr(tableName);
+    Collection<Mutation> updates = map.get(key);
+    if (updates == null) {
+      updates = new SortedCollection<Mutation>(COMPARATOR);
+      map.put(key, updates);
+    }
+    fixUpCurrentUpdates(updates, m);
+  }
+
+  /**
+   * Fix up the current updates, given the pending mutation.
+   * @param updates current updates
+   * @param pendingMutation
+   */
+  protected void fixUpCurrentUpdates(Collection<Mutation> updates, Mutation pendingMutation) {
+    // need to check for each entry to see if we have a duplicate
+    Mutation toRemove = null;
+    Delete pendingDelete = pendingMutation instanceof Delete ? (Delete) pendingMutation : null;
+    boolean sawRowMatch = false;
+    for (Mutation stored : updates) {
+      int compare = pendingMutation.compareTo(stored);
+      // skip to the right row
+      if (compare < 0) {
+        continue;
+      } else if (compare > 0) {
+        if (sawRowMatch) {
+          break;
+        }
+        continue;
+      }
+
+      // set that we saw a row match, so any greater row will necessarily be the wrong
+      sawRowMatch = true;
+
+      // skip until we hit the right timestamp
+      if (stored.getTimeStamp() < pendingMutation.getTimeStamp()) {
+        continue;
+      }
+
+      if (stored instanceof Delete) {
+        // we already have a delete for this row, so we are done.
+        if (pendingDelete != null) {
+          return;
+        }
+        // pending update must be a Put, so we ignore the Put.
+        // add a marker in the this delete that it has been canceled out already. We need to keep
+        // the delete around though so we can figure out if other Puts would also be canceled out.
+        markMutationForRemoval(stored);
+        return;
+      }
+
+      // otherwise, the stored mutation is a Put. Either way, we want to remove it. If the pending
+      // update is a delete, we need to remove the entry (no longer applies - covered by the
+      // delete), or its an older version of the row, so we cover it with the newer.
+      toRemove = stored;
+      if (pendingDelete != null) {
+        // the pending mutation, but we need to mark the mutation for removal later
+        markMutationForRemoval(pendingMutation);
+        break;
+      }
+    }
+    
+    updates.remove(toRemove);
+    updates.add(pendingMutation);
+  }
+
+  private void markMutationForRemoval(Mutation m) {
+    m.setAttribute(PHOENIX_HBASE_TEMP_DELETE_MARKER, TRUE_MARKER);
+  }
+
+  public List<Pair<Mutation, byte[]>> toMap() {
+    List<Pair<Mutation, byte[]>> updateMap = Lists.newArrayList();
+    for (Entry<ImmutableBytesPtr, Collection<Mutation>> updates : map.entrySet()) {
+      // get is ok because we always set with just the bytes
+      byte[] tableName = updates.getKey().get();
+      // TODO replace this as just storing a byte[], to avoid all the String <-> byte[] swapping
+      // HBase does
+      for (Mutation m : updates.getValue()) {
+        // skip elements that have been marked for delete
+        if (shouldBeRemoved(m)) {
+          continue;
+        }
+        updateMap.add(new Pair<Mutation, byte[]>(m, tableName));
+      }
+    }
+    return updateMap;
+  }
+
+  /**
+   * @param updates
+   */
+  public void addAll(Collection<Pair<Mutation, String>> updates) {
+    for (Pair<Mutation, String> update : updates) {
+      addIndexUpdate(Bytes.toBytes(update.getSecond()), update.getFirst());
+    }
+  }
+
+  private boolean shouldBeRemoved(Mutation m) {
+    return m.getAttribute(PHOENIX_HBASE_TEMP_DELETE_MARKER) != null;
+  }
+
+  @Override
+  public String toString() {
+    StringBuffer sb = new StringBuffer("Pending Index Updates:\n");
+    for (Entry<ImmutableBytesPtr, Collection<Mutation>> entry : map.entrySet()) {
+      String tableName = Bytes.toString(entry.getKey().get());
+      sb.append("   Table: '" + tableName + "'\n");
+      for (Mutation m : entry.getValue()) {
+        sb.append("\t");
+        if (shouldBeRemoved(m)) {
+          sb.append("[REMOVED]");
+        }
+        sb.append(m.getClass().getSimpleName() + ":"
+            + ((m instanceof Put) ? m.getTimeStamp() + " " : ""));
+        sb.append(" row=" + Bytes.toString(m.getRow()));
+        sb.append("\n");
+        if (m.getFamilyMap().isEmpty()) {
+          sb.append("\t\t=== EMPTY ===\n");
+        }
+        for (List<KeyValue> kvs : m.getFamilyMap().values()) {
+          for (KeyValue kv : kvs) {
+            sb.append("\t\t" + kv.toString() + "/value=" + Bytes.toStringBinary(kv.getValue()));
+            sb.append("\n");
+          }
+        }
+      }
+    }
+    return sb.toString();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/IndexedColumnGroup.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/IndexedColumnGroup.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/IndexedColumnGroup.java
new file mode 100644
index 0000000..3c98f0f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/IndexedColumnGroup.java
@@ -0,0 +1,28 @@
+/*
+ * 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.phoenix.hbase.index.covered.update;
+
+import java.util.List;
+
+/**
+ * Group of columns that were requested to build an index
+ */
+public interface IndexedColumnGroup {
+
+  public List<ColumnReference> getColumns();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/SortedCollection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/SortedCollection.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/SortedCollection.java
new file mode 100644
index 0000000..ee8b453
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/update/SortedCollection.java
@@ -0,0 +1,128 @@
+/*
+ * 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.phoenix.hbase.index.covered.update;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.PriorityQueue;
+
+import com.google.common.collect.Iterators;
+
+/**
+ * A collection whose elements are stored and returned sorted.
+ * <p>
+ * We can't just use something like a {@link PriorityQueue} because it doesn't return the
+ * underlying values in sorted order.
+ * @param <T>
+ */
+class SortedCollection<T> implements Collection<T>, Iterable<T> {
+
+  private PriorityQueue<T> queue;
+  private Comparator<T> comparator;
+
+  /**
+   * Use the given comparator to compare all keys for sorting
+   * @param comparator
+   */
+  public SortedCollection(Comparator<T> comparator) {
+    this.queue = new PriorityQueue<T>(1, comparator);
+    this.comparator = comparator;
+  }
+  
+  /**
+   * All passed elements are expected to be {@link Comparable}
+   */
+  public SortedCollection() {
+    this.queue = new PriorityQueue<T>();
+  }
+  
+  @Override
+  public int size() {
+    return this.queue.size();
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return this.queue.isEmpty();
+  }
+
+  @Override
+  public boolean contains(Object o) {
+    return this.queue.contains(o);
+  }
+
+  @Override
+  public Iterator<T> iterator() {
+    @SuppressWarnings("unchecked")
+    T[] array = (T[]) this.queue.toArray();
+    if (this.comparator == null) {
+      Arrays.sort(array);
+    } else {
+      Arrays.sort(
+     array, this.comparator);}
+    return Iterators.forArray(array);
+  }
+
+  @Override
+  public Object[] toArray() {
+    return this.queue.toArray();
+  }
+
+  @SuppressWarnings("hiding")
+  @Override
+  public <T> T[] toArray(T[] a) {
+    return this.queue.toArray(a);
+  }
+
+  @Override
+  public boolean add(T e) {
+    return this.queue.add(e);
+  }
+
+  @Override
+  public boolean remove(Object o) {
+    return this.queue.remove(o);
+  }
+
+  @Override
+  public boolean containsAll(Collection<?> c) {
+    return this.queue.containsAll(c);
+  }
+
+  @Override
+  public boolean addAll(Collection<? extends T> c) {
+    return this.queue.addAll(c);
+  }
+
+  @Override
+  public boolean removeAll(Collection<?> c) {
+    return queue.removeAll(c);
+  }
+
+  @Override
+  public boolean retainAll(Collection<?> c) {
+    return this.queue.retainAll(c);
+  }
+
+  @Override
+  public void clear() {
+    this.queue.clear();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/exception/IndexWriteException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/exception/IndexWriteException.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/exception/IndexWriteException.java
new file mode 100644
index 0000000..2ec29bc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/exception/IndexWriteException.java
@@ -0,0 +1,43 @@
+/*
+ * 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.phoenix.hbase.index.exception;
+
+import org.apache.hadoop.hbase.HBaseIOException;
+
+/**
+ * Generic {@link Exception} that an index write has failed
+ */
+@SuppressWarnings("serial")
+public class IndexWriteException extends HBaseIOException {
+
+  public IndexWriteException() {
+    super();
+  }
+
+  public IndexWriteException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public IndexWriteException(String message) {
+    super(message);
+  }
+
+  public IndexWriteException(Throwable cause) {
+    super(cause);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/exception/MultiIndexWriteFailureException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/exception/MultiIndexWriteFailureException.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/exception/MultiIndexWriteFailureException.java
new file mode 100644
index 0000000..546b43d
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/exception/MultiIndexWriteFailureException.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.hbase.index.exception;
+
+import java.util.List;
+
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
+
+/**
+ * Indicate a failure to write to multiple index tables.
+ */
+@SuppressWarnings("serial")
+public class MultiIndexWriteFailureException extends IndexWriteException {
+
+  private List<HTableInterfaceReference> failures;
+
+  /**
+   * @param failures the tables to which the index write did not succeed
+   */
+  public MultiIndexWriteFailureException(List<HTableInterfaceReference> failures) {
+    super("Failed to write to multiple index tables");
+    this.failures = failures;
+
+  }
+
+  public List<HTableInterfaceReference> getFailedTables() {
+    return this.failures;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/exception/SingleIndexWriteFailureException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/exception/SingleIndexWriteFailureException.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/exception/SingleIndexWriteFailureException.java
new file mode 100644
index 0000000..eb3b521
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/exception/SingleIndexWriteFailureException.java
@@ -0,0 +1,61 @@
+/*
+ * 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.phoenix.hbase.index.exception;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Mutation;
+
+/**
+ * Exception thrown if we cannot successfully write to an index table.
+ */
+@SuppressWarnings("serial")
+public class SingleIndexWriteFailureException extends IndexWriteException {
+
+  private String table;
+
+  /**
+   * Cannot reach the index, but not sure of the table or the mutations that caused the failure
+   * @param msg more description of what happened
+   * @param cause original cause
+   */
+  public SingleIndexWriteFailureException(String msg, Throwable cause) {
+    super(msg, cause);
+  }
+
+  /**
+   * Failed to write the passed mutations to an index table for some reason.
+   * @param targetTableName index table to which we attempted to write
+   * @param mutations mutations that were attempted
+   * @param cause underlying reason for the failure
+   */
+  public SingleIndexWriteFailureException(String targetTableName, List<Mutation> mutations,
+      Exception cause) {
+    super("Failed to make index update:\n\t table: " + targetTableName + "\n\t edits: " + mutations
+        + "\n\tcause: " + cause == null ? "UNKNOWN" : cause.getMessage(), cause);
+    this.table = targetTableName;
+  }
+
+  /**
+   * @return The table to which we failed to write the index updates. If unknown, returns
+   *         <tt>null</tt>
+   */
+  public String getTableName() {
+    return this.table;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/BaseTaskRunner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/BaseTaskRunner.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/BaseTaskRunner.java
new file mode 100644
index 0000000..5cd3fcb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/BaseTaskRunner.java
@@ -0,0 +1,129 @@
+/*
+ * 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.phoenix.hbase.index.parallel;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Abortable;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+
+/**
+ * {@link TaskRunner} that just manages the underlying thread pool. On called to
+ * {@link #stop(String)}, the thread pool is shutdown immediately - all pending tasks are cancelled
+ * and running tasks receive and interrupt.
+ * <p>
+ * If we find a failure the failure is propagated to the {@link TaskBatch} so any {@link Task} that
+ * is interested can kill itself as well.
+ */
+public abstract class BaseTaskRunner implements TaskRunner {
+
+  private static final Log LOG = LogFactory.getLog(BaseTaskRunner.class);
+  protected ListeningExecutorService writerPool;
+  private boolean stopped;
+
+  public BaseTaskRunner(ExecutorService service) {
+    this.writerPool = MoreExecutors.listeningDecorator(service);
+  }
+
+  @Override
+  public <R> List<R> submit(TaskBatch<R> tasks) throws CancellationException, ExecutionException,
+      InterruptedException {
+    // submit each task to the pool and queue it up to be watched
+    List<ListenableFuture<R>> futures = new ArrayList<ListenableFuture<R>>(tasks.size());
+    for (Task<R> task : tasks.getTasks()) {
+      futures.add(this.writerPool.submit(task));
+    }
+    try {
+      // This logic is actually much more synchronized than the previous logic. Now we rely on a
+      // synchronization around the status to tell us when we are done. While this does have the
+      // advantage of being (1) less code, and (2) supported as part of a library, it is just that
+      // little bit slower. If push comes to shove, we can revert back to the previous
+      // implementation, but for right now, this works just fine.
+      return submitTasks(futures).get();
+    } catch (CancellationException e) {
+      // propagate the failure back out
+      logAndNotifyAbort(e, tasks);
+      throw e;
+    } catch (ExecutionException e) {
+      // propagate the failure back out
+      logAndNotifyAbort(e, tasks);
+      throw e;
+    }
+  }
+
+  private void logAndNotifyAbort(Exception e, Abortable abort) {
+    String msg = "Found a failed task because: " + e.getMessage();
+    LOG.error(msg, e);
+    abort.abort(msg, e.getCause());
+  }
+
+  /**
+   * Build a ListenableFuture for the tasks. Implementing classes can determine return behaviors on
+   * the given tasks
+   * @param futures to wait on
+   * @return a single {@link ListenableFuture} that completes based on the passes tasks.
+   */
+  protected abstract <R> ListenableFuture<List<R>> submitTasks(List<ListenableFuture<R>> futures);
+
+  @Override
+  public <R> List<R> submitUninterruptible(TaskBatch<R> tasks) throws EarlyExitFailure,
+      ExecutionException {
+    boolean interrupted = false;
+    try {
+      while (!this.isStopped()) {
+        try {
+          return this.submit(tasks);
+        } catch (InterruptedException e) {
+          interrupted = true;
+        }
+      }
+    } finally {
+      // restore the interrupted status
+      if (interrupted) {
+        Thread.currentThread().interrupt();
+      }
+    }
+
+    // should only get here if we are interrupted while waiting for a result and have been told to
+    // shutdown by an external source
+    throw new EarlyExitFailure("Interrupted and stopped before computation was complete!");
+  }
+
+  @Override
+  public void stop(String why) {
+    if (this.stopped) {
+      return;
+    }
+    LOG.info("Shutting down task runner because " + why);
+    this.writerPool.shutdownNow();
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/EarlyExitFailure.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/EarlyExitFailure.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/EarlyExitFailure.java
new file mode 100644
index 0000000..8a0dedc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/EarlyExitFailure.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.phoenix.hbase.index.parallel;
+
+import java.io.IOException;
+
+/**
+ * Exception denoting a need to early-exit a task (or group of tasks) due to external notification
+ */
+@SuppressWarnings("serial")
+public class EarlyExitFailure extends IOException {
+
+  /**
+   * @param msg reason for the early exit
+   */
+  public EarlyExitFailure(String msg) {
+    super(msg);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/QuickFailingTaskRunner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/QuickFailingTaskRunner.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/QuickFailingTaskRunner.java
new file mode 100644
index 0000000..5b9717e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/QuickFailingTaskRunner.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.phoenix.hbase.index.parallel;
+
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+
+/**
+ * {@link TaskRunner} that attempts to run all tasks passed, but quits early if any {@link Task}
+ * fails, not waiting for the remaining {@link Task}s to complete.
+ */
+public class QuickFailingTaskRunner extends BaseTaskRunner {
+
+  static final Log LOG = LogFactory.getLog(QuickFailingTaskRunner.class);
+
+  /**
+   * @param service thread pool to which {@link Task}s are submitted. This service is then 'owned'
+   *          by <tt>this</tt> and will be shutdown on calls to {@link #stop(String)}.
+   */
+  public QuickFailingTaskRunner(ExecutorService service) {
+    super(service);
+  }
+
+  @Override
+  protected <R> ListenableFuture<List<R>> submitTasks(List<ListenableFuture<R>> futures) {
+    return Futures.allAsList(futures);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/Task.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/Task.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/Task.java
new file mode 100644
index 0000000..4b32e71
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/Task.java
@@ -0,0 +1,40 @@
+/*
+ * 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.phoenix.hbase.index.parallel;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hbase.Abortable;
+
+/**
+ * Like a {@link Callable}, but supports an internal {@link Abortable} that can be checked
+ * periodically to determine if the batch should abort
+ * @param <V> expected result of the task
+ */
+public abstract class Task<V> implements Callable<V> {
+
+  private Abortable batch;
+
+  void setBatchMonitor(Abortable abort) {
+    this.batch = abort;
+  }
+
+  protected boolean isBatchFailed() {
+    return this.batch.isAborted();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/TaskBatch.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/TaskBatch.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/TaskBatch.java
new file mode 100644
index 0000000..62e4522
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/TaskBatch.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.phoenix.hbase.index.parallel;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Abortable;
+
+/**
+ * A group of {@link Task}s. The tasks are all bound together using the same {@link Abortable} (
+ * <tt>this</tt>) to ensure that all tasks are aware when any of the other tasks fails.
+ * @param <V> expected result type from all the tasks
+ */
+public class TaskBatch<V> implements Abortable {
+  private static final Log LOG = LogFactory.getLog(TaskBatch.class);
+  private AtomicBoolean aborted = new AtomicBoolean();
+  private List<Task<V>> tasks;
+
+  /**
+   * @param size expected number of tasks
+   */
+  public TaskBatch(int size) {
+    this.tasks = new ArrayList<Task<V>>(size);
+  }
+
+  public void add(Task<V> task) {
+    this.tasks.add(task);
+    task.setBatchMonitor(this);
+  }
+
+  public Collection<Task<V>> getTasks() {
+    return this.tasks;
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+    if (this.aborted.getAndSet(true)) {
+      return;
+    }
+    LOG.info("Aborting batch of tasks because " + why);
+  }
+
+  @Override
+  public boolean isAborted() {
+    return this.aborted.get();
+  }
+
+  /**
+   * @return the number of tasks assigned to this batch
+   */
+  public int size() {
+    return this.tasks.size();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/TaskRunner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/TaskRunner.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/TaskRunner.java
new file mode 100644
index 0000000..003e18f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/TaskRunner.java
@@ -0,0 +1,60 @@
+/*
+ * 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.phoenix.hbase.index.parallel;
+
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.hadoop.hbase.Stoppable;
+
+/**
+ *
+ */
+public interface TaskRunner extends Stoppable {
+
+  /**
+   * Submit the given tasks to the pool and wait for them to complete. fail.
+   * <p>
+   * Non-interruptible method. To stop any running tasks call {@link #stop(String)} - this will
+   * shutdown the thread pool, causing any pending tasks to be failed early (whose failure will be
+   * ignored) and interrupt any running tasks. It is up to the passed tasks to respect the interrupt
+   * notification
+   * @param tasks to run
+   * @return the result from each task
+   * @throws ExecutionException if any of the tasks fails. Wraps the underyling failure, which can
+   *           be retrieved via {@link ExecutionException#getCause()}.
+   * @throws InterruptedException if the current thread is interrupted while waiting for the batch
+   *           to complete
+   */
+  public <R> List<R> submit(TaskBatch<R> tasks) throws
+      ExecutionException, InterruptedException;
+
+  /**
+   * Similar to {@link #submit(TaskBatch)}, but is not interruptible. If an interrupt is found while
+   * waiting for results, we ignore it and only stop is {@link #stop(String)} has been called. On
+   * return from the method, the interrupt status of the thread is restored.
+   * @param tasks to run
+   * @return the result from each task
+   * @throws EarlyExitFailure if there are still tasks to submit to the pool, but there is a stop
+   *           notification
+   * @throws ExecutionException if any of the tasks fails. Wraps the underyling failure, which can
+   *           be retrieved via {@link ExecutionException#getCause()}.
+   */
+  public <R> List<R> submitUninterruptible(TaskBatch<R> tasks) throws EarlyExitFailure,
+      ExecutionException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/ThreadPoolBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/ThreadPoolBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/ThreadPoolBuilder.java
new file mode 100644
index 0000000..58a976a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/ThreadPoolBuilder.java
@@ -0,0 +1,90 @@
+/*
+ * 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.phoenix.hbase.index.parallel;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * Helper utility to make a thread pool from a configuration based on reasonable defaults and passed
+ * configuration keys.
+ */
+public class ThreadPoolBuilder {
+
+  private static final Log LOG = LogFactory.getLog(ThreadPoolBuilder.class);
+  private static final long DEFAULT_TIMEOUT = 60;
+  private static final int DEFAULT_MAX_THREADS = 1;// is there a better default?
+  private Pair<String, Long> timeout;
+  private Pair<String, Integer> maxThreads;
+  private String name;
+  private Configuration conf;
+
+  public ThreadPoolBuilder(String poolName, Configuration conf) {
+    this.name = poolName;
+    this.conf = conf;
+  }
+
+  public ThreadPoolBuilder setCoreTimeout(String confkey, long defaultTime) {
+    if (defaultTime <= 0) {
+      defaultTime = DEFAULT_TIMEOUT;
+    }
+    this.timeout = new Pair<String, Long>(confkey, defaultTime);
+    return this;
+  }
+
+  public ThreadPoolBuilder setCoreTimeout(String confKey) {
+    return this.setCoreTimeout(confKey, DEFAULT_TIMEOUT);
+  }
+
+  public ThreadPoolBuilder setMaxThread(String confkey, int defaultThreads) {
+    if (defaultThreads <= 0) {
+      defaultThreads = DEFAULT_MAX_THREADS;
+    }
+    this.maxThreads = new Pair<String, Integer>(confkey, defaultThreads);
+    return this;
+  }
+
+  String getName() {
+   return this.name;
+  }
+
+  int getMaxThreads() {
+    int maxThreads = DEFAULT_MAX_THREADS;
+    if (this.maxThreads != null) {
+      String key = this.maxThreads.getFirst();
+      maxThreads =
+          key == null ? this.maxThreads.getSecond() : conf.getInt(key, this.maxThreads.getSecond());
+    }
+    LOG.trace("Creating pool builder with max " + maxThreads + " threads ");
+    return maxThreads;
+  }
+
+  long getKeepAliveTime() {
+    long timeout =DEFAULT_TIMEOUT;
+    if (this.timeout != null) {
+      String key = this.timeout.getFirst();
+      timeout =
+          key == null ? this.timeout.getSecond() : conf.getLong(key, this.timeout.getSecond());
+    }
+
+    LOG.trace("Creating pool builder with core thread timeout of " + timeout + " seconds ");
+    return timeout;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/ThreadPoolManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/ThreadPoolManager.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/ThreadPoolManager.java
new file mode 100644
index 0000000..efde03e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/ThreadPoolManager.java
@@ -0,0 +1,146 @@
+/*
+ * 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.phoenix.hbase.index.parallel;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.util.Threads;
+
+/**
+ * Manage access to thread pools
+ */
+public class ThreadPoolManager {
+
+  private static final Log LOG = LogFactory.getLog(ThreadPoolManager.class);
+
+  /**
+   * Get an executor for the given name, based on the passed {@link Configuration}. If a thread pool
+   * already exists with that name, it will be returned.
+   * @param builder
+   * @param env
+   * @return a {@link ThreadPoolExecutor} for the given name. Thread pool that only shuts down when
+   *         there are no more explicit references to it. You do not need to shutdown the threadpool
+   *         on your own - it is managed for you. When you are done, you merely need to release your
+   *         reference. If you do attempt to shutdown the pool, you should be careful to call
+   *         {@link ThreadPoolExecutor#shutdown()} XOR {@link ThreadPoolExecutor#shutdownNow()} - extra calls to either can lead to
+   *         early shutdown of the pool.
+   */
+  public static synchronized ThreadPoolExecutor getExecutor(ThreadPoolBuilder builder,
+      RegionCoprocessorEnvironment env) {
+    return getExecutor(builder, env.getSharedData());
+  }
+
+  static synchronized ThreadPoolExecutor getExecutor(ThreadPoolBuilder builder,
+      Map<String, Object> poolCache) {
+    ThreadPoolExecutor pool = (ThreadPoolExecutor) poolCache.get(builder.getName());
+    if (pool == null || pool.isTerminating() || pool.isShutdown()) {
+      pool = getDefaultExecutor(builder);
+      LOG.info("Creating new pool for " + builder.getName());
+      poolCache.put(builder.getName(), pool);
+    }
+    ((ShutdownOnUnusedThreadPoolExecutor) pool).addReference();
+
+    return pool;
+  }
+
+  /**
+   * @param conf
+   * @return
+   */
+  private static ShutdownOnUnusedThreadPoolExecutor getDefaultExecutor(ThreadPoolBuilder builder) {
+    int maxThreads = builder.getMaxThreads();
+    long keepAliveTime = builder.getKeepAliveTime();
+
+    // we prefer starting a new thread to queuing (the opposite of the usual ThreadPoolExecutor)
+    // since we are probably writing to a bunch of index tables in this case. Any pending requests
+    // are then queued up in an infinite (Integer.MAX_VALUE) queue. However, we allow core threads
+    // to timeout, to we tune up/down for bursty situations. We could be a bit smarter and more
+    // closely manage the core-thread pool size to handle the bursty traffic (so we can always keep
+    // some core threads on hand, rather than starting from scratch each time), but that would take
+    // even more time. If we shutdown the pool, but are still putting new tasks, we can just do the
+    // usual policy and throw a RejectedExecutionException because we are shutting down anyways and
+    // the worst thing is that this gets unloaded.
+    ShutdownOnUnusedThreadPoolExecutor pool =
+        new ShutdownOnUnusedThreadPoolExecutor(maxThreads, maxThreads, keepAliveTime,
+            TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(),
+            Threads.newDaemonThreadFactory(builder.getName() + "-"), builder.getName());
+    pool.allowCoreThreadTimeOut(true);
+    return pool;
+  }
+
+  /**
+   * Thread pool that only shuts down when there are no more explicit references to it. A reference
+   * is when obtained and released on calls to {@link #shutdown()} or {@link #shutdownNow()}.
+   * Therefore, users should be careful to call {@link #shutdown()} XOR {@link #shutdownNow()} -
+   * extra calls to either can lead to early shutdown of the pool.
+   */
+  private static class ShutdownOnUnusedThreadPoolExecutor extends ThreadPoolExecutor {
+
+    private AtomicInteger references;
+    private String poolName;
+
+    public ShutdownOnUnusedThreadPoolExecutor(int coreThreads, int maxThreads, long keepAliveTime,
+        TimeUnit timeUnit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory,
+        String poolName) {
+      super(coreThreads, maxThreads, keepAliveTime, timeUnit, workQueue, threadFactory);
+      this.references = new AtomicInteger();
+      this.poolName = poolName;
+    }
+
+    public void addReference() {
+      this.references.incrementAndGet();
+    }
+
+    @Override
+    protected void finalize() {
+      // override references counter if we go out of scope - ensures the pool gets cleaned up
+      LOG.info("Shutting down pool '" + poolName + "' because no more references");
+      super.finalize();
+    }
+
+    @Override
+    public void shutdown() {
+      if (references.decrementAndGet() <= 0) {
+        LOG.debug("Shutting down pool " + this.poolName);
+        super.shutdown();
+      }
+    }
+
+    @Override
+    public List<Runnable> shutdownNow() {
+      if (references.decrementAndGet() <= 0) {
+        LOG.debug("Shutting down pool " + this.poolName + " NOW!");
+        return super.shutdownNow();
+      }
+      return Collections.emptyList();
+    }
+
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/WaitForCompletionTaskRunner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/WaitForCompletionTaskRunner.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/WaitForCompletionTaskRunner.java
new file mode 100644
index 0000000..2cc5bf6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/parallel/WaitForCompletionTaskRunner.java
@@ -0,0 +1,51 @@
+/*
+ * 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.phoenix.hbase.index.parallel;
+
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+
+/**
+ * A {@link TaskRunner} that ensures that all the tasks have been attempted before we return, even
+ * if some of the tasks cause failures.
+ * <p>
+ * Because we wait until the entire batch is complete to see the failure, checking for failure of
+ * the {@link TaskBatch} on the submitted tasks will not help - they will never see the failure of
+ * the other tasks. You will need to provide an external mechanism to propagate the error.
+ * <p>
+ * Does not throw an {@link ExecutionException} if any of the tasks fail.
+ */
+public class WaitForCompletionTaskRunner extends BaseTaskRunner {
+  
+  /**
+   * @param service thread pool to which {@link Task}s are submitted. This service is then 'owned'
+   *          by <tt>this</tt> and will be shutdown on calls to {@link #stop(String)}.
+   */
+  public WaitForCompletionTaskRunner(ExecutorService service) {
+    super(service);
+  }
+
+  @Override
+  public <R> ListenableFuture<List<R>> submitTasks(List<ListenableFuture<R>> futures) {
+    return Futures.successfulAsList(futures);
+  }
+}
\ No newline at end of file


[13/15] Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexCodec.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexCodec.java
deleted file mode 100644
index 8f0ee99..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexCodec.java
+++ /dev/null
@@ -1,367 +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.hadoop.hbase.index.covered.example;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map.Entry;
-
-import org.apache.commons.lang.ArrayUtils;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-
-import com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.index.covered.IndexUpdate;
-import org.apache.hadoop.hbase.index.covered.TableState;
-import org.apache.hadoop.hbase.index.scanner.Scanner;
-import org.apache.phoenix.index.BaseIndexCodec;
-
-/**
- *
- */
-public class CoveredColumnIndexCodec extends BaseIndexCodec {
-
-  private static final byte[] EMPTY_BYTES = new byte[0];
-  public static final byte[] INDEX_ROW_COLUMN_FAMILY = Bytes.toBytes("INDEXED_COLUMNS");
-
-  private List<ColumnGroup> groups;
-
-  /**
-   * @param groups to initialize the codec with
-   * @return an instance that is initialized with the given {@link ColumnGroup}s, for testing
-   *         purposes
-   */
-  public static CoveredColumnIndexCodec getCodecForTesting(List<ColumnGroup> groups) {
-    CoveredColumnIndexCodec codec = new CoveredColumnIndexCodec();
-    codec.groups = Lists.newArrayList(groups);
-    return codec;
-  }
-
-  @Override
-  public void initialize(RegionCoprocessorEnvironment env) {
-    groups = CoveredColumnIndexSpecifierBuilder.getColumns(env.getConfiguration());
-  }
-
-  @Override
-  public Iterable<IndexUpdate> getIndexUpserts(TableState state) {
-    List<IndexUpdate> updates = new ArrayList<IndexUpdate>();
-    for (ColumnGroup group : groups) {
-      IndexUpdate update = getIndexUpdateForGroup(group, state);
-      updates.add(update);
-    }
-    return updates;
-  }
-
-  /**
-   * @param group
-   * @param state
-   * @return the update that should be made to the table
-   */
-  private IndexUpdate getIndexUpdateForGroup(ColumnGroup group, TableState state) {
-    List<CoveredColumn> refs = group.getColumns();
-    try {
-      Pair<Scanner, IndexUpdate> stateInfo = state.getIndexedColumnsTableState(refs);
-      Scanner kvs = stateInfo.getFirst();
-      Pair<Integer, List<ColumnEntry>> columns =
-          getNextEntries(refs, kvs, state.getCurrentRowKey());
-      // make sure we close the scanner
-      kvs.close();
-      if (columns.getFirst().intValue() == 0) {
-        return stateInfo.getSecond();
-      }
-      // have all the column entries, so just turn it into a Delete for the row
-      // convert the entries to the needed values
-      byte[] rowKey =
-          composeRowKey(state.getCurrentRowKey(), columns.getFirst(), columns.getSecond());
-      Put p = new Put(rowKey, state.getCurrentTimestamp());
-      // add the columns to the put
-      addColumnsToPut(p, columns.getSecond());
-
-      // update the index info
-      IndexUpdate update = stateInfo.getSecond();
-      update.setTable(Bytes.toBytes(group.getTable()));
-      update.setUpdate(p);
-      return update;
-    } catch (IOException e) {
-      throw new RuntimeException("Unexpected exception when getting state for columns: " + refs);
-    }
-  }
-
-  private static void addColumnsToPut(Put indexInsert, List<ColumnEntry> columns) {
-    // add each of the corresponding families to the put
-    int count = 0;
-    for (ColumnEntry column : columns) {
-      indexInsert.add(INDEX_ROW_COLUMN_FAMILY,
-        ArrayUtils.addAll(Bytes.toBytes(count++), toIndexQualifier(column.ref)), null);
-    }
-  }
-
-  private static byte[] toIndexQualifier(CoveredColumn column) {
-    return ArrayUtils.addAll(Bytes.toBytes(column.familyString + CoveredColumn.SEPARATOR),
-      column.getQualifier());
-  }
-
-  @Override
-  public Iterable<IndexUpdate> getIndexDeletes(TableState state) {
-    List<IndexUpdate> deletes = new ArrayList<IndexUpdate>();
-    for (ColumnGroup group : groups) {
-      deletes.add(getDeleteForGroup(group, state));
-    }
-    return deletes;
-  }
-
-
-  /**
-   * Get all the deletes necessary for a group of columns - logically, the cleanup the index table
-   * for a given index.
-   * @param group index information
-   * @return the cleanup for the given index, or <tt>null</tt> if no cleanup is necessary
-   */
-  private IndexUpdate getDeleteForGroup(ColumnGroup group, TableState state) {
-    List<CoveredColumn> refs = group.getColumns();
-    try {
-      Pair<Scanner, IndexUpdate> kvs = state.getIndexedColumnsTableState(refs);
-      Pair<Integer, List<ColumnEntry>> columns =
-          getNextEntries(refs, kvs.getFirst(), state.getCurrentRowKey());
-      // make sure we close the scanner reference
-      kvs.getFirst().close();
-      // no change, just return the passed update
-      if (columns.getFirst() == 0) {
-        return kvs.getSecond();
-      }
-      // have all the column entries, so just turn it into a Delete for the row
-      // convert the entries to the needed values
-      byte[] rowKey =
-          composeRowKey(state.getCurrentRowKey(), columns.getFirst(), columns.getSecond());
-      Delete d = new Delete(rowKey);
-      d.setTimestamp(state.getCurrentTimestamp());
-      IndexUpdate update = kvs.getSecond();
-      update.setUpdate(d);
-      update.setTable(Bytes.toBytes(group.getTable()));
-      return update;
-    } catch (IOException e) {
-      throw new RuntimeException("Unexpected exception when getting state for columns: " + refs);
-    }
-  }
-
-  /**
-   * Get the next batch of primary table values for the given columns
-   * @param refs columns to match against
-   * @param state
-   * @return the total length of all values found and the entries to add for the index
-   */
-  private Pair<Integer, List<ColumnEntry>> getNextEntries(List<CoveredColumn> refs, Scanner kvs,
-      byte[] currentRow) throws IOException {
-    int totalValueLength = 0;
-    List<ColumnEntry> entries = new ArrayList<ColumnEntry>(refs.size());
-
-    // pull out the latest state for each column reference, in order
-    for (CoveredColumn ref : refs) {
-      KeyValue first = ref.getFirstKeyValueForRow(currentRow);
-      if (!kvs.seek(first)) {
-        // no more keys, so add a null value
-        entries.add(new ColumnEntry(null, ref));
-        continue;
-      }
-      // there is a next value - we only care about the current value, so we can just snag that
-      KeyValue next = kvs.next();
-      if (ref.matchesFamily(next.getFamily()) && ref.matchesQualifier(next.getQualifier())) {
-        byte[] v = next.getValue();
-        totalValueLength += v.length;
-        entries.add(new ColumnEntry(v, ref));
-      } else {
-        // this first one didn't match at all, so we have to put in a null entry
-        entries.add(new ColumnEntry(null, ref));
-        continue;
-      }
-      // here's where is gets a little tricky - we either need to decide if we should continue
-      // adding entries (matches all qualifiers) or if we are done (matches a single qualifier)
-      if (!ref.allColumns()) {
-        continue;
-      }
-      // matches all columns, so we need to iterate until we hit the next column with the same
-      // family as the current key
-      byte[] lastQual = next.getQualifier();
-      byte[] nextQual = null;
-      while ((next = kvs.next()) != null) {
-        // different family, done with this column
-        if (!ref.matchesFamily(next.getFamily())) {
-          break;
-        }
-        nextQual = next.getQualifier();
-        // we are still on the same qualifier - skip it, since we already added a column for it
-        if (Arrays.equals(lastQual, nextQual)) {
-          continue;
-        }
-        // this must match the qualifier since its an all-qualifiers specifier, so we add it
-        byte[] v = next.getValue();
-        totalValueLength += v.length;
-        entries.add(new ColumnEntry(v, ref));
-        // update the last qualifier to check against
-        lastQual = nextQual;
-      }
-    }
-    return new Pair<Integer, List<ColumnEntry>>(totalValueLength, entries);
-  }
-
-  static class ColumnEntry {
-    byte[] value = EMPTY_BYTES;
-    CoveredColumn ref;
-
-    public ColumnEntry(byte[] value, CoveredColumn ref) {
-      this.value = value == null ? EMPTY_BYTES : value;
-      this.ref = ref;
-    }
-  }
-
-  /**
-   * Compose the final index row key.
-   * <p>
-   * This is faster than adding each value independently as we can just build a single a array and
-   * copy everything over once.
-   * @param pk primary key of the original row
-   * @param length total number of bytes of all the values that should be added
-   * @param values to use when building the key
-   * @return
-   */
-  static byte[] composeRowKey(byte[] pk, int length, List<ColumnEntry> values) {
-    // now build up expected row key, each of the values, in order, followed by the PK and then some
-    // info about lengths so we can deserialize each value
-    byte[] output = new byte[length + pk.length];
-    int pos = 0;
-    int[] lengths = new int[values.size()];
-    int i = 0;
-    for (ColumnEntry entry : values) {
-      byte[] v = entry.value;
-      // skip doing the copy attempt, if we don't need to
-      if (v.length != 0) {
-        System.arraycopy(v, 0, output, pos, v.length);
-        pos += v.length;
-      }
-      lengths[i++] = v.length;
-    }
-
-    // add the primary key to the end of the row key
-    System.arraycopy(pk, 0, output, pos, pk.length);
-
-    // add the lengths as suffixes so we can deserialize the elements again
-    for (int l : lengths) {
-      output = ArrayUtils.addAll(output, Bytes.toBytes(l));
-    }
-
-    // and the last integer is the number of values
-    return ArrayUtils.addAll(output, Bytes.toBytes(values.size()));
-  }
-
-  /**
-   * Essentially a short-cut from building a {@link Put}.
-   * @param pk row key
-   * @param timestamp timestamp of all the keyvalues
-   * @param values expected value--column pair
-   * @return a keyvalues that the index contains for a given row at a timestamp with the given value
-   *         -- column pairs.
-   */
-  public static List<KeyValue> getIndexKeyValueForTesting(byte[] pk, long timestamp,
-      List<Pair<byte[], CoveredColumn>> values) {
-  
-    int length = 0;
-    List<ColumnEntry> expected = new ArrayList<ColumnEntry>(values.size());
-    for (Pair<byte[], CoveredColumn> value : values) {
-      ColumnEntry entry = new ColumnEntry(value.getFirst(), value.getSecond());
-      length += value.getFirst().length;
-      expected.add(entry);
-    }
-  
-    byte[] rowKey = CoveredColumnIndexCodec.composeRowKey(pk, length, expected);
-    Put p = new Put(rowKey, timestamp);
-    CoveredColumnIndexCodec.addColumnsToPut(p, expected);
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
-    for (Entry<byte[], List<KeyValue>> entry : p.getFamilyMap().entrySet()) {
-      kvs.addAll(entry.getValue());
-    }
-  
-    return kvs;
-  }
-
-  public static List<byte[]> getValues(byte[] bytes) {
-    // get the total number of keys in the bytes
-    int keyCount = CoveredColumnIndexCodec.getPreviousInteger(bytes, bytes.length);
-    List<byte[]> keys = new ArrayList<byte[]>(keyCount);
-    int[] lengths = new int[keyCount];
-    int lengthPos = keyCount - 1;
-    int pos = bytes.length - Bytes.SIZEOF_INT;
-    // figure out the length of each key
-    for (int i = 0; i < keyCount; i++) {
-      lengths[lengthPos--] = CoveredColumnIndexCodec.getPreviousInteger(bytes, pos);
-      pos -= Bytes.SIZEOF_INT;
-    }
-
-    int current = 0;
-    for (int length : lengths) {
-      byte[] key = Arrays.copyOfRange(bytes, current, current + length);
-      keys.add(key);
-      current += length;
-    }
-
-    return keys;
-  }
-
-  /**
-   * Read an integer from the preceding {@value Bytes#SIZEOF_INT} bytes
-   * @param bytes array to read from
-   * @param start start point, backwards from which to read. For example, if specifying "25", we
-   *          would try to read an integer from 21 -> 25
-   * @return an integer from the proceeding {@value Bytes#SIZEOF_INT} bytes, if it exists.
-   */
-  private static int getPreviousInteger(byte[] bytes, int start) {
-    return Bytes.toInt(bytes, start - Bytes.SIZEOF_INT);
-  }
-
-  /**
-   * Check to see if an row key just contains a list of null values.
-   * @param bytes row key to examine
-   * @return <tt>true</tt> if all the values are zero-length, <tt>false</tt> otherwise
-   */
-  public static boolean checkRowKeyForAllNulls(byte[] bytes) {
-    int keyCount = CoveredColumnIndexCodec.getPreviousInteger(bytes, bytes.length);
-    int pos = bytes.length - Bytes.SIZEOF_INT;
-    for (int i = 0; i < keyCount; i++) {
-      int next = CoveredColumnIndexCodec.getPreviousInteger(bytes, pos);
-      if (next > 0) {
-        return false;
-      }
-      pos -= Bytes.SIZEOF_INT;
-    }
-
-    return true;
-  }
-
-  @Override
-  public boolean isEnabled(Mutation m) {
-    // this could be a bit smarter, looking at the groups for the mutation, but we leave it at this
-    // simple check for the moment.
-    return groups.size() > 0;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java
deleted file mode 100644
index 1ed758e..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java
+++ /dev/null
@@ -1,184 +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.hadoop.hbase.index.covered.example;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HTableDescriptor;
-
-import org.apache.hadoop.hbase.index.Indexer;
-import org.apache.hadoop.hbase.index.covered.CoveredColumnsIndexBuilder;
-import org.apache.hadoop.hbase.index.covered.IndexCodec;
-
-/**
- * Helper to build the configuration for the {@link CoveredColumnIndexer}.
- * <p>
- * This class is NOT thread-safe; all concurrent access must be managed externally.
- */
-public class CoveredColumnIndexSpecifierBuilder {
-
-  private static final String INDEX_TO_TABLE_CONF_PREFX = "hbase.index.covered.";
-  // number of index 'groups'. Each group represents a set of 'joined' columns. The data stored with
-  // each joined column are either just the columns in the group or all the most recent data in the
-  // row (a fully covered index).
-  private static final String COUNT = ".count";
-  private static final String INDEX_GROUPS_COUNT_KEY = INDEX_TO_TABLE_CONF_PREFX + ".groups" + COUNT;
-  private static final String INDEX_GROUP_PREFIX = INDEX_TO_TABLE_CONF_PREFX + "group.";
-  private static final String INDEX_GROUP_COVERAGE_SUFFIX = ".columns";
-  private static final String TABLE_SUFFIX = ".table";
-
-  // right now, we don't support this should be easy enough to add later
-  // private static final String INDEX_GROUP_FULLY_COVERED = ".covered";
-
-  List<ColumnGroup> groups = new ArrayList<ColumnGroup>();
-  private Map<String, String> specs = new HashMap<String, String>();
-
-  /**
-   * Add a group of columns to index
-   * @param columns Pairs of cf:cq (full specification of a column) to index
-   * @return the index of the group. This can be used to remove or modify the group via
-   *         {@link #remove(int)} or {@link #get(int)}, any time before building
-   */
-  public int addIndexGroup(ColumnGroup columns) {
-    if (columns == null || columns.size() == 0) {
-      throw new IllegalArgumentException("Must specify some columns to index!");
-    }
-    int size = this.groups.size();
-    this.groups.add(columns);
-    return size;
-  }
-
-  public void remove(int i) {
-    this.groups.remove(i);
-  }
-
-  public ColumnGroup get(int i) {
-    return this.groups.get(i);
-  }
-
-  /**
-   * Clear the stored {@link ColumnGroup}s for resuse.
-   */
-  public void reset() {
-    this.groups.clear();
-  }
-
-  Map<String, String> convertToMap() {
-    int total = this.groups.size();
-    // hbase.index.covered.groups = i
-    specs.put(INDEX_GROUPS_COUNT_KEY, Integer.toString(total));
-
-    int i = 0;
-    for (ColumnGroup group : groups) {
-      addIndexGroupToSpecs(specs, group, i++);
-    }
-
-    return specs;
-  }
-
-  /**
-   * @param specs
-   * @param columns
-   * @param index
-   */
-  private void addIndexGroupToSpecs(Map<String, String> specs, ColumnGroup columns, int index) {
-    // hbase.index.covered.group.<i>
-    String prefix = INDEX_GROUP_PREFIX + Integer.toString(index);
-
-    // set the table to which the group writes
-    // hbase.index.covered.group.<i>.table
-    specs.put(prefix + TABLE_SUFFIX, columns.getTable());
-    
-    // a different key for each column in the group
-    // hbase.index.covered.group.<i>.columns
-    String columnPrefix = prefix + INDEX_GROUP_COVERAGE_SUFFIX;
-    // hbase.index.covered.group.<i>.columns.count = <j>
-    String columnsSizeKey = columnPrefix + COUNT;
-    specs.put(columnsSizeKey, Integer.toString(columns.size()));
-    
-    // add each column in the group
-    int i=0; 
-    for (CoveredColumn column : columns) {
-      // hbase.index.covered.group.<i>.columns.<j>
-      String nextKey = columnPrefix + "." + Integer.toString(i);
-      String nextValue = column.serialize();
-      specs.put(nextKey, nextValue);
-      i++;
-    }
-  }
-
-  public void build(HTableDescriptor desc) throws IOException {
-    build(desc, CoveredColumnIndexCodec.class);
-  }
-
-  void build(HTableDescriptor desc, Class<? extends IndexCodec> clazz) throws IOException {
-    // add the codec for the index to the map of options
-    Map<String, String> opts = this.convertToMap();
-    opts.put(CoveredColumnsIndexBuilder.CODEC_CLASS_NAME_KEY, clazz.getName());
-    Indexer.enableIndexing(desc, CoveredColumnIndexer.class, opts);
-  }
-
-  static List<ColumnGroup> getColumns(Configuration conf) {
-    int count= conf.getInt(INDEX_GROUPS_COUNT_KEY, 0);
-    if (count ==0) {
-      return Collections.emptyList();
-    }
-
-    // parse out all the column groups we should index
-    List<ColumnGroup> columns = new ArrayList<ColumnGroup>(count);
-    for (int i = 0; i < count; i++) {
-      // parse out each group
-      String prefix = INDEX_GROUP_PREFIX + i;
-
-      // hbase.index.covered.group.<i>.table
-      String table = conf.get(prefix + TABLE_SUFFIX);
-      ColumnGroup group = new ColumnGroup(table);
-
-      // parse out each column in the group
-      // hbase.index.covered.group.<i>.columns
-      String columnPrefix = prefix + INDEX_GROUP_COVERAGE_SUFFIX;
-      // hbase.index.covered.group.<i>.columns.count = j
-      String columnsSizeKey = columnPrefix + COUNT;
-      int columnCount = conf.getInt(columnsSizeKey, 0);
-      for(int j=0; j< columnCount; j++){
-        String columnKey = columnPrefix + "." + j;
-        CoveredColumn column = CoveredColumn.parse(conf.get(columnKey));
-        group.add(column);
-      }
-
-      // add the group
-      columns.add(group);
-    }
-    return columns;
-  }
-
-  /**
-   * @param key
-   * @param value
-   */
-  public void addArbitraryConfigForTesting(String key, String value) {
-    this.specs.put(key, value);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexer.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexer.java
deleted file mode 100644
index 4244a47..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumnIndexer.java
+++ /dev/null
@@ -1,164 +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.hadoop.hbase.index.covered.example;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.hadoop.hbase.index.covered.Batch;
-import org.apache.hadoop.hbase.index.covered.CoveredColumnsIndexBuilder;
-import org.apache.hadoop.hbase.index.covered.LocalTableState;
-import org.apache.hadoop.hbase.index.covered.update.IndexUpdateManager;
-
-/**
- * Index maintainer that maintains multiple indexes based on '{@link ColumnGroup}s'. Each group is a
- * fully covered within itself and stores the fully 'pre-joined' version of that values for that
- * group of columns.
- * <p>
- * <h2>Index Layout</h2> The row key for a given index entry is the current state of the all the
- * values of the columns in a column group, followed by the primary key (row key) of the original
- * row, and then the length of each value and then finally the total number of values. This is then
- * enough information to completely rebuild the latest value of row for each column in the group.
- * <p>
- * The family is always {@link CoveredColumnIndexCodec#INDEX_ROW_COLUMN_FAMILY}
- * <p>
- * The qualifier is prepended with the integer index (serialized with {@link Bytes#toBytes(int)}) of
- * the column in the group. This index corresponds the index of the value for the group in the row
- * key.
- * 
- * <pre>
- *         ROW                            ||   FAMILY     ||    QUALIFIER     ||   VALUE
- * (v1)(v2)...(vN)(pk)(L1)(L2)...(Ln)(#V) || INDEX_FAMILY ||     1Cf1:Cq1     ||  null
- * (v1)(v2)...(vN)(pk)(L1)(L2)...(Ln)(#V) || INDEX_FAMILY ||     2Cf2:Cq2     ||  null
- * ...
- * (v1)(v2)...(vN)(pk)(L1)(L2)...(Ln)(#V) || INDEX_FAMILY ||     NCfN:CqN     ||  null
- * </pre>
- * 
- * <h2>Index Maintenance</h2>
- * <p>
- * When making an insertion into the table, we also attempt to cleanup the index. This means that we
- * need to remove the previous entry from the index. Generally, this is completed by inserting a
- * delete at the previous value of the previous row.
- * <p>
- * The main caveat here is when dealing with custom timestamps. If there is no special timestamp
- * specified, we can just insert the proper {@link Delete} at the current timestamp and move on.
- * However, when the client specifies a timestamp, we could see updates out of order. In that case,
- * we can do an insert using the specified timestamp, but a delete is different...
- * <p>
- * Taking the simple case, assume we do a single column in a group. Then if we get an out of order
- * update, we need to check the current state of that column in the current row. If the current row
- * is older, we can issue a delete as normal. If the current row is newer, however, we then have to
- * issue a delete for the index update at the time of the current row. This ensures that the index
- * update made for the 'future' time still covers the existing row.
- * <p>
- * <b>ASSUMPTION:</b> all key-values in a single {@link Delete}/{@link Put} have the same timestamp.
- * This dramatically simplifies the logic needed to manage updating the index for out-of-order
- * {@link Put}s as we don't need to manage multiple levels of timestamps across multiple columns.
- * <p>
- * We can extend this to multiple columns by picking the latest update of any column in group as the
- * delete point.
- * <p>
- * <b>NOTE:</b> this means that we need to do a lookup (point {@link Get}) of the entire row
- * <i>every time there is a write to the table</i>.
- */
-public class CoveredColumnIndexer extends CoveredColumnsIndexBuilder {
-
-  /**
-   * Create the specified index table with the necessary columns
-   * @param admin {@link HBaseAdmin} to use when creating the table
-   * @param indexTable name of the index table.
-   * @throws IOException
-   */
-  public static void createIndexTable(HBaseAdmin admin, String indexTable) throws IOException {
-    createIndexTable(admin, new HTableDescriptor(indexTable));
-  }
-
-  /**
-   * @param admin to create the table
-   * @param index descriptor to update before creating table
-   */
-  public static void createIndexTable(HBaseAdmin admin, HTableDescriptor index) throws IOException {
-    HColumnDescriptor col =
-        new HColumnDescriptor(CoveredColumnIndexCodec.INDEX_ROW_COLUMN_FAMILY);
-    // ensure that we can 'see past' delete markers when doing scans
-    col.setKeepDeletedCells(true);
-    index.addFamily(col);
-    admin.createTable(index);
-  }
-
-  @Override
-  public Collection<Pair<Mutation, byte[]>> getIndexUpdateForFilteredRows(
-      Collection<KeyValue> filtered) throws IOException {
-
-    // stores all the return values
-    IndexUpdateManager updateMap = new IndexUpdateManager();
-    // batch the updates by row to make life easier and ordered
-    Collection<Batch> batches = batchByRow(filtered);
-
-    for (Batch batch : batches) {
-      Put p = new Put(batch.getKvs().iterator().next().getRow());
-      for (KeyValue kv : batch.getKvs()) {
-        // we only need to cleanup Put entries
-        byte type = kv.getType();
-        Type t = KeyValue.Type.codeToType(type);
-        if (!t.equals(Type.Put)) {
-          continue;
-        }
-
-        // add the kv independently
-        p.add(kv);
-      }
-
-      // do the usual thing as for deletes
-      Collection<Batch> timeBatch = createTimestampBatchesFromMutation(p);
-      LocalTableState state = new LocalTableState(env, localTable, p);
-      for (Batch entry : timeBatch) {
-        //just set the timestamp on the table - it already has all the future state
-        state.setCurrentTimestamp(entry.getTimestamp());
-        this.addDeleteUpdatesToMap(updateMap, state, entry.getTimestamp());
-      }
-    }
-    return updateMap.toMap();
-  }
-
-
-  /**
-   * @param filtered
-   * @return
-   */
-  private Collection<Batch>  batchByRow(Collection<KeyValue> filtered) {
-    Map<Long, Batch> batches = new HashMap<Long, Batch>();
-    createTimestampBatchesFromKeyValues(filtered, batches);
-    return batches.values();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ApplyAndFilterDeletesFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ApplyAndFilterDeletesFilter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ApplyAndFilterDeletesFilter.java
deleted file mode 100644
index d360bd5..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ApplyAndFilterDeletesFilter.java
+++ /dev/null
@@ -1,308 +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.hadoop.hbase.index.covered.filter;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.filter.FilterBase;
-
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-/**
- * Only allow the 'latest' timestamp of each family:qualifier pair, ensuring that they aren't
- * covered by a previous delete. This is similar to some of the work the ScanQueryMatcher does to
- * ensure correct visibility of keys based on deletes.
- * <p>
- * No actual delete {@link KeyValue}s are allowed to pass through this filter - they are always
- * skipped.
- * <p>
- * Note there is a little bit of conceptually odd behavior (though it matches the HBase
- * specifications) around point deletes ({@link KeyValue} of type {@link Type#Delete}. These deletes
- * only apply to a single {@link KeyValue} at a single point in time - they essentially completely
- * 'cover' the existing {@link Put} at that timestamp. However, they don't 'cover' any other
- * keyvalues at older timestamps. Therefore, if there is a point-delete at ts = 5, and puts at ts =
- * 4, and ts = 5, we will only allow the put at ts = 4.
- * <p>
- * Expects {@link KeyValue}s to arrive in sorted order, with 'Delete' {@link Type} {@link KeyValue}s
- * ({@link Type#DeleteColumn}, {@link Type#DeleteFamily}, {@link Type#Delete})) before their regular
- * {@link Type#Put} counterparts.
- */
-public class ApplyAndFilterDeletesFilter extends FilterBase {
-
-  private boolean done = false;
-  List<ImmutableBytesPtr> families;
-  private final DeleteTracker coveringDelete = new DeleteTracker();
-  private Hinter currentHint;
-  private DeleteColumnHinter columnHint = new DeleteColumnHinter();
-  private DeleteFamilyHinter familyHint = new DeleteFamilyHinter();
-  
-  /**
-   * Setup the filter to only include the given families. This allows us to seek intelligently pass
-   * families we don't care about.
-   * @param families
-   */
-  public ApplyAndFilterDeletesFilter(Set<ImmutableBytesPtr> families) {
-    this.families = new ArrayList<ImmutableBytesPtr>(families);
-    Collections.sort(this.families);
-  }
-      
-  
-  private ImmutableBytesPtr getNextFamily(ImmutableBytesPtr family) {
-    int index = Collections.binarySearch(families, family);
-    //doesn't match exactly, be we can find the right next match
-    //this is pretty unlikely, but just incase
-    if(index < 0){
-      //the actual location of the next match
-      index = -index -1;
-    }else{
-      //its an exact match for a family, so we get the next entry
-      index = index +1;
-    }
-    //now we have the location of the next entry
-    if(index >= families.size()){
-      return null;
-    }
-    return  families.get(index);
-  }
-  
-  @Override
-  public void reset(){
-    this.coveringDelete.reset();
-    this.done = false;
-  }
-  
-  
-  @Override
-  public KeyValue getNextKeyHint(KeyValue peeked){
-    return currentHint.getHint(peeked);
-  }
-
-  @Override
-  public ReturnCode filterKeyValue(KeyValue next) {
-    // we marked ourselves done, but the END_ROW_KEY didn't manage to seek to the very last key
-    if (this.done) {
-      return ReturnCode.SKIP;
-    }
-
-    switch (KeyValue.Type.codeToType(next.getType())) {
-    /*
-     * DeleteFamily will always sort first because those KVs (we assume) don't have qualifiers (or
-     * rather are null). Therefore, we have to keep a hold of all the delete families until we get
-     * to a Put entry that is covered by that delete (in which case, we are done with the family).
-     */
-    case DeleteFamily:
-      // track the family to delete. If we are updating the delete, that means we have passed all
-      // kvs in the last column, so we can safely ignore the last deleteFamily, and just use this
-      // one. In fact, it means that all the previous deletes can be ignored because the family must
-      // not match anymore.
-      this.coveringDelete.reset();
-      this.coveringDelete.deleteFamily = next;
-      return ReturnCode.SKIP;
-    case DeleteColumn:
-      // similar to deleteFamily, all the newer deletes/puts would have been seen at this point, so
-      // we can safely replace the more recent delete column with the more recent one
-      this.coveringDelete.pointDelete = null;
-      this.coveringDelete.deleteColumn = next;
-      return ReturnCode.SKIP;
-    case Delete:
-      // we are just deleting the single column value at this point.
-      // therefore we just skip this entry and go onto the next one. The only caveat is that
-      // we should still cover the next entry if this delete applies to the next entry, so we
-      // have to keep around a reference to the KV to compare against the next valid entry
-      this.coveringDelete.pointDelete = next;
-      return ReturnCode.SKIP;
-    default:
-      // no covering deletes
-      if (coveringDelete.empty()) {
-        return ReturnCode.INCLUDE;
-      }
-
-      if (coveringDelete.matchesFamily(next)) {
-        this.currentHint = familyHint;
-        return ReturnCode.SEEK_NEXT_USING_HINT;
-      }
-
-      if (coveringDelete.matchesColumn(next)) {
-        // hint to the next column
-        this.currentHint = columnHint;
-        return ReturnCode.SEEK_NEXT_USING_HINT;
-      }
-
-      if (coveringDelete.matchesPoint(next)) {
-        return ReturnCode.SKIP;
-      }
-
-    }
-
-    // none of the deletes matches, we are done
-    return ReturnCode.INCLUDE;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    throw new UnsupportedOperationException("Server-side only filter, cannot be serialized!");
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    throw new UnsupportedOperationException("Server-side only filter, cannot be deserialized!");
-  }
-
-  /**
-   * Get the next hint for a given peeked keyvalue
-   */
-  interface Hinter {
-    public abstract KeyValue getHint(KeyValue peek);
-  }
-
-  /**
-   * Entire family has been deleted, so either seek to the next family, or if none are present in
-   * the original set of families to include, seek to the "last possible key"(or rather our best
-   * guess) and be done.
-   */
-  class DeleteFamilyHinter implements Hinter {
-
-    @Override
-    public KeyValue getHint(KeyValue peeked) {
-      // check to see if we have another column to seek
-      ImmutableBytesPtr nextFamily =
-          getNextFamily(new ImmutableBytesPtr(peeked.getBuffer(), peeked.getFamilyOffset(),
-              peeked.getFamilyLength()));
-      if (nextFamily == null) {
-        // no known next family, so we can be completely done
-        done = true;
-        return KeyValue.LOWESTKEY;
-      }
-        // there is a valid family, so we should seek to that
-      return KeyValue.createFirstOnRow(peeked.getRow(), nextFamily.copyBytesIfNecessary(),
-        HConstants.EMPTY_BYTE_ARRAY);
-    }
-
-  }
-
-  /**
-   * Hint the next column-qualifier after the given keyvalue. We can't be smart like in the
-   * ScanQueryMatcher since we don't know the columns ahead of time.
-   */
-  class DeleteColumnHinter implements Hinter {
-
-    @Override
-    public KeyValue getHint(KeyValue kv) {
-      return KeyValue.createLastOnRow(kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(),
-        kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(), kv.getBuffer(),
-        kv.getQualifierOffset(), kv.getQualifierLength());
-    }
-  }
-
-  class DeleteTracker {
-
-    public KeyValue deleteFamily;
-    public KeyValue deleteColumn;
-    public KeyValue pointDelete;
-
-    public void reset() {
-      this.deleteFamily = null;
-      this.deleteColumn = null;
-      this.pointDelete = null;
-
-    }
-
-    /**
-     * Check to see if we should skip this {@link KeyValue} based on the family.
-     * <p>
-     * Internally, also resets the currently tracked "Delete Family" marker we are tracking if the
-     * keyvalue is into another family (since CFs sort lexicographically, we can discard the current
-     * marker since it must not be applicable to any more kvs in a linear scan).
-     * @param next
-     * @return <tt>true</tt> if this {@link KeyValue} matches a delete.
-     */
-    public boolean matchesFamily(KeyValue next) {
-      if (deleteFamily == null) {
-        return false;
-      }
-      if (deleteFamily.matchingFamily(next)) {
-        // falls within the timestamp range
-        if (deleteFamily.getTimestamp() >= next.getTimestamp()) {
-          return true;
-        }
-      } else {
-        // only can reset the delete family because we are on to another family
-        deleteFamily = null;
-      }
-
-      return false;
-    }
-
-
-    /**
-     * @param next
-     * @return
-     */
-    public boolean matchesColumn(KeyValue next) {
-      if (deleteColumn == null) {
-        return false;
-      }
-      if (deleteColumn.matchingFamily(next) && deleteColumn.matchingQualifier(next)) {
-        // falls within the timestamp range
-        if (deleteColumn.getTimestamp() >= next.getTimestamp()) {
-          return true;
-        }
-      } else {
-        deleteColumn = null;
-      }
-      return false;
-    }
-
-    /**
-     * @param next
-     * @return
-     */
-    public boolean matchesPoint(KeyValue next) {
-      // point deletes only apply to the exact KV that they reference, so we only need to ensure
-      // that the timestamp matches exactly. Because we sort by timestamp first, either the next
-      // keyvalue has the exact timestamp or is an older (smaller) timestamp, and we can allow that
-      // one.
-      if (pointDelete != null && pointDelete.matchingFamily(next)
-          && pointDelete.matchingQualifier(next)) {
-        if (pointDelete.getTimestamp() == next.getTimestamp()) {
-          return true;
-        }
-        // clear the point delete since the TS must not be matching
-        coveringDelete.pointDelete = null;
-      }
-      return false;
-    }
-
-    /**
-     * @return <tt>true</tt> if no delete has been set
-     */
-    public boolean empty() {
-      return deleteFamily == null && deleteColumn == null && pointDelete == null;
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ColumnTrackingNextLargestTimestampFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ColumnTrackingNextLargestTimestampFilter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ColumnTrackingNextLargestTimestampFilter.java
deleted file mode 100644
index 2660b9f..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/ColumnTrackingNextLargestTimestampFilter.java
+++ /dev/null
@@ -1,72 +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.hadoop.hbase.index.covered.filter;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.filter.FilterBase;
-
-import org.apache.hadoop.hbase.index.covered.update.ColumnTracker;
-
-/**
- * Similar to the {@link MaxTimestampFilter}, but also updates the 'next largest' timestamp seen
- * that is not skipped by the below criteria. Note that it isn't as quick as the
- * {@link MaxTimestampFilter} as we can't just seek ahead to a key with the matching timestamp, but
- * have to iterate each kv until we find the right one with an allowed timestamp.
- * <p>
- * Inclusively filter on the maximum timestamp allowed. Excludes all elements greater than (but not
- * equal to) the given timestamp, so given ts = 5, a {@link KeyValue} with ts 6 is excluded, but not
- * one with ts = 5.
- * <p>
- * This filter generally doesn't make sense on its own - it should follow a per-column filter and
- * possible a per-delete filter to only track the most recent (but not exposed to the user)
- * timestamp.
- */
-public class ColumnTrackingNextLargestTimestampFilter extends FilterBase {
-
-  private long ts;
-  private ColumnTracker column;
-
-  public ColumnTrackingNextLargestTimestampFilter(long maxTime, ColumnTracker toTrack) {
-    this.ts = maxTime;
-    this.column = toTrack;
-  }
-
-  @Override
-  public ReturnCode filterKeyValue(KeyValue v) {
-    long timestamp = v.getTimestamp();
-    if (timestamp > ts) {
-      this.column.setTs(timestamp);
-      return ReturnCode.SKIP;
-    }
-    return ReturnCode.INCLUDE;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    throw new UnsupportedOperationException("Server-side only filter, cannot be serialized!");
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    throw new UnsupportedOperationException("Server-side only filter, cannot be deserialized!");
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/FamilyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/FamilyOnlyFilter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/FamilyOnlyFilter.java
deleted file mode 100644
index 8591f88..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/FamilyOnlyFilter.java
+++ /dev/null
@@ -1,80 +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.hadoop.hbase.index.covered.filter;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.filter.BinaryComparator;
-import org.apache.hadoop.hbase.filter.FamilyFilter;
-import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
-
-/**
- * Similar to the {@link FamilyFilter} but stops when the end of the family is reached and only
- * supports equality
- */
-public class FamilyOnlyFilter extends FamilyFilter {
-
-  boolean done = false;
-  private boolean previousMatchFound;
-
-  /**
-   * Filter on exact binary matches to the passed family
-   * @param family to compare against
-   */
-  public FamilyOnlyFilter(final byte[] family) {
-    this(new BinaryComparator(family));
-  }
-
-  public FamilyOnlyFilter(final WritableByteArrayComparable familyComparator) {
-    super(CompareOp.EQUAL, familyComparator);
-  }
-
-
-  @Override
-  public boolean filterAllRemaining() {
-    return done;
-  }
-
-  @Override
-  public void reset() {
-    done = false;
-    previousMatchFound = false;
-  }
-
-  @Override
-  public ReturnCode filterKeyValue(KeyValue v) {
-    if (done) {
-      return ReturnCode.SKIP;
-    }
-    ReturnCode code = super.filterKeyValue(v);
-    if (previousMatchFound) {
-      // we found a match before, and now we are skipping the key because of the family, therefore
-      // we are done (no more of the family).
-      if (code.equals(ReturnCode.SKIP)) {
-      done = true;
-      }
-    } else {
-      // if we haven't seen a match before, then it doesn't matter what we see now, except to mark
-      // if we've seen a match
-      if (code.equals(ReturnCode.INCLUDE)) {
-        previousMatchFound = true;
-      }
-    }
-    return code;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/MaxTimestampFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/MaxTimestampFilter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/MaxTimestampFilter.java
deleted file mode 100644
index 002f2ac..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/MaxTimestampFilter.java
+++ /dev/null
@@ -1,74 +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.hadoop.hbase.index.covered.filter;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.filter.FilterBase;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * Inclusive filter on the maximum timestamp allowed. Excludes all elements greater than (but not
- * equal to) the given timestamp, so given ts = 5, a {@link KeyValue} with ts 6 is excluded, but not
- * one with ts = 5.
- */
-public class MaxTimestampFilter extends FilterBase {
-
-  private long ts;
-
-  public MaxTimestampFilter(long maxTime) {
-    this.ts = maxTime;
-  }
-
-  @Override
-  public KeyValue getNextKeyHint(KeyValue currentKV) {
-    // this might be a little excessive right now - better safe than sorry though, so we don't mess
-    // with other filters too much.
-    KeyValue kv = currentKV.deepCopy();
-    int offset =kv.getTimestampOffset();
-    //set the timestamp in the buffer
-    byte[] buffer = kv.getBuffer();
-    byte[] ts = Bytes.toBytes(this.ts);
-    System.arraycopy(ts, 0, buffer, offset, ts.length);
-
-    return kv;
-  }
-
-  @Override
-  public ReturnCode filterKeyValue(KeyValue v) {
-    long timestamp = v.getTimestamp();
-    if (timestamp > ts) {
-      return ReturnCode.SEEK_NEXT_USING_HINT;
-    }
-    return ReturnCode.INCLUDE;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    throw new UnsupportedOperationException("Server-side only filter, cannot be serialized!");
-
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    throw new UnsupportedOperationException("Server-side only filter, cannot be deserialized!");
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/NewerTimestampFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/NewerTimestampFilter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/NewerTimestampFilter.java
deleted file mode 100644
index ada2601..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/filter/NewerTimestampFilter.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.index.covered.filter;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.filter.FilterBase;
-
-/**
- * Server-side only class used in the indexer to filter out keyvalues newer than a given timestamp
- * (so allows anything <code><=</code> timestamp through).
- * <p>
- * Note,<tt>this</tt> doesn't support {@link #write(DataOutput)} or {@link #readFields(DataInput)}.
- */
-public class NewerTimestampFilter extends FilterBase {
-
-  private long timestamp;
-
-  public NewerTimestampFilter(long timestamp) {
-    this.timestamp = timestamp;
-  }
-
-  @Override
-  public ReturnCode filterKeyValue(KeyValue ignored) {
-    return ignored.getTimestamp() > timestamp ? ReturnCode.SKIP : ReturnCode.INCLUDE;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    throw new UnsupportedOperationException("TimestampFilter is server-side only!");
-  }
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    throw new UnsupportedOperationException("TimestampFilter is server-side only!");
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnReference.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnReference.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnReference.java
deleted file mode 100644
index ca2b5e4..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnReference.java
+++ /dev/null
@@ -1,166 +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.hadoop.hbase.index.covered.update;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Bytes;
-
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-/**
- * 
- */
-public class ColumnReference implements Comparable<ColumnReference> {
-    
-  public static final byte[] ALL_QUALIFIERS = new byte[0];
-  
-  private static int calcHashCode(byte[] family, byte[] qualifier) {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + Bytes.hashCode(family);
-    result = prime * result + Bytes.hashCode(qualifier);
-    return result;
-  }
-
-  private final int hashCode;
-  protected final byte[] family;
-  protected final byte[] qualifier;
-    private volatile ImmutableBytesWritable familyPtr;
-    private volatile ImmutableBytesWritable qualifierPtr;
-
-  public ColumnReference(byte[] family, byte[] qualifier) {
-    this.family = family;
-    this.qualifier = qualifier;
-    this.hashCode = calcHashCode(family, qualifier);
-  }
-
-  public byte[] getFamily() {
-    return this.family;
-  }
-
-  public byte[] getQualifier() {
-    return this.qualifier;
-  }
-  
-    public ImmutableBytesWritable getFamilyWritable() {
-        if (this.familyPtr == null) {
-            synchronized (this.family) {
-                if (this.familyPtr == null) {
-                    this.familyPtr = new ImmutableBytesPtr(this.family);
-                }
-            }
-        }
-        return this.familyPtr;
-    }
-
-    public ImmutableBytesWritable getQualifierWritable() {
-        if (this.qualifierPtr == null) {
-            synchronized (this.qualifier) {
-                if (this.qualifierPtr == null) {
-                    this.qualifierPtr = new ImmutableBytesPtr(this.qualifier);
-                }
-            }
-        }
-        return this.qualifierPtr;
-    }
-
-  public boolean matches(KeyValue kv) {
-    if (matchesFamily(kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength())) {
-      return matchesQualifier(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
-    }
-    return false;
-  }
-
-  /**
-   * @param qual to check against
-   * @return <tt>true</tt> if this column covers the given qualifier.
-   */
-  public boolean matchesQualifier(byte[] qual) {
-    return matchesQualifier(qual, 0, qual.length);
-  }
-
-  public boolean matchesQualifier(byte[] bytes, int offset, int length) {
-    return allColumns() ? true : match(bytes, offset, length, qualifier);
-  }
-
-  /**
-   * @param family to check against
-   * @return <tt>true</tt> if this column covers the given family.
-   */
-  public boolean matchesFamily(byte[] family) {
-    return matchesFamily(family, 0, family.length);
-  }
-
-  public boolean matchesFamily(byte[] bytes, int offset, int length) {
-    return match(bytes, offset, length, family);
-  }
-
-  /**
-   * @return <tt>true</tt> if this should include all column qualifiers, <tt>false</tt> otherwise
-   */
-  public boolean allColumns() {
-    return this.qualifier == ALL_QUALIFIERS;
-  }
-
-  /**
-   * Check to see if the passed bytes match the stored bytes
-   * @param first
-   * @param storedKey the stored byte[], should never be <tt>null</tt>
-   * @return <tt>true</tt> if they are byte-equal
-   */
-  private boolean match(byte[] first, int offset, int length, byte[] storedKey) {
-    return first == null ? false : Bytes.equals(first, offset, length, storedKey, 0,
-      storedKey.length);
-  }
-
-  public KeyValue getFirstKeyValueForRow(byte[] row) {
-    return KeyValue.createFirstOnRow(row, family, qualifier == ALL_QUALIFIERS ? null : qualifier);
-  }
-
-  @Override
-  public int compareTo(ColumnReference o) {
-    int c = Bytes.compareTo(family, o.family);
-    if (c == 0) {
-      // matching families, compare qualifiers
-      c = Bytes.compareTo(qualifier, o.qualifier);
-    }
-    return c;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof ColumnReference) {
-      ColumnReference other = (ColumnReference) o;
-      if (hashCode == other.hashCode && Bytes.equals(family, other.family)) {
-        return Bytes.equals(qualifier, other.qualifier);
-      }
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return hashCode;
-  }
-
-  @Override
-  public String toString() {
-    return "ColumnReference - " + Bytes.toString(family) + ":" + Bytes.toString(qualifier);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnTracker.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnTracker.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnTracker.java
deleted file mode 100644
index 3b4d266..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/ColumnTracker.java
+++ /dev/null
@@ -1,115 +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.hadoop.hbase.index.covered.update;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-
-/**
- * Simple POJO for tracking a bunch of column references and the next-newest timestamp for those
- * columns
- * <p>
- * Two {@link ColumnTracker}s are considered equal if they track the same columns, even if their
- * timestamps are different.
- */
-public class ColumnTracker implements IndexedColumnGroup {
-
-  public static final long NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP = Long.MAX_VALUE;
-  public static final long GUARANTEED_NEWER_UPDATES = Long.MIN_VALUE;
-  private final List<ColumnReference> columns;
-  private long ts = NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP;
-  private final int hashCode;
-
-  private static int calcHashCode(List<ColumnReference> columns) {
-      return columns.hashCode();
-    }
-
-  public ColumnTracker(Collection<? extends ColumnReference> columns) {
-    this.columns = new ArrayList<ColumnReference>(columns);
-    // sort the columns
-    Collections.sort(this.columns);
-    this.hashCode = calcHashCode(this.columns);
-  }
-
-  /**
-   * Set the current timestamp, only if the passed timestamp is strictly less than the currently
-   * stored timestamp
-   * @param ts the timestmap to potentially store.
-   * @return the currently stored timestamp.
-   */
-  public long setTs(long ts) {
-    this.ts = this.ts > ts ? ts : this.ts;
-    return this.ts;
-  }
-
-  public long getTS() {
-    return this.ts;
-  }
-
-  @Override
-  public int hashCode() {
-    return hashCode;
-  }
-
-  @Override
-  public boolean equals(Object o){
-    if(!(o instanceof ColumnTracker)){
-      return false;
-    }
-    ColumnTracker other = (ColumnTracker)o;
-    if (hashCode != other.hashCode) {
-        return false;
-    }
-    if (other.columns.size() != columns.size()) {
-      return false;
-    }
-
-    // check each column to see if they match
-    for (int i = 0; i < columns.size(); i++) {
-      if (!columns.get(i).equals(other.columns.get(i))) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  @Override
-  public List<ColumnReference> getColumns() {
-    return this.columns;
-  }
-
-  /**
-   * @return <tt>true</tt> if this set of columns has seen a column with a timestamp newer than the
-   *         requested timestamp, <tt>false</tt> otherwise.
-   */
-  public boolean hasNewerTimestamps() {
-    return !isNewestTime(this.ts);
-  }
-
-  /**
-   * @param ts timestamp to check
-   * @return <tt>true</tt> if the timestamp is at the most recent timestamp for a column
-   */
-  public static boolean isNewestTime(long ts) {
-    return ts == NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexUpdateManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexUpdateManager.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexUpdateManager.java
deleted file mode 100644
index 173a2ea..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexUpdateManager.java
+++ /dev/null
@@ -1,238 +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.hadoop.hbase.index.covered.update;
-
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-
-import com.google.common.collect.Lists;
-import com.google.common.primitives.Longs;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-/**
- * Keeps track of the index updates
- */
-public class IndexUpdateManager {
-
-  public Comparator<Mutation> COMPARATOR = new MutationComparator();
-  class MutationComparator implements Comparator<Mutation> {
-
-    @Override
-    public int compare(Mutation o1, Mutation o2) {
-      // always sort rows first
-      int compare = o1.compareTo(o2);
-      if (compare != 0) {
-        return compare;
-      }
-
-      // if same row, sort by reverse timestamp (larger first)
-      compare = Longs.compare(o2.getTimeStamp(), o1.getTimeStamp());
-      if (compare != 0) {
-        return compare;
-      }
-      // deletes always sort before puts for the same row
-      if (o1 instanceof Delete) {
-        // same row, same ts == same delete since we only delete rows
-        if (o2 instanceof Delete) {
-          return 0;
-        } else {
-          // o2 has to be a put
-          return -1;
-        }
-      }
-      // o1 must be a put
-      if (o2 instanceof Delete) {
-        return 1;
-      } else if (o2 instanceof Put) {
-        return comparePuts((Put) o1, (Put) o2);
-      }
-
-      throw new RuntimeException(
-          "Got unexpected mutation types! Can only be Put or Delete, but got: " + o1 + ", and "
-              + o2);
-    }
-
-    private int comparePuts(Put p1, Put p2) {
-      int p1Size = p1.size();
-      int p2Size = p2.size();
-      int compare = p1Size - p2Size;
-      if (compare == 0) {
-        // TODO: make this a real comparison
-        // this is a little cheating, but we don't really need to worry too much about this being
-        // the same - chances are that exact matches here are really the same update.
-        return Longs.compare(p1.heapSize(), p2.heapSize());
-      }
-      return compare;
-    }
-
-  }
-
-  private static final String PHOENIX_HBASE_TEMP_DELETE_MARKER = "phoenix.hbase.temp.delete.marker";
-  private static final byte[] TRUE_MARKER = new byte[] { 1 };
-
-  protected final Map<ImmutableBytesPtr, Collection<Mutation>> map =
-      new HashMap<ImmutableBytesPtr, Collection<Mutation>>();
-
-  /**
-   * Add an index update. Keeps the latest {@link Put} for a given timestamp
-   * @param tableName
-   * @param m
-   */
-  public void addIndexUpdate(byte[] tableName, Mutation m) {
-    // we only keep the most recent update
-    ImmutableBytesPtr key = new ImmutableBytesPtr(tableName);
-    Collection<Mutation> updates = map.get(key);
-    if (updates == null) {
-      updates = new SortedCollection<Mutation>(COMPARATOR);
-      map.put(key, updates);
-    }
-    fixUpCurrentUpdates(updates, m);
-  }
-
-  /**
-   * Fix up the current updates, given the pending mutation.
-   * @param updates current updates
-   * @param pendingMutation
-   */
-  protected void fixUpCurrentUpdates(Collection<Mutation> updates, Mutation pendingMutation) {
-    // need to check for each entry to see if we have a duplicate
-    Mutation toRemove = null;
-    Delete pendingDelete = pendingMutation instanceof Delete ? (Delete) pendingMutation : null;
-    boolean sawRowMatch = false;
-    for (Mutation stored : updates) {
-      int compare = pendingMutation.compareTo(stored);
-      // skip to the right row
-      if (compare < 0) {
-        continue;
-      } else if (compare > 0) {
-        if (sawRowMatch) {
-          break;
-        }
-        continue;
-      }
-
-      // set that we saw a row match, so any greater row will necessarily be the wrong
-      sawRowMatch = true;
-
-      // skip until we hit the right timestamp
-      if (stored.getTimeStamp() < pendingMutation.getTimeStamp()) {
-        continue;
-      }
-
-      if (stored instanceof Delete) {
-        // we already have a delete for this row, so we are done.
-        if (pendingDelete != null) {
-          return;
-        }
-        // pending update must be a Put, so we ignore the Put.
-        // add a marker in the this delete that it has been canceled out already. We need to keep
-        // the delete around though so we can figure out if other Puts would also be canceled out.
-        markMutationForRemoval(stored);
-        return;
-      }
-
-      // otherwise, the stored mutation is a Put. Either way, we want to remove it. If the pending
-      // update is a delete, we need to remove the entry (no longer applies - covered by the
-      // delete), or its an older version of the row, so we cover it with the newer.
-      toRemove = stored;
-      if (pendingDelete != null) {
-        // the pending mutation, but we need to mark the mutation for removal later
-        markMutationForRemoval(pendingMutation);
-        break;
-      }
-    }
-    
-    updates.remove(toRemove);
-    updates.add(pendingMutation);
-  }
-
-  private void markMutationForRemoval(Mutation m) {
-    m.setAttribute(PHOENIX_HBASE_TEMP_DELETE_MARKER, TRUE_MARKER);
-  }
-
-  public List<Pair<Mutation, byte[]>> toMap() {
-    List<Pair<Mutation, byte[]>> updateMap = Lists.newArrayList();
-    for (Entry<ImmutableBytesPtr, Collection<Mutation>> updates : map.entrySet()) {
-      // get is ok because we always set with just the bytes
-      byte[] tableName = updates.getKey().get();
-      // TODO replace this as just storing a byte[], to avoid all the String <-> byte[] swapping
-      // HBase does
-      for (Mutation m : updates.getValue()) {
-        // skip elements that have been marked for delete
-        if (shouldBeRemoved(m)) {
-          continue;
-        }
-        updateMap.add(new Pair<Mutation, byte[]>(m, tableName));
-      }
-    }
-    return updateMap;
-  }
-
-  /**
-   * @param updates
-   */
-  public void addAll(Collection<Pair<Mutation, String>> updates) {
-    for (Pair<Mutation, String> update : updates) {
-      addIndexUpdate(Bytes.toBytes(update.getSecond()), update.getFirst());
-    }
-  }
-
-  private boolean shouldBeRemoved(Mutation m) {
-    return m.getAttribute(PHOENIX_HBASE_TEMP_DELETE_MARKER) != null;
-  }
-
-  @Override
-  public String toString() {
-    StringBuffer sb = new StringBuffer("Pending Index Updates:\n");
-    for (Entry<ImmutableBytesPtr, Collection<Mutation>> entry : map.entrySet()) {
-      String tableName = Bytes.toString(entry.getKey().get());
-      sb.append("   Table: '" + tableName + "'\n");
-      for (Mutation m : entry.getValue()) {
-        sb.append("\t");
-        if (shouldBeRemoved(m)) {
-          sb.append("[REMOVED]");
-        }
-        sb.append(m.getClass().getSimpleName() + ":"
-            + ((m instanceof Put) ? m.getTimeStamp() + " " : ""));
-        sb.append(" row=" + Bytes.toString(m.getRow()));
-        sb.append("\n");
-        if (m.getFamilyMap().isEmpty()) {
-          sb.append("\t\t=== EMPTY ===\n");
-        }
-        for (List<KeyValue> kvs : m.getFamilyMap().values()) {
-          for (KeyValue kv : kvs) {
-            sb.append("\t\t" + kv.toString() + "/value=" + Bytes.toStringBinary(kv.getValue()));
-            sb.append("\n");
-          }
-        }
-      }
-    }
-    return sb.toString();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexedColumnGroup.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexedColumnGroup.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexedColumnGroup.java
deleted file mode 100644
index 76128d3..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/IndexedColumnGroup.java
+++ /dev/null
@@ -1,28 +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.hadoop.hbase.index.covered.update;
-
-import java.util.List;
-
-/**
- * Group of columns that were requested to build an index
- */
-public interface IndexedColumnGroup {
-
-  public List<ColumnReference> getColumns();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/SortedCollection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/SortedCollection.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/SortedCollection.java
deleted file mode 100644
index d76646d..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/update/SortedCollection.java
+++ /dev/null
@@ -1,128 +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.hadoop.hbase.index.covered.update;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.PriorityQueue;
-
-import com.google.common.collect.Iterators;
-
-/**
- * A collection whose elements are stored and returned sorted.
- * <p>
- * We can't just use something like a {@link PriorityQueue} because it doesn't return the
- * underlying values in sorted order.
- * @param <T>
- */
-class SortedCollection<T> implements Collection<T>, Iterable<T> {
-
-  private PriorityQueue<T> queue;
-  private Comparator<T> comparator;
-
-  /**
-   * Use the given comparator to compare all keys for sorting
-   * @param comparator
-   */
-  public SortedCollection(Comparator<T> comparator) {
-    this.queue = new PriorityQueue<T>(1, comparator);
-    this.comparator = comparator;
-  }
-  
-  /**
-   * All passed elements are expected to be {@link Comparable}
-   */
-  public SortedCollection() {
-    this.queue = new PriorityQueue<T>();
-  }
-  
-  @Override
-  public int size() {
-    return this.queue.size();
-  }
-
-  @Override
-  public boolean isEmpty() {
-    return this.queue.isEmpty();
-  }
-
-  @Override
-  public boolean contains(Object o) {
-    return this.queue.contains(o);
-  }
-
-  @Override
-  public Iterator<T> iterator() {
-    @SuppressWarnings("unchecked")
-    T[] array = (T[]) this.queue.toArray();
-    if (this.comparator == null) {
-      Arrays.sort(array);
-    } else {
-      Arrays.sort(
-     array, this.comparator);}
-    return Iterators.forArray(array);
-  }
-
-  @Override
-  public Object[] toArray() {
-    return this.queue.toArray();
-  }
-
-  @SuppressWarnings("hiding")
-  @Override
-  public <T> T[] toArray(T[] a) {
-    return this.queue.toArray(a);
-  }
-
-  @Override
-  public boolean add(T e) {
-    return this.queue.add(e);
-  }
-
-  @Override
-  public boolean remove(Object o) {
-    return this.queue.remove(o);
-  }
-
-  @Override
-  public boolean containsAll(Collection<?> c) {
-    return this.queue.containsAll(c);
-  }
-
-  @Override
-  public boolean addAll(Collection<? extends T> c) {
-    return this.queue.addAll(c);
-  }
-
-  @Override
-  public boolean removeAll(Collection<?> c) {
-    return queue.removeAll(c);
-  }
-
-  @Override
-  public boolean retainAll(Collection<?> c) {
-    return this.queue.retainAll(c);
-  }
-
-  @Override
-  public void clear() {
-    this.queue.clear();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/IndexWriteException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/IndexWriteException.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/IndexWriteException.java
deleted file mode 100644
index 45045cc..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/IndexWriteException.java
+++ /dev/null
@@ -1,43 +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.hadoop.hbase.index.exception;
-
-import org.apache.hadoop.hbase.HBaseIOException;
-
-/**
- * Generic {@link Exception} that an index write has failed
- */
-@SuppressWarnings("serial")
-public class IndexWriteException extends HBaseIOException {
-
-  public IndexWriteException() {
-    super();
-  }
-
-  public IndexWriteException(String message, Throwable cause) {
-    super(message, cause);
-  }
-
-  public IndexWriteException(String message) {
-    super(message);
-  }
-
-  public IndexWriteException(Throwable cause) {
-    super(cause);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/MultiIndexWriteFailureException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/MultiIndexWriteFailureException.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/MultiIndexWriteFailureException.java
deleted file mode 100644
index cfc55c7..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/MultiIndexWriteFailureException.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.index.exception;
-
-import java.util.List;
-
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-
-/**
- * Indicate a failure to write to multiple index tables.
- */
-@SuppressWarnings("serial")
-public class MultiIndexWriteFailureException extends IndexWriteException {
-
-  private List<HTableInterfaceReference> failures;
-
-  /**
-   * @param failures the tables to which the index write did not succeed
-   */
-  public MultiIndexWriteFailureException(List<HTableInterfaceReference> failures) {
-    super("Failed to write to multiple index tables");
-    this.failures = failures;
-
-  }
-
-  public List<HTableInterfaceReference> getFailedTables() {
-    return this.failures;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/SingleIndexWriteFailureException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/SingleIndexWriteFailureException.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/SingleIndexWriteFailureException.java
deleted file mode 100644
index 76b9eb8..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/exception/SingleIndexWriteFailureException.java
+++ /dev/null
@@ -1,61 +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.hadoop.hbase.index.exception;
-
-import java.util.List;
-
-import org.apache.hadoop.hbase.client.Mutation;
-
-/**
- * Exception thrown if we cannot successfully write to an index table.
- */
-@SuppressWarnings("serial")
-public class SingleIndexWriteFailureException extends IndexWriteException {
-
-  private String table;
-
-  /**
-   * Cannot reach the index, but not sure of the table or the mutations that caused the failure
-   * @param msg more description of what happened
-   * @param cause original cause
-   */
-  public SingleIndexWriteFailureException(String msg, Throwable cause) {
-    super(msg, cause);
-  }
-
-  /**
-   * Failed to write the passed mutations to an index table for some reason.
-   * @param targetTableName index table to which we attempted to write
-   * @param mutations mutations that were attempted
-   * @param cause underlying reason for the failure
-   */
-  public SingleIndexWriteFailureException(String targetTableName, List<Mutation> mutations,
-      Exception cause) {
-    super("Failed to make index update:\n\t table: " + targetTableName + "\n\t edits: " + mutations
-        + "\n\tcause: " + cause == null ? "UNKNOWN" : cause.getMessage(), cause);
-    this.table = targetTableName;
-  }
-
-  /**
-   * @return The table to which we failed to write the index updates. If unknown, returns
-   *         <tt>null</tt>
-   */
-  public String getTableName() {
-    return this.table;
-  }
-}
\ No newline at end of file


[14/15] Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumnsIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumnsIndexBuilder.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumnsIndexBuilder.java
deleted file mode 100644
index fca581e..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/CoveredColumnsIndexBuilder.java
+++ /dev/null
@@ -1,488 +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.hadoop.hbase.index.covered;
-
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
-
-import com.google.common.collect.Lists;
-import com.google.common.primitives.Longs;
-import org.apache.hadoop.hbase.index.builder.BaseIndexBuilder;
-import org.apache.hadoop.hbase.index.covered.data.LocalHBaseState;
-import org.apache.hadoop.hbase.index.covered.data.LocalTable;
-import org.apache.hadoop.hbase.index.covered.update.ColumnTracker;
-import org.apache.hadoop.hbase.index.covered.update.IndexUpdateManager;
-import org.apache.hadoop.hbase.index.covered.update.IndexedColumnGroup;
-
-/**
- * Build covered indexes for phoenix updates.
- * <p>
- * Before any call to prePut/preDelete, the row has already been locked. This ensures that we don't
- * need to do any extra synchronization in the IndexBuilder.
- * <p>
- * NOTE: This implementation doesn't cleanup the index when we remove a key-value on compaction or
- * flush, leading to a bloated index that needs to be cleaned up by a background process.
- */
-public class CoveredColumnsIndexBuilder extends BaseIndexBuilder {
-
-  private static final Log LOG = LogFactory.getLog(CoveredColumnsIndexBuilder.class);
-  public static final String CODEC_CLASS_NAME_KEY = "org.apache.hadoop.hbase.index.codec.class";
-
-  protected RegionCoprocessorEnvironment env;
-  protected IndexCodec codec;
-  protected LocalHBaseState localTable;
-
-  @Override
-  public void setup(RegionCoprocessorEnvironment env) throws IOException {
-    this.env = env;
-    // setup the phoenix codec. Generally, this will just be in standard one, but abstracting here
-    // so we can use it later when generalizing covered indexes
-    Configuration conf = env.getConfiguration();
-    Class<? extends IndexCodec> codecClass =
-        conf.getClass(CODEC_CLASS_NAME_KEY, null, IndexCodec.class);
-    try {
-      Constructor<? extends IndexCodec> meth = codecClass.getDeclaredConstructor(new Class[0]);
-      meth.setAccessible(true);
-      this.codec = meth.newInstance();
-      this.codec.initialize(env);
-    } catch (IOException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-    
-    this.localTable = new LocalTable(env);
-  }
-
-  @Override
-  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Mutation mutation) throws IOException {
-    // build the index updates for each group
-    IndexUpdateManager updateMap = new IndexUpdateManager();
-
-    batchMutationAndAddUpdates(updateMap, mutation);
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Found index updates for Mutation: " + mutation + "\n" + updateMap);
-    }
-
-    return updateMap.toMap();
-  }
-
-  /**
-   * Split the mutation into batches based on the timestamps of each keyvalue. We need to check each
-   * key-value in the update to see if it matches the others. Generally, this will be the case, but
-   * you can add kvs to a mutation that don't all have the timestamp, so we need to manage
-   * everything in batches based on timestamp.
-   * <p>
-   * Adds all the updates in the {@link Mutation} to the state, as a side-effect.
-   * @param updateMap index updates into which to add new updates. Modified as a side-effect.
-   * @param state current state of the row for the mutation.
-   * @param m mutation to batch
- * @throws IOException 
-   */
-  private void batchMutationAndAddUpdates(IndexUpdateManager manager, Mutation m) throws IOException {
-    // split the mutation into timestamp-based batches
-    Collection<Batch> batches = createTimestampBatchesFromMutation(m);
-
-    // create a state manager, so we can manage each batch
-    LocalTableState state = new LocalTableState(env, localTable, m);
-
-    // go through each batch of keyvalues and build separate index entries for each
-    boolean cleanupCurrentState = true;
-    for (Batch batch : batches) {
-      /*
-       * We have to split the work between the cleanup and the update for each group because when we
-       * update the current state of the row for the current batch (appending the mutations for the
-       * current batch) the next group will see that as the current state, which will can cause the
-       * a delete and a put to be created for the next group.
-       */
-      if (addMutationsForBatch(manager, batch, state, cleanupCurrentState)) {
-        cleanupCurrentState = false;
-      }
-    }
-  }
-
-  /**
-   * Batch all the {@link KeyValue}s in a {@link Mutation} by timestamp. Updates any
-   * {@link KeyValue} with a timestamp == {@link HConstants#LATEST_TIMESTAMP} to the timestamp at
-   * the time the method is called.
-   * @param m {@link Mutation} from which to extract the {@link KeyValue}s
-   * @return the mutation, broken into batches and sorted in ascending order (smallest first)
-   */
-  protected Collection<Batch> createTimestampBatchesFromMutation(Mutation m) {
-    Map<Long, Batch> batches = new HashMap<Long, Batch>();
-    for (List<KeyValue> family : m.getFamilyMap().values()) {
-      createTimestampBatchesFromKeyValues(family, batches);
-    }
-    // sort the batches
-    List<Batch> sorted = new ArrayList<Batch>(batches.values());
-    Collections.sort(sorted, new Comparator<Batch>() {
-      @Override
-      public int compare(Batch o1, Batch o2) {
-        return Longs.compare(o1.getTimestamp(), o2.getTimestamp());
-      }
-    });
-    return sorted;
-  }
-
-  /**
-   * Batch all the {@link KeyValue}s in a collection of kvs by timestamp. Updates any
-   * {@link KeyValue} with a timestamp == {@link HConstants#LATEST_TIMESTAMP} to the timestamp at
-   * the time the method is called.
-   * @param kvs {@link KeyValue}s to break into batches
-   * @param batches to update with the given kvs
-   */
-  protected void createTimestampBatchesFromKeyValues(Collection<KeyValue> kvs,
-      Map<Long, Batch> batches) {
-    long now = EnvironmentEdgeManager.currentTimeMillis();
-    byte[] nowBytes = Bytes.toBytes(now);
-
-    // batch kvs by timestamp
-    for (KeyValue kv : kvs) {
-      long ts = kv.getTimestamp();
-      // override the timestamp to the current time, so the index and primary tables match
-      // all the keys with LATEST_TIMESTAMP will then be put into the same batch
-      if (kv.updateLatestStamp(nowBytes)) {
-        ts = now;
-      }
-      Batch batch = batches.get(ts);
-      if (batch == null) {
-        batch = new Batch(ts);
-        batches.put(ts, batch);
-      }
-      batch.add(kv);
-    }
-  }
-
-  /**
-   * For a single batch, get all the index updates and add them to the updateMap
-   * <p>
-   * This method manages cleaning up the entire history of the row from the given timestamp forward
-   * for out-of-order (e.g. 'back in time') updates.
-   * <p>
-   * If things arrive out of order (client is using custom timestamps) we should still see the index
-   * in the correct order (assuming we scan after the out-of-order update in finished). Therefore,
-   * we when we aren't the most recent update to the index, we need to delete the state at the
-   * current timestamp (similar to above), but also issue a delete for the added index updates at
-   * the next newest timestamp of any of the columns in the update; we need to cleanup the insert so
-   * it looks like it was also deleted at that next newest timestamp. However, its not enough to
-   * just update the one in front of us - that column will likely be applied to index entries up the
-   * entire history in front of us, which also needs to be fixed up.
-   * <p>
-   * However, the current update usually will be the most recent thing to be added. In that case,
-   * all we need to is issue a delete for the previous index row (the state of the row, without the
-   * update applied) at the current timestamp. This gets rid of anything currently in the index for
-   * the current state of the row (at the timestamp). Then we can just follow that by applying the
-   * pending update and building the index update based on the new row state.
-   * @param updateMap map to update with new index elements
-   * @param batch timestamp-based batch of edits
-   * @param state local state to update and pass to the codec
-   * @param requireCurrentStateCleanup <tt>true</tt> if we should should attempt to cleanup the
-   *          current state of the table, in the event of a 'back in time' batch. <tt>false</tt>
-   *          indicates we should not attempt the cleanup, e.g. an earlier batch already did the
-   *          cleanup.
-   * @return <tt>true</tt> if we cleaned up the current state forward (had a back-in-time put),
-   *         <tt>false</tt> otherwise
- * @throws IOException 
-   */
-  private boolean addMutationsForBatch(IndexUpdateManager updateMap, Batch batch,
-      LocalTableState state, boolean requireCurrentStateCleanup) throws IOException {
-
-    // need a temporary manager for the current batch. It should resolve any conflicts for the
-    // current batch. Essentially, we can get the case where a batch doesn't change the current
-    // state of the index (all Puts are covered by deletes), in which case we don't want to add
-    // anything
-    // A. Get the correct values for the pending state in the batch
-    // A.1 start by cleaning up the current state - as long as there are key-values in the batch
-    // that are indexed, we need to change the current state of the index. Its up to the codec to
-    // determine if we need to make any cleanup given the pending update.
-    long batchTs = batch.getTimestamp();
-    state.setPendingUpdates(batch.getKvs());
-    addCleanupForCurrentBatch(updateMap, batchTs, state);
-
-    // A.2 do a single pass first for the updates to the current state
-    state.applyPendingUpdates();
-    long minTs = addUpdateForGivenTimestamp(batchTs, state, updateMap);
-    // if all the updates are the latest thing in the index, we are done - don't go and fix history
-    if (ColumnTracker.isNewestTime(minTs)) {
-      return false;
-    }
-
-    // A.3 otherwise, we need to roll up through the current state and get the 'correct' view of the
-    // index. after this, we have the correct view of the index, from the batch up to the index
-    while(!ColumnTracker.isNewestTime(minTs) ){
-      minTs = addUpdateForGivenTimestamp(minTs, state, updateMap);
-    }
-
-    // B. only cleanup the current state if we need to - its a huge waste of effort otherwise.
-   if (requireCurrentStateCleanup) {
-      // roll back the pending update. This is needed so we can remove all the 'old' index entries.
-      // We don't need to do the puts here, but just the deletes at the given timestamps since we
-      // just want to completely hide the incorrect entries.
-      state.rollback(batch.getKvs());
-      // setup state
-      state.setPendingUpdates(batch.getKvs());
-
-      // cleanup the pending batch. If anything in the correct history is covered by Deletes used to
-      // 'fix' history (same row key and ts), we just drop the delete (we don't want to drop both
-      // because the update may have a different set of columns or value based on the update).
-      cleanupIndexStateFromBatchOnward(updateMap, batchTs, state);
-
-      // have to roll the state forward again, so the current state is correct
-      state.applyPendingUpdates();
-      return true;
-    }
-    return false;
-  }
-
-  private long addUpdateForGivenTimestamp(long ts, LocalTableState state,
-      IndexUpdateManager updateMap) throws IOException {
-    state.setCurrentTimestamp(ts);
-    ts = addCurrentStateMutationsForBatch(updateMap, state);
-    return ts;
-  }
-
-  private void addCleanupForCurrentBatch(IndexUpdateManager updateMap, long batchTs,
-      LocalTableState state) throws IOException {
-    // get the cleanup for the current state
-    state.setCurrentTimestamp(batchTs);
-    addDeleteUpdatesToMap(updateMap, state, batchTs);
-    // ignore any index tracking from the delete
-    state.resetTrackedColumns();
-  }
-  
-  /**
-   * Add the necessary mutations for the pending batch on the local state. Handles rolling up
-   * through history to determine the index changes after applying the batch (for the case where the
-   * batch is back in time).
-   * @param updateMap to update with index mutations
-   * @param batch to apply to the current state
-   * @param state current state of the table
-   * @return the minimum timestamp across all index columns requested. If
-   *         {@link ColumnTracker#isNewestTime(long)} returns <tt>true</tt> on the returned
-   *         timestamp, we know that this <i>was not a back-in-time update</i>.
- * @throws IOException 
-   */
-  private long
-      addCurrentStateMutationsForBatch(IndexUpdateManager updateMap, LocalTableState state) throws IOException {
-
-    // get the index updates for this current batch
-    Iterable<IndexUpdate> upserts = codec.getIndexUpserts(state);
-    state.resetTrackedColumns();
-
-    /*
-     * go through all the pending updates. If we are sure that all the entries are the latest
-     * timestamp, we can just add the index updates and move on. However, if there are columns that
-     * we skip past (based on the timestamp of the batch), we need to roll back up the history.
-     * Regardless of whether or not they are the latest timestamp, the entries here are going to be
-     * correct for the current batch timestamp, so we add them to the updates. The only thing we
-     * really care about it if we need to roll up the history and fix it as we go.
-     */
-    // timestamp of the next update we need to track
-    long minTs = ColumnTracker.NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP;
-    List<IndexedColumnGroup> columnHints = new ArrayList<IndexedColumnGroup>();
-    for (IndexUpdate update : upserts) {
-      // this is the one bit where we check the timestamps
-      final ColumnTracker tracker = update.getIndexedColumns();
-      long trackerTs = tracker.getTS();
-      // update the next min TS we need to track
-      if (trackerTs < minTs) {
-        minTs = tracker.getTS();
-      }
-      // track index hints for the next round. Hint if we need an update for that column for the
-      // next timestamp. These columns clearly won't need to update as we go through time as they
-      // already match the most recent possible thing.
-      boolean needsCleanup = false;
-      if (tracker.hasNewerTimestamps()) {
-        columnHints.add(tracker);
-        // this update also needs to be cleaned up at the next timestamp because it not the latest.
-        needsCleanup = true;
-      }
-
-
-      // only make the put if the index update has been setup
-      if (update.isValid()) {
-        byte[] table = update.getTableName();
-        Mutation mutation = update.getUpdate();
-        updateMap.addIndexUpdate(table, mutation);
-
-        // only make the cleanup if we made a put and need cleanup
-        if (needsCleanup) {
-          // there is a TS for the interested columns that is greater than the columns in the
-          // put. Therefore, we need to issue a delete at the same timestamp
-          Delete d = new Delete(mutation.getRow());
-          d.setTimestamp(tracker.getTS());
-          updateMap.addIndexUpdate(table, d);
-        }
-      }
-    }
-    return minTs;
-  }
-
-  /**
-   * Cleanup the index based on the current state from the given batch. Iterates over each timestamp
-   * (for the indexed rows) for the current state of the table and cleans up all the existing
-   * entries generated by the codec.
-   * <p>
-   * Adds all pending updates to the updateMap
-   * @param updateMap updated with the pending index updates from the codec
-   * @param batchTs timestamp from which we should cleanup
-   * @param state current state of the primary table. Should already by setup to the correct state
-   *          from which we want to cleanup.
- * @throws IOException 
-   */
-  private void cleanupIndexStateFromBatchOnward(IndexUpdateManager updateMap,
-      long batchTs, LocalTableState state) throws IOException {
-    // get the cleanup for the current state
-    state.setCurrentTimestamp(batchTs);
-    addDeleteUpdatesToMap(updateMap, state, batchTs);
-    Set<ColumnTracker> trackers = state.getTrackedColumns();
-    long minTs = ColumnTracker.NO_NEWER_PRIMARY_TABLE_ENTRY_TIMESTAMP;
-    for (ColumnTracker tracker : trackers) {
-      if (tracker.getTS() < minTs) {
-        minTs = tracker.getTS();
-      }
-    }
-    state.resetTrackedColumns();
-    if (!ColumnTracker.isNewestTime(minTs)) {
-      state.setHints(Lists.newArrayList(trackers));
-      cleanupIndexStateFromBatchOnward(updateMap, minTs, state);
-    }
-  }
-
-
-  /**
-   * Get the index deletes from the codec {@link IndexCodec#getIndexDeletes(TableState)} and then
-   * add them to the update map.
-   * <p>
-   * Expects the {@link LocalTableState} to already be correctly setup (correct timestamp, updates
-   * applied, etc).
- * @throws IOException 
-   */
-  protected void
-      addDeleteUpdatesToMap(IndexUpdateManager updateMap,
-      LocalTableState state, long ts) throws IOException {
-    Iterable<IndexUpdate> cleanup = codec.getIndexDeletes(state);
-    if (cleanup != null) {
-      for (IndexUpdate d : cleanup) {
-        if (!d.isValid()) {
-          continue;
-        }
-        // override the timestamps in the delete to match the current batch.
-        Delete remove = (Delete)d.getUpdate();
-        remove.setTimestamp(ts);
-        updateMap.addIndexUpdate(d.getTableName(), remove);
-      }
-    }
-  }
-
-  @Override
-  public Collection<Pair<Mutation, byte[]>> getIndexUpdate(Delete d) throws IOException {
-    // stores all the return values
-    IndexUpdateManager updateMap = new IndexUpdateManager();
-
-    // We have to figure out which kind of delete it is, since we need to do different things if its
-    // a general (row) delete, versus a delete of just a single column or family
-    Map<byte[], List<KeyValue>> families = d.getFamilyMap();
-
-    /*
-     * Option 1: its a row delete marker, so we just need to delete the most recent state for each
-     * group, as of the specified timestamp in the delete. This can happen if we have a single row
-     * update and it is part of a batch mutation (prepare doesn't happen until later... maybe a
-     * bug?). In a single delete, this delete gets all the column families appended, so the family
-     * map won't be empty by the time it gets here.
-     */
-    if (families.size() == 0) {
-      LocalTableState state = new LocalTableState(env, localTable, d);
-      // get a consistent view of name
-      long now = d.getTimeStamp();
-      if (now == HConstants.LATEST_TIMESTAMP) {
-        now = EnvironmentEdgeManager.currentTimeMillis();
-        // update the delete's idea of 'now' to be consistent with the index
-        d.setTimestamp(now);
-      }
-      // get deletes from the codec
-      // we only need to get deletes and not add puts because this delete covers all columns
-      addDeleteUpdatesToMap(updateMap, state, now);
-
-      /*
-       * Update the current state for all the kvs in the delete. Generally, we would just iterate
-       * the family map, but since we go here, the family map is empty! Therefore, we need to fake a
-       * bunch of family deletes (just like hos HRegion#prepareDelete works). This is just needed
-       * for current version of HBase that has an issue where the batch update doesn't update the
-       * deletes before calling the hook.
-       */
-      byte[] deleteRow = d.getRow();
-      for (byte[] family : this.env.getRegion().getTableDesc().getFamiliesKeys()) {
-        state.addPendingUpdates(new KeyValue(deleteRow, family, null, now,
-            KeyValue.Type.DeleteFamily));
-      }
-    } else {
-      // Option 2: Its actually a bunch single updates, which can have different timestamps.
-      // Therefore, we need to do something similar to the put case and batch by timestamp
-      batchMutationAndAddUpdates(updateMap, d);
-    }
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Found index updates for Delete: " + d + "\n" + updateMap);
-    }
-
-    return updateMap.toMap();
-  }
-
-  @Override
-  public Collection<Pair<Mutation, byte[]>> getIndexUpdateForFilteredRows(
-      Collection<KeyValue> filtered) throws IOException {
-    // TODO Implement IndexBuilder.getIndexUpdateForFilteredRows
-    return null;
-  }
-
-  /**
-   * Exposed for testing!
-   * @param codec codec to use for this instance of the builder
-   */
-  public void setIndexCodecForTesting(IndexCodec codec) {
-    this.codec = codec;
-  }
-
-  @Override
-  public boolean isEnabled(Mutation m) throws IOException {
-    // ask the codec to see if we should even attempt indexing
-    return this.codec.isEnabled(m);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexCodec.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexCodec.java
deleted file mode 100644
index c9fc5da..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexCodec.java
+++ /dev/null
@@ -1,110 +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.hadoop.hbase.index.covered;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-
-import org.apache.phoenix.index.BaseIndexCodec;
-
-
-/**
- * Codec for creating index updates from the current state of a table.
- * <p>
- * Generally, you should extend {@link BaseIndexCodec} instead, so help maintain compatibility as
- * features need to be added to the codec, as well as potentially not haivng to implement some
- * methods.
- */
-public interface IndexCodec {
-
-  /**
-   * Do any code initialization necessary
-   * @param env environment in which the codec is operating
-   * @throws IOException if the codec cannot be initalized correctly
-   */
-  public void initialize(RegionCoprocessorEnvironment env) throws IOException;
-
-  /**
-   * Get the index cleanup entries. Currently, this must return just single row deletes (where just
-   * the row-key is specified and no columns are returned) mapped to the table name. For instance,
-   * to you have an index 'myIndex' with row :
-   * 
-   * <pre>
-   * v1,v2,v3 | CF:CQ0  | rowkey
-   *          | CF:CQ1  | rowkey
-   * </pre>
-   * 
-   * To then cleanup this entry, you would just return 'v1,v2,v3', 'myIndex'.
-   * @param state the current state of the table that needs to be cleaned up. Generally, you only
-   *          care about the latest column values, for each column you are indexing for each index
-   *          table.
-   * @return the pairs of (deletes, index table name) that should be applied.
- * @throws IOException 
-   */
-  public Iterable<IndexUpdate> getIndexDeletes(TableState state) throws IOException;
-
-  // table state has the pending update already applied, before calling
-  // get the new index entries
-  /**
-   * Get the index updates for the primary table state, for each index table. The returned
-   * {@link Put}s need to be fully specified (including timestamp) to minimize passes over the same
-   * key-values multiple times.
-   * <p>
-   * You must specify the same timestamps on the Put as {@link TableState#getCurrentTimestamp()} so
-   * the index entries match the primary table row. This could be managed at a higher level, but
-   * would require iterating all the kvs in the Put again - very inefficient when compared to the
-   * current interface where you must provide a timestamp anyways (so you might as well provide the
-   * right one).
-   * @param state the current state of the table that needs to an index update Generally, you only
-   *          care about the latest column values, for each column you are indexing for each index
-   *          table.
-   * @return the pairs of (updates,index table name) that should be applied.
- * @throws IOException 
-   */
-  public Iterable<IndexUpdate> getIndexUpserts(TableState state) throws IOException;
-
-  /**
-   * This allows the codec to dynamically change whether or not indexing should take place for a
-   * table. If it doesn't take place, we can save a lot of time on the regular Put patch. By making
-   * it dynamic, we can save offlining and then onlining a table just to turn indexing on.
-   * <p>
-   * We can also be smart about even indexing a given update here too - if the update doesn't
-   * contain any columns that we care about indexing, we can save the effort of analyzing the put
-   * and further.
-   * @param m mutation that should be indexed.
-   * @return <tt>true</tt> if indexing is enabled for the given table. This should be on a per-table
-   *         basis, as each codec is instantiated per-region.
- * @throws IOException 
-   */
-  public boolean isEnabled(Mutation m) throws IOException;
-
-  /**
-   * Get the batch identifier of the given mutation. Generally, updates to the table will take place
-   * in a batch of updates; if we know that the mutation is part of a batch, we can build the state
-   * much more intelligently.
-   * <p>
-   * <b>If you have batches that have multiple updates to the same row state, you must specify a
-   * batch id for each batch. Otherwise, we cannot guarantee index correctness</b>
-   * @param m mutation that may or may not be part of the batch
-   * @return <tt>null</tt> if the mutation is not part of a batch or an id for the batch.
-   */
-  public byte[] getBatchId(Mutation m);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexUpdate.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexUpdate.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexUpdate.java
deleted file mode 100644
index 620bc58..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/IndexUpdate.java
+++ /dev/null
@@ -1,77 +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.hadoop.hbase.index.covered;
-
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.util.Bytes;
-
-import org.apache.hadoop.hbase.index.covered.update.ColumnTracker;
-
-/**
- * Update to make to the index table.
- */
-public class IndexUpdate {
-  Mutation update;
-  byte[] tableName;
-  ColumnTracker columns;
-
-  IndexUpdate(ColumnTracker tracker) {
-    this.columns = tracker;
-  }
-
-  public void setUpdate(Mutation p) {
-    this.update = p;
-  }
-
-  public void setTable(byte[] tableName) {
-    this.tableName = tableName;
-  }
-
-  public Mutation getUpdate() {
-    return update;
-  }
-
-  public byte[] getTableName() {
-    return tableName;
-  }
-
-  public ColumnTracker getIndexedColumns() {
-    return columns;
-  }
-
-  @Override
-  public String toString() {
-    return "IndexUpdate: \n\ttable - " + Bytes.toString(tableName) + "\n\tupdate: " + update
-        + "\n\tcolumns: " + columns;
-  }
-
-  public static IndexUpdate createIndexUpdateForTesting(ColumnTracker tracker, byte[] table, Put p) {
-    IndexUpdate update = new IndexUpdate(tracker);
-    update.setTable(table);
-    update.setUpdate(p);
-    return update;
-  }
-
-  /**
-   * @return <tt>true</tt> if the necessary state for a valid index update has been set.
-   */
-  public boolean isValid() {
-    return this.tableName != null && this.update != null;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/KeyValueStore.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/KeyValueStore.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/KeyValueStore.java
deleted file mode 100644
index 0c511b9..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/KeyValueStore.java
+++ /dev/null
@@ -1,33 +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.hadoop.hbase.index.covered;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
-
-/**
- * Store a collection of KeyValues in memory.
- */
-public interface KeyValueStore {
-
-  public void add(KeyValue kv, boolean overwrite);
-
-  public KeyValueScanner getScanner();
-
-  public void rollback(KeyValue kv);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/LocalTableState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/LocalTableState.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/LocalTableState.java
deleted file mode 100644
index ec3292d..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/LocalTableState.java
+++ /dev/null
@@ -1,242 +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.hadoop.hbase.index.covered;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.hadoop.hbase.index.covered.data.IndexMemStore;
-import org.apache.hadoop.hbase.index.covered.data.LocalHBaseState;
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-import org.apache.hadoop.hbase.index.covered.update.ColumnTracker;
-import org.apache.hadoop.hbase.index.covered.update.IndexedColumnGroup;
-import org.apache.hadoop.hbase.index.scanner.Scanner;
-import org.apache.hadoop.hbase.index.scanner.ScannerBuilder;
-
-/**
- * Manage the state of the HRegion's view of the table, for the single row.
- * <p>
- * Currently, this is a single-use object - you need to create a new one for each row that you need
- * to manage. In the future, we could make this object reusable, but for the moment its easier to
- * manage as a throw-away object.
- * <p>
- * This class is <b>not</b> thread-safe - it requires external synchronization is access
- * concurrently.
- */
-public class LocalTableState implements TableState {
-
-  private long ts;
-  private RegionCoprocessorEnvironment env;
-  private KeyValueStore memstore;
-  private LocalHBaseState table;
-  private Mutation update;
-  private Set<ColumnTracker> trackedColumns = new HashSet<ColumnTracker>();
-  private ScannerBuilder scannerBuilder;
-  private List<KeyValue> kvs = new ArrayList<KeyValue>();
-  private List<? extends IndexedColumnGroup> hints;
-  private CoveredColumns columnSet;
-
-  public LocalTableState(RegionCoprocessorEnvironment environment, LocalHBaseState table, Mutation update) {
-    this.env = environment;
-    this.table = table;
-    this.update = update;
-    this.memstore = new IndexMemStore();
-    this.scannerBuilder = new ScannerBuilder(memstore, update);
-    this.columnSet = new CoveredColumns();
-  }
-
-  public void addPendingUpdates(KeyValue... kvs) {
-    if (kvs == null) return;
-    addPendingUpdates(Arrays.asList(kvs));
-  }
-
-  public void addPendingUpdates(List<KeyValue> kvs) {
-    if(kvs == null) return;
-    setPendingUpdates(kvs);
-    addUpdate(kvs);
-  }
-
-  private void addUpdate(List<KeyValue> list) {
-    addUpdate(list, true);
-  }
-
-  private void addUpdate(List<KeyValue> list, boolean overwrite) {
-    if (list == null) return;
-    for (KeyValue kv : list) {
-      this.memstore.add(kv, overwrite);
-    }
-  }
-
-  @Override
-  public RegionCoprocessorEnvironment getEnvironment() {
-    return this.env;
-  }
-
-  @Override
-  public long getCurrentTimestamp() {
-    return this.ts;
-  }
-
-  @Override
-  public void setCurrentTimestamp(long timestamp) {
-    this.ts = timestamp;
-  }
-
-  public void resetTrackedColumns() {
-    this.trackedColumns.clear();
-  }
-
-  public Set<ColumnTracker> getTrackedColumns() {
-    return this.trackedColumns;
-  }
-
-  @Override
-  public Pair<Scanner, IndexUpdate> getIndexedColumnsTableState(
-      Collection<? extends ColumnReference> indexedColumns) throws IOException {
-    ensureLocalStateInitialized(indexedColumns);
-    // filter out things with a newer timestamp and track the column references to which it applies
-    ColumnTracker tracker = new ColumnTracker(indexedColumns);
-    synchronized (this.trackedColumns) {
-      // we haven't seen this set of columns before, so we need to create a new tracker
-      if (!this.trackedColumns.contains(tracker)) {
-        this.trackedColumns.add(tracker);
-      }
-    }
-
-    Scanner scanner =
-        this.scannerBuilder.buildIndexedColumnScanner(indexedColumns, tracker, ts);
-
-    return new Pair<Scanner, IndexUpdate>(scanner, new IndexUpdate(tracker));
-  }
-
-  /**
-   * Initialize the managed local state. Generally, this will only be called by
-   * {@link #getNonIndexedColumnsTableState(List)}, which is unlikely to be called concurrently from the outside.
-   * Even then, there is still fairly low contention as each new Put/Delete will have its own table
-   * state.
-   */
-  private synchronized void ensureLocalStateInitialized(
-      Collection<? extends ColumnReference> columns) throws IOException {
-    // check to see if we haven't initialized any columns yet
-    Collection<? extends ColumnReference> toCover = this.columnSet.findNonCoveredColumns(columns);
-    // we have all the columns loaded, so we are good to go.
-    if (toCover.isEmpty()) {
-      return;
-    }
-
-    // add the current state of the row
-    this.addUpdate(this.table.getCurrentRowState(update, toCover).list(), false);
-
-    // add the covered columns to the set
-    for (ColumnReference ref : toCover) {
-      this.columnSet.addColumn(ref);
-    }
-  }
-
-  @Override
-  public Map<String, byte[]> getUpdateAttributes() {
-    return this.update.getAttributesMap();
-  }
-
-  @Override
-  public byte[] getCurrentRowKey() {
-    return this.update.getRow();
-  }
-
-  public Result getCurrentRowState() {
-    KeyValueScanner scanner = this.memstore.getScanner();
-    List<KeyValue> kvs = new ArrayList<KeyValue>();
-    while (scanner.peek() != null) {
-      try {
-        kvs.add(scanner.next());
-      } catch (IOException e) {
-        // this should never happen - something has gone terribly arwy if it has
-        throw new RuntimeException("Local MemStore threw IOException!");
-      }
-    }
-    return new Result(kvs);
-  }
-
-  /**
-   * Helper to add a {@link Mutation} to the values stored for the current row
-   * @param pendingUpdate update to apply
-   */
-  public void addUpdateForTesting(Mutation pendingUpdate) {
-    for (Map.Entry<byte[], List<KeyValue>> e : pendingUpdate.getFamilyMap().entrySet()) {
-      List<KeyValue> edits = e.getValue();
-      addUpdate(edits);
-    }
-  }
-
-  /**
-   * @param hints
-   */
-  public void setHints(List<? extends IndexedColumnGroup> hints) {
-    this.hints = hints;
-  }
-
-  @Override
-  public List<? extends IndexedColumnGroup> getIndexColumnHints() {
-    return this.hints;
-  }
-
-  @Override
-  public Collection<KeyValue> getPendingUpdate() {
-    return this.kvs;
-  }
-
-  /**
-   * Set the {@link KeyValue}s in the update for which we are currently building an index update,
-   * but don't actually apply them.
-   * @param update pending {@link KeyValue}s
-   */
-  public void setPendingUpdates(Collection<KeyValue> update) {
-    this.kvs.clear();
-    this.kvs.addAll(update);
-  }
-
-  /**
-   * Apply the {@link KeyValue}s set in {@link #setPendingUpdates(Collection)}.
-   */
-  public void applyPendingUpdates() {
-    this.addUpdate(kvs);
-  }
-
-  /**
-   * Rollback all the given values from the underlying state.
-   * @param values
-   */
-  public void rollback(Collection<KeyValue> values) {
-    for (KeyValue kv : values) {
-      this.memstore.rollback(kv);
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/TableState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/TableState.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/TableState.java
deleted file mode 100644
index 9f59d55..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/TableState.java
+++ /dev/null
@@ -1,116 +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.hadoop.hbase.index.covered;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-import org.apache.hadoop.hbase.index.covered.update.IndexedColumnGroup;
-import org.apache.hadoop.hbase.index.scanner.Scanner;
-
-/**
- * Interface for the current state of the table. This is generally going to be as of a timestamp - a
- * view on the current state of the HBase table - so you don't have to worry about exposing too much
- * information.
- */
-public interface TableState {
-
-  // use this to get batch ids/ptable stuff
-  /**
-   * WARNING: messing with this can affect the indexing plumbing. Use with caution :)
-   * @return get the current environment in which this table lives.
-   */
-  public RegionCoprocessorEnvironment getEnvironment();
-
-  /**
-   * @return the current timestamp up-to-which we are releasing table state.
-   */
-  public long getCurrentTimestamp();
-
-  /**
-   * Set the current timestamp up to which the table should allow access to the underlying table.
-   * This overrides the timestamp view provided by the indexer - use with care!
-   * @param timestamp timestamp up to which the table should allow access.
-   */
-  public void setCurrentTimestamp(long timestamp);
-
-  /**
-   * @return the attributes attached to the current update (e.g. {@link Mutation}).
-   */
-  public Map<String, byte[]> getUpdateAttributes();
-
-  /**
-   * Get a scanner on the columns that are needed by the index.
-   * <p>
-   * The returned scanner is already pre-seeked to the first {@link KeyValue} that matches the given
-   * columns with a timestamp earlier than the timestamp to which the table is currently set (the
-   * current state of the table for which we need to build an update).
-   * <p>
-   * If none of the passed columns matches any of the columns in the pending update (as determined
-   * by {@link ColumnReference#matchesFamily(byte[])} and
-   * {@link ColumnReference#matchesQualifier(byte[])}, then an empty scanner will be returned. This
-   * is because it doesn't make sense to build index updates when there is no change in the table
-   * state for any of the columns you are indexing.
-   * <p>
-   * <i>NOTE:</i> This method should <b>not</b> be used during
-   * {@link IndexCodec#getIndexDeletes(TableState)} as the pending update will not yet have been
-   * applied - you are merely attempting to cleanup the current state and therefore do <i>not</i>
-   * need to track the indexed columns.
-   * <p>
-   * As a side-effect, we update a timestamp for the next-most-recent timestamp for the columns you
-   * request - you will never see a column with the timestamp we are tracking, but the next oldest
-   * timestamp for that column.
-   * @param indexedColumns the columns to that will be indexed
-   * @return an iterator over the columns and the {@link IndexUpdate} that should be passed back to
-   *         the builder. Even if no update is necessary for the requested columns, you still need
-   *         to return the {@link IndexUpdate}, just don't set the update for the
-   *         {@link IndexUpdate}.
-   * @throws IOException
-   */
-  Pair<Scanner, IndexUpdate> getIndexedColumnsTableState(
-      Collection<? extends ColumnReference> indexedColumns) throws IOException;
-
-  /**
-   * @return the row key for the current row for which we are building an index update.
-   */
-  byte[] getCurrentRowKey();
-
-  /**
-   * Get the 'hint' for the columns that were indexed last time for the same set of keyvalues.
-   * Generally, this will only be used when fixing up a 'back in time' put or delete as we need to
-   * fix up all the indexes that reference the changed columns.
-   * @return the hint the index columns that were queried on the last iteration for the changed
-   *         column
-   */
-  List<? extends IndexedColumnGroup> getIndexColumnHints();
-
-  /**
-   * Can be used to help the codec to determine which columns it should attempt to index.
-   * @return the keyvalues in the pending update to the table.
-   */
-  Collection<KeyValue> getPendingUpdate();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/IndexMemStore.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/IndexMemStore.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/IndexMemStore.java
deleted file mode 100644
index 6290f06..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/IndexMemStore.java
+++ /dev/null
@@ -1,331 +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.hadoop.hbase.index.covered.data;
-
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.SortedSet;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.KeyComparator;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.regionserver.IndexKeyValueSkipListSet;
-import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
-import org.apache.hadoop.hbase.regionserver.MemStore;
-import org.apache.hadoop.hbase.regionserver.NonLazyKeyValueScanner;
-import org.apache.hadoop.hbase.util.Bytes;
-
-import org.apache.hadoop.hbase.index.covered.KeyValueStore;
-import org.apache.hadoop.hbase.index.covered.LocalTableState;
-
-/**
- * Like the HBase {@link MemStore}, but without all that extra work around maintaining snapshots and
- * sizing (for right now). We still support the concurrent access (in case indexes are built in
- * parallel).
- * <p>
- * 
- We basically wrap a KeyValueSkipListSet, just like a regular MemStore, except we are:
- * <ol>
- *  <li>not dealing with
- *    <ul>
- *      <li>space considerations</li>
- *      <li>a snapshot set</li>
- *    </ul>
- *  </li>
- *  <li>ignoring memstore timestamps in favor of deciding when we want to overwrite keys based on how
- *    we obtain them</li>
- *   <li>ignoring time range updates (so 
- *    {@link KeyValueScanner#shouldUseScanner(Scan, SortedSet, long)} isn't supported from 
- *    {@link #getScanner()}).</li>
- * </ol>
- * <p>
- * We can ignore the memstore timestamps because we know that anything we get from the local region
- * is going to be MVCC visible - so it should just go in. However, we also want overwrite any
- * existing state with our pending write that we are indexing, so that needs to clobber the KVs we
- * get from the HRegion. This got really messy with a regular memstore as each KV from the MemStore
- * frequently has a higher MemStoreTS, but we can't just up the pending KVs' MemStoreTs because a
- * memstore relies on the MVCC readpoint, which generally is less than {@link Long#MAX_VALUE}.
- * <p>
- * By realizing that we don't need the snapshot or space requirements, we can go much faster than
- * the previous implementation. Further, by being smart about how we manage the KVs, we can drop the
- * extra object creation we were doing to wrap the pending KVs (which we did previously to ensure
- * they sorted before the ones we got from the HRegion). We overwrite {@link KeyValue}s when we add
- * them from external sources {@link #add(KeyValue, boolean)}, but then don't overwrite existing
- * keyvalues when read them from the underlying table (because pending keyvalues should always
- * overwrite current ones) - this logic is all contained in LocalTableState.
- * @see LocalTableState
- */
-public class IndexMemStore implements KeyValueStore {
-
-  private static final Log LOG = LogFactory.getLog(IndexMemStore.class);
-  private IndexKeyValueSkipListSet kvset;
-  private Comparator<KeyValue> comparator;
-
-  /**
-   * Compare two {@link KeyValue}s based only on their row keys. Similar to the standard
-   * {@link KeyValue#COMPARATOR}, but doesn't take into consideration the memstore timestamps. We
-   * instead manage which KeyValue to retain based on how its loaded here
-   */
-  public static final Comparator<KeyValue> COMPARATOR = new Comparator<KeyValue>() {
-
-    private final KeyComparator rawcomparator = new KeyComparator();
-
-    @Override
-    public int compare(final KeyValue left, final KeyValue right) {
-      return rawcomparator.compare(left.getBuffer(), left.getOffset() + KeyValue.ROW_OFFSET,
-        left.getKeyLength(), right.getBuffer(), right.getOffset() + KeyValue.ROW_OFFSET,
-        right.getKeyLength());
-    }
-  };
-
-  public IndexMemStore() {
-    this(COMPARATOR);
-  }
-
-  /**
-   * Create a store with the given comparator. This comparator is used to determine both sort order
-   * <b>as well as equality of {@link KeyValue}s</b>.
-   * <p>
-   * Exposed for subclassing/testing.
-   * @param comparator to use
-   */
-  IndexMemStore(Comparator<KeyValue> comparator) {
-    this.comparator = comparator;
-    this.kvset = IndexKeyValueSkipListSet.create(comparator);
-  }
-
-  @Override
-  public void add(KeyValue kv, boolean overwrite) {
-    if (LOG.isDebugEnabled()) {
-      LOG.info("Inserting: " + toString(kv));
-    }
-    // if overwriting, we will always update
-    if (!overwrite) {
-      // null if there was no previous value, so we added the kv
-      kvset.putIfAbsent(kv);
-    } else {
-      kvset.add(kv);
-    }
-
-    if (LOG.isTraceEnabled()) {
-      dump();
-    }
-  }
-
-  private void dump() {
-    LOG.trace("Current kv state:\n");
-    for (KeyValue kv : this.kvset) {
-      LOG.trace("KV: " + toString(kv));
-    }
-    LOG.trace("========== END MemStore Dump ==================\n");
-  }
-
-  private String toString(KeyValue kv) {
-    return kv.toString() + "/value=" + Bytes.toString(kv.getValue());
-  }
-
-  @Override
-  public void rollback(KeyValue kv) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Rolling back: " + toString(kv));
-    }
-    // If the key is in the store, delete it
-    this.kvset.remove(kv);
-    if (LOG.isTraceEnabled()) {
-      dump();
-    }
-  }
-
-  @Override
-  public KeyValueScanner getScanner() {
-    return new MemStoreScanner();
-  }
-
-  /*
-   * MemStoreScanner implements the KeyValueScanner. It lets the caller scan the contents of a
-   * memstore -- both current map and snapshot. This behaves as if it were a real scanner but does
-   * not maintain position.
-   */
-  // This class is adapted from org.apache.hadoop.hbase.MemStore.MemStoreScanner, HBase 0.94.12
-  // It does basically the same thing as the MemStoreScanner, but it only keeps track of a single
-  // set, rather than a primary and a secondary set of KeyValues.
-  protected class MemStoreScanner extends NonLazyKeyValueScanner {
-    // Next row information for the set
-    private KeyValue nextRow = null;
-
-    // last iterated KVs for kvset and snapshot (to restore iterator state after reseek)
-    private KeyValue kvsetItRow = null;
-
-    // iterator based scanning.
-    private Iterator<KeyValue> kvsetIt;
-
-    // The kvset at the time of creating this scanner
-    volatile IndexKeyValueSkipListSet kvsetAtCreation;
-
-    MemStoreScanner() {
-      super();
-      kvsetAtCreation = kvset;
-    }
-
-    private KeyValue getNext(Iterator<KeyValue> it) {
-      // in the original implementation we cared about the current thread's readpoint from MVCC.
-      // However, we don't need to worry here because everything the index can see, is also visible
-      // to the client (or is the pending primary table update, so it will be once the index is
-      // written, so it might as well be).
-      KeyValue v = null;
-      try {
-        while (it.hasNext()) {
-          v = it.next();
-          return v;
-        }
-
-        return null;
-      } finally {
-        if (v != null) {
-          kvsetItRow = v;
-        }
-      }
-    }
-
-    /**
-     * Set the scanner at the seek key. Must be called only once: there is no thread safety between
-     * the scanner and the memStore.
-     * @param key seek value
-     * @return false if the key is null or if there is no data
-     */
-    @Override
-    public synchronized boolean seek(KeyValue key) {
-      if (key == null) {
-        close();
-        return false;
-      }
-
-      // kvset and snapshot will never be null.
-      // if tailSet can't find anything, SortedSet is empty (not null).
-      kvsetIt = kvsetAtCreation.tailSet(key).iterator();
-      kvsetItRow = null;
-
-      return seekInSubLists(key);
-    }
-
-    /**
-     * (Re)initialize the iterators after a seek or a reseek.
-     */
-    private synchronized boolean seekInSubLists(KeyValue key) {
-      nextRow = getNext(kvsetIt);
-      return nextRow != null;
-    }
-
-    /**
-     * Move forward on the sub-lists set previously by seek.
-     * @param key seek value (should be non-null)
-     * @return true if there is at least one KV to read, false otherwise
-     */
-    @Override
-    public synchronized boolean reseek(KeyValue key) {
-      /*
-       * See HBASE-4195 & HBASE-3855 & HBASE-6591 for the background on this implementation. This
-       * code is executed concurrently with flush and puts, without locks. Two points must be known
-       * when working on this code: 1) It's not possible to use the 'kvTail' and 'snapshot'
-       * variables, as they are modified during a flush. 2) The ideal implementation for performance
-       * would use the sub skip list implicitly pointed by the iterators 'kvsetIt' and 'snapshotIt'.
-       * Unfortunately the Java API does not offer a method to get it. So we remember the last keys
-       * we iterated to and restore the reseeked set to at least that point.
-       */
-
-      kvsetIt = kvsetAtCreation.tailSet(getHighest(key, kvsetItRow)).iterator();
-      return seekInSubLists(key);
-    }
-
-    /*
-     * Returns the higher of the two key values, or null if they are both null. This uses
-     * comparator.compare() to compare the KeyValue using the memstore comparator.
-     */
-    private KeyValue getHighest(KeyValue first, KeyValue second) {
-      if (first == null && second == null) {
-        return null;
-      }
-      if (first != null && second != null) {
-        int compare = comparator.compare(first, second);
-        return (compare > 0 ? first : second);
-      }
-      return (first != null ? first : second);
-    }
-
-    @Override
-    public synchronized KeyValue peek() {
-      // DebugPrint.println(" MS@" + hashCode() + " peek = " + getLowest());
-      return nextRow;
-    }
-
-    @Override
-    public synchronized KeyValue next() {
-      if (nextRow == null) {
-        return null;
-      }
-
-      final KeyValue ret = nextRow;
-
-      // Advance the iterators
-      nextRow = getNext(kvsetIt);
-
-      return ret;
-    }
-
-    @Override
-    public synchronized void close() {
-      this.nextRow = null;
-      this.kvsetIt = null;
-      this.kvsetItRow = null;
-    }
-
-    /**
-     * MemStoreScanner returns max value as sequence id because it will always have the latest data
-     * among all files.
-     */
-    @Override
-    public long getSequenceID() {
-      return Long.MAX_VALUE;
-    }
-
-    @Override
-    public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns, long oldestUnexpiredTS) {
-      throw new UnsupportedOperationException(this.getClass().getName()
-          + " doesn't support checking to see if it should use a scanner!");
-    }
-
-    /*
-    @Override
-    public boolean backwardSeek(KeyValue arg0) throws IOException {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public boolean seekToLastRow() throws IOException {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public boolean seekToPreviousRow(KeyValue arg0) throws IOException {
-        throw new UnsupportedOperationException();
-    }
-    */
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LazyValueGetter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LazyValueGetter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LazyValueGetter.java
deleted file mode 100644
index 7329e8f..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LazyValueGetter.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.index.covered.data;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.hbase.KeyValue;
-
-import org.apache.hadoop.hbase.index.ValueGetter;
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-import org.apache.hadoop.hbase.index.scanner.Scanner;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-/**
- * {@link ValueGetter} that uses lazy initialization to get the value for the given
- * {@link ColumnReference}. Once stored, the mapping for that reference is retained.
- */
-public class LazyValueGetter implements ValueGetter {
-
-  private Scanner scan;
-  private volatile Map<ColumnReference, ImmutableBytesPtr> values;
-  private byte[] row;
-  
-  /**
-   * Back the getter with a {@link Scanner} to actually access the local data.
-   * @param scan backing scanner
-   * @param currentRow row key for the row to seek in the scanner
-   */
-  public LazyValueGetter(Scanner scan, byte[] currentRow) {
-    this.scan = scan;
-    this.row = currentRow;
-  }
-
-  @Override
-  public ImmutableBytesPtr getLatestValue(ColumnReference ref) throws IOException {
-    // ensure we have a backing map
-    if (values == null) {
-      synchronized (this) {
-        values = Collections.synchronizedMap(new HashMap<ColumnReference, ImmutableBytesPtr>());
-      }
-    }
-
-    // check the value in the map
-    ImmutableBytesPtr value = values.get(ref);
-    if (value == null) {
-      value = get(ref);
-      values.put(ref, value);
-    }
-
-    return value;
-  }
-
-  /**
-   * @param ref
-   * @return the first value on the scanner for the given column
-   */
-  private ImmutableBytesPtr get(ColumnReference ref) throws IOException {
-    KeyValue first = ref.getFirstKeyValueForRow(row);
-    if (!scan.seek(first)) {
-      return null;
-    }
-    // there is a next value - we only care about the current value, so we can just snag that
-    KeyValue next = scan.next();
-    if (ref.matches(next)) {
-      return new ImmutableBytesPtr(next.getBuffer(), next.getValueOffset(), next.getValueLength());
-    }
-    return null;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalHBaseState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalHBaseState.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalHBaseState.java
deleted file mode 100644
index 97c4d22..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalHBaseState.java
+++ /dev/null
@@ -1,47 +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.hadoop.hbase.index.covered.data;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Result;
-
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-
-/**
- * Access the current state of the row in the local HBase table, given a mutation
- */
-public interface LocalHBaseState {
-
-  /**
-   * @param m mutation for which we should get the current table state
-   * @param toCover all the columns the current row state needs to cover; hint the underlying lookup
-   *          to save getting all the columns for the row
-   * @return the full state of the given row. Includes all current versions (even if they are not
-   *         usually visible to the client (unless they are also doing a raw scan)). Never returns a
-   *         <tt>null</tt> {@link Result} - instead, when there is not data for the row, returns a
-   *         {@link Result} with no stored {@link KeyValue}s.
-   * @throws IOException if there is an issue reading the row
-   */
-  public Result getCurrentRowState(Mutation m, Collection<? extends ColumnReference> toCover)
-      throws IOException;
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalTable.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalTable.java
deleted file mode 100644
index 8719e33..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/data/LocalTable.java
+++ /dev/null
@@ -1,72 +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.hadoop.hbase.index.covered.data;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.RegionScanner;
-
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
-
-/**
- * Wrapper around a lazily instantiated, local HTable.
- * <p>
- * Previously, we had used various row and batch caches. However, this ends up being very
- * complicated when attempting manage updating and invalidating the cache with no real gain as any
- * row accessed multiple times will likely be in HBase's block cache, invalidating any extra caching
- * we are doing here. In the end, its simpler and about as efficient to just get the current state
- * of the row from HBase and let HBase manage caching the row from disk on its own.
- */
-public class LocalTable implements LocalHBaseState {
-
-  private RegionCoprocessorEnvironment env;
-
-  public LocalTable(RegionCoprocessorEnvironment env) {
-    this.env = env;
-  }
-
-  @Override
-  public Result getCurrentRowState(Mutation m, Collection<? extends ColumnReference> columns)
-      throws IOException {
-    byte[] row = m.getRow();
-    // need to use a scan here so we can get raw state, which Get doesn't provide.
-    Scan s = IndexManagementUtil.newLocalStateScan(Collections.singletonList(columns));
-    s.setStartRow(row);
-    s.setStopRow(row);
-    HRegion region = this.env.getRegion();
-    RegionScanner scanner = region.getScanner(s);
-    List<KeyValue> kvs = new ArrayList<KeyValue>(1);
-    boolean more = scanner.next(kvs);
-    assert !more : "Got more than one result when scanning" + " a single row in the primary table!";
-
-    Result r = new Result(kvs);
-    scanner.close();
-    return r;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/ColumnGroup.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/ColumnGroup.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/ColumnGroup.java
deleted file mode 100644
index b7c22cf..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/ColumnGroup.java
+++ /dev/null
@@ -1,112 +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.hadoop.hbase.index.covered.example;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.hadoop.hbase.util.Bytes;
-
-
-/**
- * A collection of {@link CoveredColumn}s that should be included in a covered index.
- */
-public class ColumnGroup implements Iterable<CoveredColumn> {
-
-  private List<CoveredColumn> columns = new ArrayList<CoveredColumn>();
-  private String table;
-
-  public ColumnGroup(String tableName) {
-    this.table = tableName;
-  }
-
-  public void add(CoveredColumn column) {
-    this.columns.add(column);
-  }
-
-  public String getTable() {
-    return table;
-  }
-
-  /**
-   * Check to see if any {@link CoveredColumn} in <tt>this</tt> matches the given family
-   * @param family to check
-   * @return <tt>true</tt> if any column covers the family
-   */
-  public boolean matches(String family) {
-    for (CoveredColumn column : columns) {
-      if (column.matchesFamily(family)) {
-        return true;
-      }
-    }
-
-    return false;
-  }
-
-  /**
-   * Check to see if any column matches the family/qualifier pair
-   * @param family family to match against
-   * @param qualifier qualifier to match, can be <tt>null</tt>, in which case we match all
-   *          qualifiers
-   * @return <tt>true</tt> if any column matches, <tt>false</tt> otherwise
-   */
-  public boolean matches(byte[] family, byte[] qualifier) {
-    // families are always printable characters
-    String fam = Bytes.toString(family);
-    for (CoveredColumn column : columns) {
-      if (column.matchesFamily(fam)) {
-        // check the qualifier
-          if (column.matchesQualifier(qualifier)) {
-            return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  /**
-   * @return the number of columns in the group
-   */
-  public int size() {
-    return this.columns.size();
-  }
-
-  @Override
-  public Iterator<CoveredColumn> iterator() {
-    return columns.iterator();
-  }
-
-  /**
-   * @param index index of the column to get
-   * @return the specified column
-   */
-  public CoveredColumn getColumnForTesting(int index) {
-    return this.columns.get(index);
-  }
-
-  @Override
-  public String toString() {
-    return "ColumnGroup - table: " + table + ", columns: " + columns;
-  }
-
-  public List<CoveredColumn> getColumns() {
-    return this.columns;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumn.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumn.java
deleted file mode 100644
index 640cf8a..0000000
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/index/covered/example/CoveredColumn.java
+++ /dev/null
@@ -1,107 +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.hadoop.hbase.index.covered.example;
-
-import org.apache.hadoop.hbase.util.Bytes;
-
-import org.apache.hadoop.hbase.index.covered.update.ColumnReference;
-
-/**
- * A single Column (either a Column Family or a full Family:Qualifier pair) in a {@link ColumnGroup}
- * . If no column qualifier is specified (null), matches all known qualifiers of the family.
- */
-public class CoveredColumn extends ColumnReference {
-
-  public static final String SEPARATOR = ":";
-  String familyString;
-  private final int hashCode;
-
-  public CoveredColumn(byte[] family, byte[] qualifier){
-    this(Bytes.toString(family), qualifier);
-  }
-
-  public CoveredColumn(String family, byte[] qualifier) {
-    super(Bytes.toBytes(family), qualifier == null ? ColumnReference.ALL_QUALIFIERS : qualifier);
-    this.familyString = family;
-    this.hashCode = calcHashCode(family, qualifier);
-  }
-
-  public static CoveredColumn parse(String spec) {
-    int sep = spec.indexOf(SEPARATOR);
-    if (sep < 0) {
-      throw new IllegalArgumentException(spec + " is not a valid specifier!");
-    }
-    String family = spec.substring(0, sep);
-    String qual = spec.substring(sep + 1);
-    byte[] column = qual.length() == 0 ? null : Bytes.toBytes(qual);
-    return new CoveredColumn(family, column);
-  }
-
-  public String serialize() {
-    return CoveredColumn.serialize(familyString, qualifier);
-  }
-
-  public static String serialize(String first, byte[] second) {
-    String nextValue = first + CoveredColumn.SEPARATOR;
-    if (second != null) {
-      nextValue += Bytes.toString(second);
-    }
-    return nextValue;
-  }
-
-  /**
-   * @param family2 to check
-   * @return <tt>true</tt> if the passed family matches the family this column covers
-   */
-  public boolean matchesFamily(String family2) {
-    return this.familyString.equals(family2);
-  }
-
-  @Override
-  public int hashCode() {
-    return hashCode;
-  }
-
-  private static int calcHashCode(String familyString, byte[] qualifier) {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + familyString.hashCode();
-    if (qualifier != null) {
-      result = prime * result + Bytes.hashCode(qualifier);
-    }
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) return true;
-    if (!super.equals(obj)) return false;
-    if (getClass() != obj.getClass()) return false;
-    CoveredColumn other = (CoveredColumn) obj;
-    if (hashCode != other.hashCode) return false;
-    if (!familyString.equals(other.familyString)) return false;
-    return Bytes.equals(qualifier, other.qualifier);
-  }
-
-  @Override
-  public String toString() {
-    String qualString = qualifier == null ? "null" : Bytes.toString(qualifier);
-    return "CoveredColumn:[" + familyString + ":" + qualString + "]";
-  }
-}
\ No newline at end of file


[04/15] Rename package from org.apache.hadoop.hbase.index.* to org.apache.phoenix.index.* to fix classloader issue causing mutable index performance regression - https://issues.apache.org/jira/browse/PHOENIX-38

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/parallel/TestThreadPoolManager.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/parallel/TestThreadPoolManager.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/parallel/TestThreadPoolManager.java
deleted file mode 100644
index 0d6ac86..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/parallel/TestThreadPoolManager.java
+++ /dev/null
@@ -1,93 +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.hadoop.hbase.index.parallel;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.conf.Configuration;
-import org.junit.Rule;
-import org.junit.Test;
-
-import org.apache.hadoop.hbase.index.TableName;
-
-public class TestThreadPoolManager {
-
-  @Rule
-  public TableName name = new TableName();
-
-  @Test
-  public void testShutdownGetsNewThreadPool() throws Exception{
-    Map<String, Object> cache = new HashMap<String, Object>();
-    ThreadPoolBuilder builder = new ThreadPoolBuilder(name.getTableNameString(), new Configuration(false));
-    ThreadPoolExecutor exec = ThreadPoolManager.getExecutor(builder, cache);
-    assertNotNull("Got a null exector from the pool!", exec);
-    //shutdown the pool and ensure that it actually shutdown
-    exec.shutdown();
-    ThreadPoolExecutor exec2 = ThreadPoolManager.getExecutor(builder, cache);
-    assertFalse("Got the same exectuor, even though the original shutdown", exec2 == exec);
-  }
-
-  @Test
-  public void testShutdownWithReferencesDoesNotStopExecutor() throws Exception {
-    Map<String, Object> cache = new HashMap<String, Object>();
-    ThreadPoolBuilder builder =
-        new ThreadPoolBuilder(name.getTableNameString(), new Configuration(false));
-    ThreadPoolExecutor exec = ThreadPoolManager.getExecutor(builder, cache);
-    assertNotNull("Got a null exector from the pool!", exec);
-    ThreadPoolExecutor exec2 = ThreadPoolManager.getExecutor(builder, cache);
-    assertTrue("Should have gotten the same executor", exec2 == exec);
-    exec.shutdown();
-    assertFalse("Executor is shutting down, even though we have a live reference!",
-      exec.isShutdown() || exec.isTerminating());
-    exec2.shutdown();
-    // wait 5 minutes for thread pool to shutdown
-    assertTrue("Executor is NOT shutting down, after releasing live reference!",
-      exec.awaitTermination(300, TimeUnit.SECONDS));
-  }
-
-  @Test
-  public void testGetExpectedExecutorForName() throws Exception {
-    Map<String, Object> cache = new HashMap<String, Object>();
-    ThreadPoolBuilder builder =
-        new ThreadPoolBuilder(name.getTableNameString(), new Configuration(false));
-    ThreadPoolExecutor exec = ThreadPoolManager.getExecutor(builder, cache);
-    assertNotNull("Got a null exector from the pool!", exec);
-    ThreadPoolExecutor exec2 = ThreadPoolManager.getExecutor(builder, cache);
-    assertTrue("Got a different exectuor, even though they have the same name", exec2 == exec);
-    builder = new ThreadPoolBuilder(name.getTableNameString(), new Configuration(false));
-    exec2 = ThreadPoolManager.getExecutor(builder, cache);
-    assertTrue(
-      "Got a different exectuor, even though they have the same name, but different confs",
-      exec2 == exec);
-
-    builder =
-        new ThreadPoolBuilder(name.getTableNameString() + "-some-other-pool", new Configuration(
-            false));
-    exec2 = ThreadPoolManager.getExecutor(builder, cache);
-    assertFalse(
-      "Got a different exectuor, even though they have the same name, but different confs",
-      exec2 == exec);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/util/TestIndexManagementUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/util/TestIndexManagementUtil.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/util/TestIndexManagementUtil.java
deleted file mode 100644
index b09d6ba..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/util/TestIndexManagementUtil.java
+++ /dev/null
@@ -1,66 +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.hadoop.hbase.index.util;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.regionserver.wal.IndexedHLogReader;
-import org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec;
-import org.apache.hadoop.hbase.regionserver.wal.WALEditCodec;
-import org.junit.Test;
-
-public class TestIndexManagementUtil {
-
-  @Test
-  public void testUncompressedWal() throws Exception {
-    Configuration conf = new Configuration(false);
-    // works with WALEditcodec
-    conf.set(WALEditCodec.WAL_EDIT_CODEC_CLASS_KEY, IndexedWALEditCodec.class.getName());
-    IndexManagementUtil.ensureMutableIndexingCorrectlyConfigured(conf);
-    // clear the codec and set the wal reader
-    conf = new Configuration(false);
-    conf.set(IndexManagementUtil.HLOG_READER_IMPL_KEY, IndexedHLogReader.class.getName());
-    IndexManagementUtil.ensureMutableIndexingCorrectlyConfigured(conf);
-  }
-
-  /**
-   * Compressed WALs are supported when we have the WALEditCodec installed
-   * @throws Exception
-   */
-  @Test
-  public void testCompressedWALWithCodec() throws Exception {
-    Configuration conf = new Configuration(false);
-    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
-    // works with WALEditcodec
-    conf.set(WALEditCodec.WAL_EDIT_CODEC_CLASS_KEY, IndexedWALEditCodec.class.getName());
-    IndexManagementUtil.ensureMutableIndexingCorrectlyConfigured(conf);
-  }
-
-  /**
-   * We cannot support WAL Compression with the IndexedHLogReader
-   * @throws Exception
-   */
-  @Test(expected = IllegalStateException.class)
-  public void testCompressedWALWithHLogReader() throws Exception {
-    Configuration conf = new Configuration(false);
-    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
-    // works with WALEditcodec
-    conf.set(IndexManagementUtil.HLOG_READER_IMPL_KEY, IndexedHLogReader.class.getName());
-    IndexManagementUtil.ensureMutableIndexingCorrectlyConfigured(conf);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/FakeTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/FakeTableFactory.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/FakeTableFactory.java
deleted file mode 100644
index acabe2a..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/FakeTableFactory.java
+++ /dev/null
@@ -1,51 +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.hadoop.hbase.index.write;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.hadoop.hbase.client.HTableInterface;
-
-import org.apache.hadoop.hbase.index.table.HTableFactory;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-/**
- * Simple table factory that just looks up the tables based on name. Useful for mocking up
- * {@link HTableInterface}s without having to mock up the factory too.
- */
-class FakeTableFactory implements HTableFactory {
-
-  boolean shutdown = false;
-  private Map<ImmutableBytesPtr, HTableInterface> tables;
-
-  public FakeTableFactory(Map<ImmutableBytesPtr, HTableInterface> tables) {
-    this.tables = tables;
-  }
-
-  @Override
-  public HTableInterface getTable(ImmutableBytesPtr tablename) throws IOException {
-    return this.tables.get(tablename);
-  }
-
-  @Override
-  public void shutdown() {
-    shutdown = true;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestCachingHTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestCachingHTableFactory.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestCachingHTableFactory.java
deleted file mode 100644
index 740fda8..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestCachingHTableFactory.java
+++ /dev/null
@@ -1,58 +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.hadoop.hbase.index.write;
-
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-import org.apache.hadoop.hbase.index.table.CachingHTableFactory;
-import org.apache.hadoop.hbase.index.table.HTableFactory;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-public class TestCachingHTableFactory {
-
-  @Test
-  public void testCacheCorrectlyExpiresTable() throws Exception {
-    // setup the mocks for the tables we will request
-    HTableFactory delegate = Mockito.mock(HTableFactory.class);
-    ImmutableBytesPtr t1 = new ImmutableBytesPtr(Bytes.toBytes("t1"));
-    ImmutableBytesPtr t2 = new ImmutableBytesPtr(Bytes.toBytes("t2"));
-    ImmutableBytesPtr t3 = new ImmutableBytesPtr(Bytes.toBytes("t3"));
-    HTableInterface table1 = Mockito.mock(HTableInterface.class);
-    HTableInterface table2 = Mockito.mock(HTableInterface.class);
-    HTableInterface table3 = Mockito.mock(HTableInterface.class);
-    Mockito.when(delegate.getTable(t1)).thenReturn(table1);
-    Mockito.when(delegate.getTable(t2)).thenReturn(table2);
-    Mockito.when(delegate.getTable(t3)).thenReturn(table3);
-    
-    // setup our factory with a cache size of 2
-    CachingHTableFactory factory = new CachingHTableFactory(delegate, 2);
-    factory.getTable(t1);
-    factory.getTable(t2);
-    factory.getTable(t3);
-    // get the same table a second time, after it has gone out of cache
-    factory.getTable(t1);
-    
-    Mockito.verify(delegate, Mockito.times(2)).getTable(t1);
-    Mockito.verify(delegate, Mockito.times(1)).getTable(t2);
-    Mockito.verify(delegate, Mockito.times(1)).getTable(t3);
-    Mockito.verify(table1).close();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestIndexWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestIndexWriter.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestIndexWriter.java
deleted file mode 100644
index fe6db47..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestIndexWriter.java
+++ /dev/null
@@ -1,284 +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.hadoop.hbase.index.write;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.junit.Rule;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import org.apache.hadoop.hbase.index.StubAbortable;
-import org.apache.hadoop.hbase.index.TableName;
-import org.apache.hadoop.hbase.index.exception.IndexWriteException;
-import org.apache.hadoop.hbase.index.exception.SingleIndexWriteFailureException;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-public class TestIndexWriter {
-  private static final Log LOG = LogFactory.getLog(TestIndexWriter.class);
-  @Rule
-  public TableName testName = new TableName();
-  private final byte[] row = Bytes.toBytes("row");
-
-  @Test
-  public void getDefaultWriter() throws Exception {
-    Configuration conf = new Configuration(false);
-    RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
-    Mockito.when(env.getConfiguration()).thenReturn(conf);
-    assertNotNull(IndexWriter.getCommitter(env));
-  }
-
-  @Test
-  public void getDefaultFailurePolicy() throws Exception {
-    Configuration conf = new Configuration(false);
-    RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
-    Mockito.when(env.getConfiguration()).thenReturn(conf);
-    assertNotNull(IndexWriter.getFailurePolicy(env));
-  }
-
-  /**
-   * With the move to using a pool of threads to write, we need to ensure that we still block until
-   * all index writes for a mutation/batch are completed.
-   * @throws Exception on failure
-   */
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testSynchronouslyCompletesAllWrites() throws Exception {
-    LOG.info("Starting " + testName.getTableNameString());
-    LOG.info("Current thread is interrupted: " + Thread.interrupted());
-    Abortable abort = new StubAbortable();
-    Stoppable stop = Mockito.mock(Stoppable.class);
-    ExecutorService exec = Executors.newFixedThreadPool(1);
-    Map<ImmutableBytesPtr, HTableInterface> tables = new HashMap<ImmutableBytesPtr, HTableInterface>();
-    FakeTableFactory factory = new FakeTableFactory(tables);
-
-    byte[] tableName = this.testName.getTableName();
-    Put m = new Put(row);
-    m.add(Bytes.toBytes("family"), Bytes.toBytes("qual"), null);
-    Collection<Pair<Mutation, byte[]>> indexUpdates = Arrays.asList(new Pair<Mutation, byte[]>(m,
-        tableName));
-
-    HTableInterface table = Mockito.mock(HTableInterface.class);
-    final boolean[] completed = new boolean[] { false };
-    Mockito.when(table.batch(Mockito.anyList())).thenAnswer(new Answer<Void>() {
-
-      @Override
-      public Void answer(InvocationOnMock invocation) throws Throwable {
-        // just keep track that it was called
-        completed[0] = true;
-        return null;
-      }
-    });
-    Mockito.when(table.getTableName()).thenReturn(testName.getTableName());
-    // add the table to the set of tables, so its returned to the writer
-    tables.put(new ImmutableBytesPtr(tableName), table);
-
-    // setup the writer and failure policy
-    ParallelWriterIndexCommitter committer = new ParallelWriterIndexCommitter();
-    committer.setup(factory, exec, abort, stop, 2);
-    KillServerOnFailurePolicy policy = new KillServerOnFailurePolicy();
-    policy.setup(stop, abort);
-    IndexWriter writer = new IndexWriter(committer, policy);
-    writer.write(indexUpdates);
-    assertTrue("Writer returned before the table batch completed! Likely a race condition tripped",
-      completed[0]);
-    writer.stop(this.testName.getTableNameString() + " finished");
-    assertTrue("Factory didn't get shutdown after writer#stop!", factory.shutdown);
-    assertTrue("ExectorService isn't terminated after writer#stop!", exec.isShutdown());
-  }
-
-  /**
-   * Index updates can potentially be queued up if there aren't enough writer threads. If a running
-   * index write fails, then we should early exit the pending indexupdate, when it comes up (if the
-   * pool isn't already shutdown).
-   * <p>
-   * This test is a little bit racey - we could actually have the failure of the first task before
-   * the third task is even submitted. However, we should never see the third task attempt to make
-   * the batch write, so we should never see a failure here.
-   * @throws Exception on failure
-   */
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testFailureOnRunningUpdateAbortsPending() throws Exception {
-    Abortable abort = new StubAbortable();
-    Stoppable stop = Mockito.mock(Stoppable.class);
-    // single thread factory so the older request gets queued
-    ExecutorService exec = Executors.newFixedThreadPool(1);
-    Map<ImmutableBytesPtr, HTableInterface> tables = new HashMap<ImmutableBytesPtr, HTableInterface>();
-    FakeTableFactory factory = new FakeTableFactory(tables);
-
-    // updates to two different tables
-    byte[] tableName = Bytes.add(this.testName.getTableName(), new byte[] { 1, 2, 3, 4 });
-    Put m = new Put(row);
-    m.add(Bytes.toBytes("family"), Bytes.toBytes("qual"), null);
-    byte[] tableName2 = this.testName.getTableName();// this will sort after the first tablename
-    List<Pair<Mutation, byte[]>> indexUpdates = new ArrayList<Pair<Mutation, byte[]>>();
-    indexUpdates.add(new Pair<Mutation, byte[]>(m, tableName));
-    indexUpdates.add(new Pair<Mutation, byte[]>(m, tableName2));
-    indexUpdates.add(new Pair<Mutation, byte[]>(m, tableName2));
-
-    // first table will fail
-    HTableInterface table = Mockito.mock(HTableInterface.class);
-    Mockito.when(table.batch(Mockito.anyList())).thenThrow(
-      new IOException("Intentional IOException for failed first write."));
-    Mockito.when(table.getTableName()).thenReturn(tableName);
-
-    // second table just blocks to make sure that the abort propagates to the third task
-    final CountDownLatch waitOnAbortedLatch = new CountDownLatch(1);
-    final boolean[] failed = new boolean[] { false };
-    HTableInterface table2 = Mockito.mock(HTableInterface.class);
-    Mockito.when(table2.getTableName()).thenReturn(tableName2);
-    Mockito.when(table2.batch(Mockito.anyList())).thenAnswer(new Answer<Void>() {
-      @Override
-      public Void answer(InvocationOnMock invocation) throws Throwable {
-        waitOnAbortedLatch.await();
-        return null;
-      }
-    }).thenAnswer(new Answer<Void>() {
-      @Override
-      public Void answer(InvocationOnMock invocation) throws Throwable {
-        failed[0] = true;
-        throw new RuntimeException(
-            "Unexpected exception - second index table shouldn't have been written to");
-      }
-    });
-
-    // add the tables to the set of tables, so its returned to the writer
-    tables.put(new ImmutableBytesPtr(tableName), table);
-    tables.put(new ImmutableBytesPtr(tableName2), table2);
-
-    ParallelWriterIndexCommitter committer = new ParallelWriterIndexCommitter();
-    committer.setup(factory, exec, abort, stop, 2);
-    KillServerOnFailurePolicy policy = new KillServerOnFailurePolicy();
-    policy.setup(stop, abort);
-    IndexWriter writer = new IndexWriter(committer, policy);
-    try {
-      writer.write(indexUpdates);
-      fail("Should not have successfully completed all index writes");
-    } catch (SingleIndexWriteFailureException e) {
-      LOG.info("Correctly got a failure to reach the index", e);
-      // should have correctly gotten the correct abort, so let the next task execute
-      waitOnAbortedLatch.countDown();
-    }
-    assertFalse(
-      "Third set of index writes never have been attempted - should have seen the abort before done!",
-      failed[0]);
-    writer.stop(this.testName.getTableNameString() + " finished");
-    assertTrue("Factory didn't get shutdown after writer#stop!", factory.shutdown);
-    assertTrue("ExectorService isn't terminated after writer#stop!", exec.isShutdown());
-  }
-
-  /**
-   * Test that if we get an interruption to to the thread while doing a batch (e.g. via shutdown),
-   * that we correctly end the task
-   * @throws Exception on failure
-   */
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testShutdownInterruptsAsExpected() throws Exception {
-    Stoppable stop = Mockito.mock(Stoppable.class);
-    Abortable abort = new StubAbortable();
-    // single thread factory so the older request gets queued
-    ExecutorService exec = Executors.newFixedThreadPool(1);
-    Map<ImmutableBytesPtr, HTableInterface> tables = new HashMap<ImmutableBytesPtr, HTableInterface>();
-    FakeTableFactory factory = new FakeTableFactory(tables);
-
-    byte[] tableName = this.testName.getTableName();
-    HTableInterface table = Mockito.mock(HTableInterface.class);
-    Mockito.when(table.getTableName()).thenReturn(tableName);
-    final CountDownLatch writeStartedLatch = new CountDownLatch(1);
-    // latch never gets counted down, so we wait forever
-    final CountDownLatch waitOnAbortedLatch = new CountDownLatch(1);
-    Mockito.when(table.batch(Mockito.anyList())).thenAnswer(new Answer<Void>() {
-      @Override
-      public Void answer(InvocationOnMock invocation) throws Throwable {
-        LOG.info("Write started");
-        writeStartedLatch.countDown();
-        // when we interrupt the thread for shutdown, we should see this throw an interrupt too
-        try {
-        waitOnAbortedLatch.await();
-        } catch (InterruptedException e) {
-          LOG.info("Correctly interrupted while writing!");
-          throw e;
-        }
-        return null;
-      }
-    });
-    // add the tables to the set of tables, so its returned to the writer
-    tables.put(new ImmutableBytesPtr(tableName), table);
-
-    // update a single table
-    Put m = new Put(row);
-    m.add(Bytes.toBytes("family"), Bytes.toBytes("qual"), null);
-    final List<Pair<Mutation, byte[]>> indexUpdates = new ArrayList<Pair<Mutation, byte[]>>();
-    indexUpdates.add(new Pair<Mutation, byte[]>(m, tableName));
-
-    // setup the writer
-    ParallelWriterIndexCommitter committer = new ParallelWriterIndexCommitter();
-    committer.setup(factory, exec, abort, stop, 2);
-    KillServerOnFailurePolicy policy = new KillServerOnFailurePolicy();
-    policy.setup(stop, abort);
-    final IndexWriter writer = new IndexWriter(committer, policy);
-
-    final boolean[] failedWrite = new boolean[] { false };
-    Thread primaryWriter = new Thread() {
-
-      @Override
-      public void run() {
-        try {
-          writer.write(indexUpdates);
-        } catch (IndexWriteException e) {
-          failedWrite[0] = true;
-        }
-      }
-    };
-    primaryWriter.start();
-    // wait for the write to start before intentionally shutdown the pool
-    writeStartedLatch.await();
-    writer.stop("Shutting down writer for test " + this.testName.getTableNameString());
-    primaryWriter.join();
-    assertTrue("Writer should have failed because of the stop we issued", failedWrite[0]);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestParalleIndexWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestParalleIndexWriter.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestParalleIndexWriter.java
deleted file mode 100644
index 05c0a4c..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestParalleIndexWriter.java
+++ /dev/null
@@ -1,117 +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.hadoop.hbase.index.write;
-
-import static org.junit.Assert.assertTrue;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Rule;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.index.StubAbortable;
-import org.apache.hadoop.hbase.index.TableName;
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-public class TestParalleIndexWriter {
-
-  private static final Log LOG = LogFactory.getLog(TestParalleIndexWriter.class);
-  @Rule
-  public TableName test = new TableName();
-  private final byte[] row = Bytes.toBytes("row");
-
-  @Test
-  public void testCorrectlyCleansUpResources() throws Exception{
-    ExecutorService exec = Executors.newFixedThreadPool(1);
-    FakeTableFactory factory = new FakeTableFactory(
-        Collections.<ImmutableBytesPtr, HTableInterface> emptyMap());
-    ParallelWriterIndexCommitter writer = new ParallelWriterIndexCommitter();
-    Abortable mockAbort = Mockito.mock(Abortable.class);
-    Stoppable mockStop = Mockito.mock(Stoppable.class);
-    // create a simple writer
-    writer.setup(factory, exec, mockAbort, mockStop, 1);
-    // stop the writer
-    writer.stop(this.test.getTableNameString() + " finished");
-    assertTrue("Factory didn't get shutdown after writer#stop!", factory.shutdown);
-    assertTrue("ExectorService isn't terminated after writer#stop!", exec.isShutdown());
-    Mockito.verifyZeroInteractions(mockAbort, mockStop);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testSynchronouslyCompletesAllWrites() throws Exception {
-    LOG.info("Starting " + test.getTableNameString());
-    LOG.info("Current thread is interrupted: " + Thread.interrupted());
-    Abortable abort = new StubAbortable();
-    Stoppable stop = Mockito.mock(Stoppable.class);
-    ExecutorService exec = Executors.newFixedThreadPool(1);
-    Map<ImmutableBytesPtr, HTableInterface> tables =
-        new HashMap<ImmutableBytesPtr, HTableInterface>();
-    FakeTableFactory factory = new FakeTableFactory(tables);
-
-    ImmutableBytesPtr tableName = new ImmutableBytesPtr(this.test.getTableName());
-    Put m = new Put(row);
-    m.add(Bytes.toBytes("family"), Bytes.toBytes("qual"), null);
-    Multimap<HTableInterfaceReference, Mutation> indexUpdates =
-        ArrayListMultimap.<HTableInterfaceReference, Mutation> create();
-    indexUpdates.put(new HTableInterfaceReference(tableName), m);
-
-    HTableInterface table = Mockito.mock(HTableInterface.class);
-    final boolean[] completed = new boolean[] { false };
-    Mockito.when(table.batch(Mockito.anyList())).thenAnswer(new Answer<Void>() {
-
-      @Override
-      public Void answer(InvocationOnMock invocation) throws Throwable {
-        // just keep track that it was called
-        completed[0] = true;
-        return null;
-      }
-    });
-    Mockito.when(table.getTableName()).thenReturn(test.getTableName());
-    // add the table to the set of tables, so its returned to the writer
-    tables.put(tableName, table);
-
-    // setup the writer and failure policy
-    ParallelWriterIndexCommitter writer = new ParallelWriterIndexCommitter();
-    writer.setup(factory, exec, abort, stop, 1);
-    writer.write(indexUpdates);
-    assertTrue("Writer returned before the table batch completed! Likely a race condition tripped",
-      completed[0]);
-    writer.stop(this.test.getTableNameString() + " finished");
-    assertTrue("Factory didn't get shutdown after writer#stop!", factory.shutdown);
-    assertTrue("ExectorService isn't terminated after writer#stop!", exec.isShutdown());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestParalleWriterIndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestParalleWriterIndexCommitter.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestParalleWriterIndexCommitter.java
deleted file mode 100644
index b8aae4b..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestParalleWriterIndexCommitter.java
+++ /dev/null
@@ -1,117 +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.hadoop.hbase.index.write;
-
-import static org.junit.Assert.assertTrue;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Rule;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.index.StubAbortable;
-import org.apache.hadoop.hbase.index.TableName;
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-
-public class TestParalleWriterIndexCommitter {
-
-  private static final Log LOG = LogFactory.getLog(TestParalleWriterIndexCommitter.class);
-  @Rule
-  public TableName test = new TableName();
-  private final byte[] row = Bytes.toBytes("row");
-
-  @Test
-  public void testCorrectlyCleansUpResources() throws Exception{
-    ExecutorService exec = Executors.newFixedThreadPool(1);
-    FakeTableFactory factory = new FakeTableFactory(
-        Collections.<ImmutableBytesPtr, HTableInterface> emptyMap());
-    ParallelWriterIndexCommitter writer = new ParallelWriterIndexCommitter();
-    Abortable mockAbort = Mockito.mock(Abortable.class);
-    Stoppable mockStop = Mockito.mock(Stoppable.class);
-    // create a simple writer
-    writer.setup(factory, exec, mockAbort, mockStop, 1);
-    // stop the writer
-    writer.stop(this.test.getTableNameString() + " finished");
-    assertTrue("Factory didn't get shutdown after writer#stop!", factory.shutdown);
-    assertTrue("ExectorService isn't terminated after writer#stop!", exec.isShutdown());
-    Mockito.verifyZeroInteractions(mockAbort, mockStop);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testSynchronouslyCompletesAllWrites() throws Exception {
-    LOG.info("Starting " + test.getTableNameString());
-    LOG.info("Current thread is interrupted: " + Thread.interrupted());
-    Abortable abort = new StubAbortable();
-    Stoppable stop = Mockito.mock(Stoppable.class);
-    ExecutorService exec = Executors.newFixedThreadPool(1);
-    Map<ImmutableBytesPtr, HTableInterface> tables =
-        new HashMap<ImmutableBytesPtr, HTableInterface>();
-    FakeTableFactory factory = new FakeTableFactory(tables);
-
-    ImmutableBytesPtr tableName = new ImmutableBytesPtr(this.test.getTableName());
-    Put m = new Put(row);
-    m.add(Bytes.toBytes("family"), Bytes.toBytes("qual"), null);
-    Multimap<HTableInterfaceReference, Mutation> indexUpdates =
-        ArrayListMultimap.<HTableInterfaceReference, Mutation> create();
-    indexUpdates.put(new HTableInterfaceReference(tableName), m);
-
-    HTableInterface table = Mockito.mock(HTableInterface.class);
-    final boolean[] completed = new boolean[] { false };
-    Mockito.when(table.batch(Mockito.anyList())).thenAnswer(new Answer<Void>() {
-
-      @Override
-      public Void answer(InvocationOnMock invocation) throws Throwable {
-        // just keep track that it was called
-        completed[0] = true;
-        return null;
-      }
-    });
-    Mockito.when(table.getTableName()).thenReturn(test.getTableName());
-    // add the table to the set of tables, so its returned to the writer
-    tables.put(tableName, table);
-
-    // setup the writer and failure policy
-    ParallelWriterIndexCommitter writer = new ParallelWriterIndexCommitter();
-    writer.setup(factory, exec, abort, stop, 1);
-    writer.write(indexUpdates);
-    assertTrue("Writer returned before the table batch completed! Likely a race condition tripped",
-      completed[0]);
-    writer.stop(this.test.getTableNameString() + " finished");
-    assertTrue("Factory didn't get shutdown after writer#stop!", factory.shutdown);
-    assertTrue("ExectorService isn't terminated after writer#stop!", exec.isShutdown());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestWALRecoveryCaching.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestWALRecoveryCaching.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestWALRecoveryCaching.java
deleted file mode 100644
index b7dcb1a..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/TestWALRecoveryCaching.java
+++ /dev/null
@@ -1,368 +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.hadoop.hbase.index.write;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
-import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
-import org.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-
-import com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.index.IndexTestingUtils;
-import org.apache.hadoop.hbase.index.Indexer;
-import org.apache.hadoop.hbase.index.TableName;
-import org.apache.hadoop.hbase.index.covered.example.ColumnGroup;
-import org.apache.hadoop.hbase.index.covered.example.CoveredColumn;
-import org.apache.hadoop.hbase.index.covered.example.CoveredColumnIndexSpecifierBuilder;
-import org.apache.hadoop.hbase.index.covered.example.CoveredColumnIndexer;
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
-import org.apache.hadoop.hbase.index.write.recovery.PerRegionIndexWriteCache;
-import org.apache.hadoop.hbase.index.write.recovery.StoreFailuresInCachePolicy;
-
-/**
- * When a regionserver crashes, its WAL is split and then replayed to the server. If the index
- * region was present on the same server, we have to make a best effort to not kill the server for
- * not succeeding on index writes while the index region is coming up.
- */
-public class TestWALRecoveryCaching {
-
-  private static final Log LOG = LogFactory.getLog(TestWALRecoveryCaching.class);
-  private static final long ONE_SEC = 1000;
-  private static final long ONE_MIN = 60 * ONE_SEC;
-  private static final long TIMEOUT = ONE_MIN;
-
-  @Rule
-  public TableName testTable = new TableName();
-
-  private String getIndexTableName() {
-    return this.testTable.getTableNameString() + "_index";
-  }
-
-  // -----------------------------------------------------------------------------------------------
-  // Warning! The classes here rely on this static. Adding multiple tests to this class and running
-  // them concurrently could have unexpected results (including, but not limited to, odd failures
-  // and flapping tests).
-  // -----------------------------------------------------------------------------------------------
-  private static CountDownLatch allowIndexTableToRecover;
-
-  public static class IndexTableBlockingReplayObserver extends BaseRegionObserver {
-
-    @Override
-    public void preWALRestore(ObserverContext<RegionCoprocessorEnvironment> env, HRegionInfo info,
-        HLogKey logKey, WALEdit logEdit) throws IOException {
-      try {
-        LOG.debug("Restoring logs for index table");
-        if (allowIndexTableToRecover != null) {
-          allowIndexTableToRecover.await();
-          LOG.debug("Completed index table recovery wait latch");
-        }
-      } catch (InterruptedException e) {
-        Assert.fail("Should not be interrupted while waiting to allow the index to restore WALs.");
-      }
-    }
-  }
-
-  public static class ReleaseLatchOnFailurePolicy extends StoreFailuresInCachePolicy {
-
-    /**
-     * @param failedIndexEdits
-     */
-    public ReleaseLatchOnFailurePolicy(PerRegionIndexWriteCache failedIndexEdits) {
-      super(failedIndexEdits);
-    }
-
-    @Override
-    public void handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted,
-        Exception cause) throws IOException {
-      LOG.debug("Found index update failure!");
-      if (allowIndexTableToRecover != null) {
-        LOG.info("failed index write on WAL recovery - allowing index table to be restored.");
-        allowIndexTableToRecover.countDown();
-      }
-      super.handleFailure(attempted, cause);
-    }
-
-  }
-
-  //TODO: Jesse to fix
-  @Ignore("Configuration issue - valid test, just needs fixing")
-  @Test
-  public void testWaitsOnIndexRegionToReload() throws Exception {
-    HBaseTestingUtility util = new HBaseTestingUtility();
-    Configuration conf = util.getConfiguration();
-
-    // setup other useful stats
-    IndexTestingUtils.setupConfig(conf);
-    conf.setBoolean(Indexer.CHECK_VERSION_CONF_KEY, false);
-
-    // make sure everything is setup correctly
-    IndexManagementUtil.ensureMutableIndexingCorrectlyConfigured(conf);
-
-    // start the cluster with 2 rs
-    util.startMiniCluster(2);
-
-    HBaseAdmin admin = util.getHBaseAdmin();
-    // setup the index
-    byte[] family = Bytes.toBytes("family");
-    byte[] qual = Bytes.toBytes("qualifier");
-    byte[] nonIndexedFamily = Bytes.toBytes("nonIndexedFamily");
-    String indexedTableName = getIndexTableName();
-    ColumnGroup columns = new ColumnGroup(indexedTableName);
-    columns.add(new CoveredColumn(family, qual));
-    CoveredColumnIndexSpecifierBuilder builder = new CoveredColumnIndexSpecifierBuilder();
-    builder.addIndexGroup(columns);
-
-    // create the primary table w/ indexing enabled
-    HTableDescriptor primaryTable = new HTableDescriptor(testTable.getTableName());
-    primaryTable.addFamily(new HColumnDescriptor(family));
-    primaryTable.addFamily(new HColumnDescriptor(nonIndexedFamily));
-    builder.addArbitraryConfigForTesting(Indexer.RecoveryFailurePolicyKeyForTesting,
-      ReleaseLatchOnFailurePolicy.class.getName());
-    builder.build(primaryTable);
-    admin.createTable(primaryTable);
-
-    // create the index table
-    HTableDescriptor indexTableDesc = new HTableDescriptor(Bytes.toBytes(getIndexTableName()));
-    indexTableDesc.addCoprocessor(IndexTableBlockingReplayObserver.class.getName());
-    CoveredColumnIndexer.createIndexTable(admin, indexTableDesc);
-
-    // figure out where our tables live
-    ServerName shared =
-        ensureTablesLiveOnSameServer(util.getMiniHBaseCluster(), Bytes.toBytes(indexedTableName),
-          testTable.getTableName());
-
-    // load some data into the table
-    Put p = new Put(Bytes.toBytes("row"));
-    p.add(family, qual, Bytes.toBytes("value"));
-    HTable primary = new HTable(conf, testTable.getTableName());
-    primary.put(p);
-    primary.flushCommits();
-
-    // turn on the recovery latch
-    allowIndexTableToRecover = new CountDownLatch(1);
-
-    // kill the server where the tables live - this should trigger distributed log splitting
-    // find the regionserver that matches the passed server
-    List<HRegion> online = new ArrayList<HRegion>();
-    online.addAll(getRegionsFromServerForTable(util.getMiniHBaseCluster(), shared,
-      testTable.getTableName()));
-    online.addAll(getRegionsFromServerForTable(util.getMiniHBaseCluster(), shared,
-      Bytes.toBytes(indexedTableName)));
-
-    // log all the current state of the server
-    LOG.info("Current Server/Region paring: ");
-    for (RegionServerThread t : util.getMiniHBaseCluster().getRegionServerThreads()) {
-      // check all the conditions for the server to be done
-      HRegionServer server = t.getRegionServer();
-      if (server.isStopping() || server.isStopped() || server.isAborted()) {
-        LOG.info("\t== Offline: " + server.getServerName());
-        continue;
-      }
-      List<HRegionInfo> regions = server.getOnlineRegions();
-      LOG.info("\t" + server.getServerName() + " regions: " + regions);
-    }
-
-    LOG.debug("Killing server " + shared);
-    util.getMiniHBaseCluster().killRegionServer(shared);
-    LOG.debug("Waiting on server " + shared + "to die");
-    util.getMiniHBaseCluster().waitForRegionServerToStop(shared, TIMEOUT);
-    // force reassign the regions from the table
-    // LOG.debug("Forcing region reassignment from the killed server: " + shared);
-    // for (HRegion region : online) {
-    // util.getMiniHBaseCluster().getMaster().assign(region.getRegionName());
-    // }
-    System.out.println(" ====== Killed shared server ==== ");
-
-    // make a second put that (1), isn't indexed, so we can be sure of the index state and (2)
-    // ensures that our table is back up
-    Put p2 = new Put(p.getRow());
-    p2.add(nonIndexedFamily, Bytes.toBytes("Not indexed"), Bytes.toBytes("non-indexed value"));
-    primary.put(p2);
-    primary.flushCommits();
-
-    // make sure that we actually failed the write once (within a 5 minute window)
-    assertTrue("Didn't find an error writing to index table within timeout!",
-      allowIndexTableToRecover.await(ONE_MIN * 5, TimeUnit.MILLISECONDS));
-
-    // scan the index to make sure it has the one entry, (that had to be replayed from the WAL,
-    // since we hard killed the server)
-    Scan s = new Scan();
-    HTable index = new HTable(conf, getIndexTableName());
-    ResultScanner scanner = index.getScanner(s);
-    int count = 0;
-    for (Result r : scanner) {
-      LOG.info("Got index table result:" + r);
-      count++;
-    }
-    assertEquals("Got an unexpected found of index rows", 1, count);
-
-    // cleanup
-    scanner.close();
-    index.close();
-    primary.close();
-    util.shutdownMiniCluster();
-  }
-
-  /**
-   * @param miniHBaseCluster
-   * @param server
-   * @param bs
-   * @return
-   */
-  private List<HRegion> getRegionsFromServerForTable(MiniHBaseCluster cluster, ServerName server,
-      byte[] table) {
-    List<HRegion> online = Collections.emptyList();
-    for (RegionServerThread rst : cluster.getRegionServerThreads()) {
-      // if its the server we are going to kill, get the regions we want to reassign
-      if (rst.getRegionServer().getServerName().equals(server)) {
-        online = rst.getRegionServer().getOnlineRegions(table);
-        break;
-      }
-    }
-    return online;
-  }
-
-  /**
-   * @param miniHBaseCluster
-   * @param indexedTableName
-   * @param tableNameString
-   */
-  private ServerName ensureTablesLiveOnSameServer(MiniHBaseCluster cluster, byte[] indexTable,
-      byte[] primaryTable) throws Exception {
-
-    ServerName shared = getSharedServer(cluster, indexTable, primaryTable);
-    boolean tryIndex = true;
-    while (shared == null) {
-
-      // start killing servers until we get an overlap
-      Set<ServerName> servers;
-      byte[] table = null;
-      // switch which server we kill each time to get region movement
-      if (tryIndex) {
-        table = indexTable;
-      } else {
-        table = primaryTable;
-      }
-      servers = getServersForTable(cluster, table);
-      tryIndex = !tryIndex;
-      for (ServerName server : servers) {
-        // find the regionserver that matches the passed server
-        List<HRegion> online = getRegionsFromServerForTable(cluster, server, table);
-
-        LOG.info("Shutting down and reassigning regions from " + server);
-        cluster.stopRegionServer(server);
-        cluster.waitForRegionServerToStop(server, TIMEOUT);
-
-        // force reassign the regions from the table
-        for (HRegion region : online) {
-          cluster.getMaster().assign(region.getRegionName());
-        }
-
-        LOG.info("Starting region server:" + server.getHostname());
-        cluster.startRegionServer(server.getHostname());
-
-        cluster.waitForRegionServerToStart(server.getHostname(), TIMEOUT);
-
-        // start a server to get back to the base number of servers
-        LOG.info("STarting server to replace " + server);
-        cluster.startRegionServer();
-        break;
-      }
-
-      shared = getSharedServer(cluster, indexTable, primaryTable);
-    }
-    return shared;
-  }
-
-  /**
-   * @param cluster
-   * @param indexTable
-   * @param primaryTable
-   * @return
-   * @throws Exception
-   */
-  private ServerName getSharedServer(MiniHBaseCluster cluster, byte[] indexTable,
-      byte[] primaryTable) throws Exception {
-    Set<ServerName> indexServers = getServersForTable(cluster, indexTable);
-    Set<ServerName> primaryServers = getServersForTable(cluster, primaryTable);
-
-    Set<ServerName> joinSet = new HashSet<ServerName>(indexServers);
-    joinSet.addAll(primaryServers);
-    // if there is already an overlap, then find it and return it
-    if (joinSet.size() < indexServers.size() + primaryServers.size()) {
-      // find the first overlapping server
-      for (ServerName server : joinSet) {
-        if (indexServers.contains(server) && primaryServers.contains(server)) {
-          return server;
-        }
-      }
-      throw new RuntimeException(
-          "Couldn't find a matching server on which both the primary and index table live, "
-              + "even though they have overlapping server sets");
-    }
-    return null;
-  }
-
-  private Set<ServerName> getServersForTable(MiniHBaseCluster cluster, byte[] table)
-      throws Exception {
-    List<HRegion> indexRegions = cluster.getRegions(table);
-    Set<ServerName> indexServers = new HashSet<ServerName>();
-    for (HRegion region : indexRegions) {
-      indexServers.add(cluster.getServerHoldingRegion(region.getRegionName()));
-    }
-    return indexServers;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/recovery/TestPerRegionIndexWriteCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/recovery/TestPerRegionIndexWriteCache.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/recovery/TestPerRegionIndexWriteCache.java
deleted file mode 100644
index 2b05053..0000000
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/index/write/recovery/TestPerRegionIndexWriteCache.java
+++ /dev/null
@@ -1,168 +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.hadoop.hbase.index.write.recovery;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Multimap;
-import org.apache.hadoop.hbase.index.table.HTableInterfaceReference;
-import org.apache.hadoop.hbase.index.util.ImmutableBytesPtr;
-import org.apache.hadoop.hbase.index.write.recovery.PerRegionIndexWriteCache;
-
-public class TestPerRegionIndexWriteCache {
-
-  private static final byte[] row = Bytes.toBytes("row");
-  private static final byte[] family = Bytes.toBytes("family");
-  private static final byte[] qual = Bytes.toBytes("qual");
-  private static final byte[] val = Bytes.toBytes("val");
-
-  Put p = new Put(row);
-  Put p2 = new Put(Bytes.toBytes("other row"));
-  {
-    p.add(family, qual, val);
-    p2.add(family, qual, val);
-  }
-
-
-  HRegion r1 = new HRegion() {
-    @Override
-    public int hashCode() {
-      return 1;
-    }
-
-    @Override
-    public String toString() {
-      return "testRegion1";
-    }
-  };
-  HRegion r2 = new HRegion() {
-    @Override
-    public int hashCode() {
-      return 2;
-    }
-
-    @Override
-    public String toString() {
-      return "testRegion1";
-    }
-  };
-
-  @Test
-  public void testAddRemoveSingleRegion() {
-    PerRegionIndexWriteCache cache = new PerRegionIndexWriteCache();
-    HTableInterfaceReference t1 = new HTableInterfaceReference(new ImmutableBytesPtr(Bytes.toBytes("t1")));
-    List<Mutation> mutations = new ArrayList<Mutation>();
-    mutations.add(p);
-    cache.addEdits(r1, t1, mutations);
-    Multimap<HTableInterfaceReference, Mutation> edits = cache.getEdits(r1);
-    Set<Entry<HTableInterfaceReference, Collection<Mutation>>> entries = edits.asMap().entrySet();
-    assertEquals("Got more than one table in the the edit map!", 1, entries.size());
-    for (Entry<HTableInterfaceReference, Collection<Mutation>> entry : entries) {
-     //ensure that we are still storing a list here - otherwise it breaks the parallel writer implementation
-      final List<Mutation> stored = (List<Mutation>) entry.getValue();
-      assertEquals("Got an unexpected amount of mutations in the entry", 1, stored.size());
-      assertEquals("Got an unexpected mutation in the entry", p, stored.get(0));
-    }
-
-    // ensure that a second get doesn't have any more edits. This ensures that we don't keep
-    // references around to these edits and have a memory leak
-    assertNull("Got an entry for a region we removed", cache.getEdits(r1));
-  }
-
-  @Test
-  public void testMultipleAddsForSingleRegion() {
-    PerRegionIndexWriteCache cache = new PerRegionIndexWriteCache();
-    HTableInterfaceReference t1 =
-        new HTableInterfaceReference(new ImmutableBytesPtr(Bytes.toBytes("t1")));
-    List<Mutation> mutations = Lists.<Mutation> newArrayList(p);
-    cache.addEdits(r1, t1, mutations);
-
-    // add a second set
-    mutations = Lists.<Mutation> newArrayList(p2);
-    cache.addEdits(r1, t1, mutations);
-
-    Multimap<HTableInterfaceReference, Mutation> edits = cache.getEdits(r1);
-    Set<Entry<HTableInterfaceReference, Collection<Mutation>>> entries = edits.asMap().entrySet();
-    assertEquals("Got more than one table in the the edit map!", 1, entries.size());
-    for (Entry<HTableInterfaceReference, Collection<Mutation>> entry : entries) {
-      // ensure that we are still storing a list here - otherwise it breaks the parallel writer
-      // implementation
-      final List<Mutation> stored = (List<Mutation>) entry.getValue();
-      assertEquals("Got an unexpected amount of mutations in the entry", 2, stored.size());
-      assertEquals("Got an unexpected mutation in the entry", p, stored.get(0));
-      assertEquals("Got an unexpected mutation in the entry", p2, stored.get(1));
-    }
-  }
-
-  @Test
-  public void testMultipleRegions() {
-    PerRegionIndexWriteCache cache = new PerRegionIndexWriteCache();
-    HTableInterfaceReference t1 =
-        new HTableInterfaceReference(new ImmutableBytesPtr(Bytes.toBytes("t1")));
-    List<Mutation> mutations = Lists.<Mutation> newArrayList(p);
-    List<Mutation> m2 = Lists.<Mutation> newArrayList(p2);
-    // add each region
-    cache.addEdits(r1, t1, mutations);
-    cache.addEdits(r2, t1, m2);
-
-    // check region1
-    Multimap<HTableInterfaceReference, Mutation> edits = cache.getEdits(r1);
-    Set<Entry<HTableInterfaceReference, Collection<Mutation>>> entries = edits.asMap().entrySet();
-    assertEquals("Got more than one table in the the edit map!", 1, entries.size());
-    for (Entry<HTableInterfaceReference, Collection<Mutation>> entry : entries) {
-      // ensure that we are still storing a list here - otherwise it breaks the parallel writer
-      // implementation
-      final List<Mutation> stored = (List<Mutation>) entry.getValue();
-      assertEquals("Got an unexpected amount of mutations in the entry for region1", 1,
-        stored.size());
-      assertEquals("Got an unexpected mutation in the entry for region2", p, stored.get(0));
-    }
-
-    // check region2
-    edits = cache.getEdits(r2);
-    entries = edits.asMap().entrySet();
-    assertEquals("Got more than one table in the the edit map!", 1, entries.size());
-    for (Entry<HTableInterfaceReference, Collection<Mutation>> entry : entries) {
-      // ensure that we are still storing a list here - otherwise it breaks the parallel writer
-      // implementation
-      final List<Mutation> stored = (List<Mutation>) entry.getValue();
-      assertEquals("Got an unexpected amount of mutations in the entry for region2", 1,
-        stored.size());
-      assertEquals("Got an unexpected mutation in the entry for region2", p2, stored.get(0));
-    }
-
-
-    // ensure that a second get doesn't have any more edits. This ensures that we don't keep
-    // references around to these edits and have a memory leak
-    assertNull("Got an entry for a region we removed", cache.getEdits(r1));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadWriteKeyValuesWithCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadWriteKeyValuesWithCodec.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadWriteKeyValuesWithCodec.java
index b8eb338..f091ea7 100644
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadWriteKeyValuesWithCodec.java
+++ b/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadWriteKeyValuesWithCodec.java
@@ -38,8 +38,8 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import org.apache.hadoop.hbase.index.IndexTestingUtils;
-import org.apache.hadoop.hbase.index.wal.IndexedKeyValue;
+import org.apache.phoenix.hbase.index.IndexTestingUtils;
+import org.apache.phoenix.hbase.index.wal.IndexedKeyValue;
 
 /**
  * Simple test to read/write simple files via our custom {@link WALEditCodec} to ensure properly

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndCompressedWAL.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndCompressedWAL.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndCompressedWAL.java
index d13a48a..6bdc52f 100644
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndCompressedWAL.java
+++ b/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndCompressedWAL.java
@@ -53,12 +53,12 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import org.apache.hadoop.hbase.index.IndexTestingUtils;
-import org.apache.hadoop.hbase.index.TableName;
-import org.apache.hadoop.hbase.index.covered.example.ColumnGroup;
-import org.apache.hadoop.hbase.index.covered.example.CoveredColumn;
-import org.apache.hadoop.hbase.index.covered.example.CoveredColumnIndexSpecifierBuilder;
-import org.apache.hadoop.hbase.index.covered.example.CoveredColumnIndexer;
+import org.apache.phoenix.hbase.index.IndexTestingUtils;
+import org.apache.phoenix.hbase.index.TableName;
+import org.apache.phoenix.hbase.index.covered.example.ColumnGroup;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumn;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumnIndexSpecifierBuilder;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumnIndexer;
 
 /**
  * For pre-0.94.9 instances, this class tests correctly deserializing WALEdits w/o compression. Post

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndUncompressedWALInHBase_094_9.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndUncompressedWALInHBase_094_9.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndUncompressedWALInHBase_094_9.java
index a2ee610..42813ed 100644
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndUncompressedWALInHBase_094_9.java
+++ b/phoenix-core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayWithIndexWritesAndUncompressedWALInHBase_094_9.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.regionserver.wal;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 
-import org.apache.hadoop.hbase.index.util.IndexManagementUtil;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
 
 /**
  * Do the WAL Replay test but with the WALEditCodec, rather than an {@link IndexedHLogReader}, but

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/end2end/QueryTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/end2end/QueryTest.java b/phoenix-core/src/test/java/org/apache/phoenix/end2end/QueryTest.java
index 03ccb2c..795638a 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/end2end/QueryTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/end2end/QueryTest.java
@@ -77,7 +77,8 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.primitives.Doubles;
 import com.google.common.primitives.Floats;
-import org.apache.hadoop.hbase.index.write.IndexWriterUtils;
+
+import org.apache.phoenix.hbase.index.write.IndexWriterUtils;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.ConstraintViolationException;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/IndexTestingUtils.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/IndexTestingUtils.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/IndexTestingUtils.java
new file mode 100644
index 0000000..92f45b9
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/IndexTestingUtils.java
@@ -0,0 +1,94 @@
+/*
+ * 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.phoenix.hbase.index;
+
+import static org.junit.Assert.assertEquals;
+
+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.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec;
+import org.apache.hadoop.hbase.regionserver.wal.WALEditCodec;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+
+/**
+ * Utility class for testing indexing
+ */
+public class IndexTestingUtils {
+
+  private static final Log LOG = LogFactory.getLog(IndexTestingUtils.class);
+  private static final String MASTER_INFO_PORT_KEY = "hbase.master.info.port";
+  private static final String RS_INFO_PORT_KEY = "hbase.regionserver.info.port";
+  
+  private IndexTestingUtils() {
+    // private ctor for util class
+  }
+
+  public static void setupConfig(Configuration conf) {
+      conf.setInt(MASTER_INFO_PORT_KEY, -1);
+      conf.setInt(RS_INFO_PORT_KEY, -1);
+    // setup our codec, so we get proper replay/write
+      conf.set(WALEditCodec.WAL_EDIT_CODEC_CLASS_KEY, IndexedWALEditCodec.class.getName());
+  }
+  /**
+   * Verify the state of the index table between the given key and time ranges against the list of
+   * expected keyvalues.
+   * @throws IOException
+   */
+  @SuppressWarnings("javadoc")
+  public static void verifyIndexTableAtTimestamp(HTable index1, List<KeyValue> expected,
+      long start, long end, byte[] startKey, byte[] endKey) throws IOException {
+    LOG.debug("Scanning " + Bytes.toString(index1.getTableName()) + " between times (" + start
+        + ", " + end + "] and keys: [" + Bytes.toString(startKey) + ", " + Bytes.toString(endKey)
+        + "].");
+    Scan s = new Scan(startKey, endKey);
+    // s.setRaw(true);
+    s.setMaxVersions();
+    s.setTimeRange(start, end);
+    List<KeyValue> received = new ArrayList<KeyValue>();
+    ResultScanner scanner = index1.getScanner(s);
+    for (Result r : scanner) {
+      received.addAll(r.list());
+      LOG.debug("Received: " + r.list());
+    }
+    scanner.close();
+    assertEquals("Didn't get the expected kvs from the index table!", expected, received);
+  }
+
+  public static void verifyIndexTableAtTimestamp(HTable index1, List<KeyValue> expected, long ts,
+      byte[] startKey) throws IOException {
+    IndexTestingUtils.verifyIndexTableAtTimestamp(index1, expected, ts, startKey, HConstants.EMPTY_END_ROW);
+  }
+
+  public static void verifyIndexTableAtTimestamp(HTable index1, List<KeyValue> expected, long start,
+      byte[] startKey, byte[] endKey) throws IOException {
+    verifyIndexTableAtTimestamp(index1, expected, start, start + 1, startKey, endKey);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/StubAbortable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/StubAbortable.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/StubAbortable.java
new file mode 100644
index 0000000..e9e025c
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/StubAbortable.java
@@ -0,0 +1,41 @@
+/*
+ * 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.phoenix.hbase.index;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Abortable;
+
+/**
+ * TEst helper to stub out an {@link Abortable} when needed.
+ */
+public class StubAbortable implements Abortable {
+  private static final Log LOG = LogFactory.getLog(StubAbortable.class);
+  private boolean abort;
+
+  @Override
+  public void abort(String reason, Throwable e) {
+    LOG.info("Aborting: " + reason, e);
+    abort = true;
+  }
+
+  @Override
+  public boolean isAborted() {
+    return abort;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/TableName.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/TableName.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/TableName.java
new file mode 100644
index 0000000..835e12c
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/TableName.java
@@ -0,0 +1,45 @@
+/*
+ * 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.phoenix.hbase.index;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.rules.TestWatcher;
+import org.junit.runner.Description;
+
+/**
+ * Returns a {@code byte[]} containing the name of the currently running test method.
+ */
+public class TableName extends TestWatcher {
+  private String tableName;
+
+  /**
+   * Invoked when a test is about to start
+   */
+  @Override
+  protected void starting(Description description) {
+    tableName = description.getMethodName();
+  }
+
+  public byte[] getTableName() {
+    return Bytes.toBytes(tableName);
+  }
+
+  public String getTableNameString() {
+    return this.tableName;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/TestFailForUnsupportedHBaseVersions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/TestFailForUnsupportedHBaseVersions.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/TestFailForUnsupportedHBaseVersions.java
new file mode 100644
index 0000000..43ec826
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/TestFailForUnsupportedHBaseVersions.java
@@ -0,0 +1,156 @@
+/*
+ * 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.phoenix.hbase.index;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.VersionInfo;
+import org.junit.Test;
+
+import org.apache.phoenix.hbase.index.Indexer;
+import org.apache.phoenix.hbase.index.covered.example.ColumnGroup;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumn;
+import org.apache.phoenix.hbase.index.covered.example.CoveredColumnIndexSpecifierBuilder;
+
+/**
+ * Test that we correctly fail for versions of HBase that don't support current properties
+ */
+public class TestFailForUnsupportedHBaseVersions {
+  private static final Log LOG = LogFactory.getLog(TestFailForUnsupportedHBaseVersions.class);
+
+  /**
+   * We don't support WAL Compression for HBase &lt; 0.94.9, so we shouldn't even allow the server
+   * to start if both indexing and WAL Compression are enabled for the wrong versions.
+   */
+  @Test
+  public void testDoesNotSupportCompressedWAL() {
+    Configuration conf = HBaseConfiguration.create();
+    IndexTestingUtils.setupConfig(conf);
+    // get the current version
+    String version = VersionInfo.getVersion();
+    
+    // ensure WAL Compression not enabled
+    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
+    
+    //we support all versions without WAL Compression
+    String supported = Indexer.validateVersion(version, conf);
+    assertNull(
+      "WAL Compression wasn't enabled, but version "+version+" of HBase wasn't supported! All versions should"
+          + " support writing without a compressed WAL. Message: "+supported, supported);
+
+    // enable WAL Compression
+    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
+
+    // set the version to something we know isn't supported
+    version = "0.94.4";
+    supported = Indexer.validateVersion(version, conf);
+    assertNotNull("WAL Compression was enabled, but incorrectly marked version as supported",
+      supported);
+    
+    //make sure the first version of 0.94 that supports Indexing + WAL Compression works
+    version = "0.94.9";
+    supported = Indexer.validateVersion(version, conf);
+    assertNull(
+      "WAL Compression wasn't enabled, but version "+version+" of HBase wasn't supported! Message: "+supported, supported);
+    
+    //make sure we support snapshot builds too
+    version = "0.94.9-SNAPSHOT";
+    supported = Indexer.validateVersion(version, conf);
+    assertNull(
+      "WAL Compression wasn't enabled, but version "+version+" of HBase wasn't supported! Message: "+supported, supported);
+  }
+
+  /**
+   * Test that we correctly abort a RegionServer when we run tests with an unsupported HBase
+   * version. The 'completeness' of this test requires that we run the test with both a version of
+   * HBase that wouldn't be supported with WAL Compression. Currently, this is the default version
+   * (0.94.4) so just running 'mvn test' will run the full test. However, this test will not fail
+   * when running against a version of HBase with WALCompression enabled. Therefore, to fully test
+   * this functionality, we need to run the test against both a supported and an unsupported version
+   * of HBase (as long as we want to support an version of HBase that doesn't support custom WAL
+   * Codecs).
+   * @throws Exception on failure
+   */
+  @Test(timeout = 300000 /* 5 mins */)
+  public void testDoesNotStartRegionServerForUnsupportedCompressionAndVersion() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    IndexTestingUtils.setupConfig(conf);
+    // enable WAL Compression
+    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
+
+    // check the version to see if it isn't supported
+    String version = VersionInfo.getVersion();
+    boolean supported = false;
+    if (Indexer.validateVersion(version, conf) == null) {
+      supported = true;
+    }
+
+    // start the minicluster
+    HBaseTestingUtility util = new HBaseTestingUtility(conf);
+    util.startMiniCluster();
+
+    // setup the primary table
+    HTableDescriptor desc = new HTableDescriptor(
+        "testDoesNotStartRegionServerForUnsupportedCompressionAndVersion");
+    byte[] family = Bytes.toBytes("f");
+    desc.addFamily(new HColumnDescriptor(family));
+
+    // enable indexing to a non-existant index table
+    String indexTableName = "INDEX_TABLE";
+    ColumnGroup fam1 = new ColumnGroup(indexTableName);
+    fam1.add(new CoveredColumn(family, CoveredColumn.ALL_QUALIFIERS));
+    CoveredColumnIndexSpecifierBuilder builder = new CoveredColumnIndexSpecifierBuilder();
+    builder.addIndexGroup(fam1);
+    builder.build(desc);
+
+    // get a reference to the regionserver, so we can ensure it aborts
+    HRegionServer server = util.getMiniHBaseCluster().getRegionServer(0);
+
+    // create the primary table
+    HBaseAdmin admin = util.getHBaseAdmin();
+    if (supported) {
+      admin.createTable(desc);
+      assertFalse("Hosting regeion server failed, even the HBase version (" + version
+          + ") supports WAL Compression.", server.isAborted());
+    } else {
+      admin.createTableAsync(desc, null);
+
+      // wait for the regionserver to abort - if this doesn't occur in the timeout, assume its
+      // broken.
+      while (!server.isAborted()) {
+        LOG.debug("Waiting on regionserver to abort..");
+      }
+    }
+
+    // cleanup
+    util.shutdownMiniCluster();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/CoveredIndexCodecForTesting.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/CoveredIndexCodecForTesting.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/CoveredIndexCodecForTesting.java
new file mode 100644
index 0000000..cb63380
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/CoveredIndexCodecForTesting.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.phoenix.hbase.index.covered;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+import org.apache.phoenix.hbase.index.covered.IndexCodec;
+import org.apache.phoenix.hbase.index.covered.IndexUpdate;
+import org.apache.phoenix.hbase.index.covered.TableState;
+import org.apache.phoenix.index.BaseIndexCodec;
+
+/**
+ * An {@link IndexCodec} for testing that allow you to specify the index updates/deletes, regardless
+ * of the current tables' state.
+ */
+public class CoveredIndexCodecForTesting extends BaseIndexCodec {
+
+  private List<IndexUpdate> deletes = new ArrayList<IndexUpdate>();
+  private List<IndexUpdate> updates = new ArrayList<IndexUpdate>();
+
+  public void addIndexDelete(IndexUpdate... deletes) {
+    this.deletes.addAll(Arrays.asList(deletes));
+  }
+  
+  public void addIndexUpserts(IndexUpdate... updates) {
+    this.updates.addAll(Arrays.asList(updates));
+  }
+
+  public void clear() {
+    this.deletes.clear();
+    this.updates.clear();
+  }
+  
+  @Override
+  public Iterable<IndexUpdate> getIndexDeletes(TableState state) {
+    return this.deletes;
+  }
+
+  @Override
+  public Iterable<IndexUpdate> getIndexUpserts(TableState state) {
+    return this.updates;
+  }
+
+  @Override
+  public void initialize(RegionCoprocessorEnvironment env) throws IOException {
+    // noop
+  }
+
+  @Override
+  public boolean isEnabled(Mutation m) {
+    return true;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/bbacf6e0/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestCoveredColumns.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestCoveredColumns.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestCoveredColumns.java
new file mode 100644
index 0000000..dd5223b
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestCoveredColumns.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.phoenix.hbase.index.covered;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+
+import org.apache.phoenix.hbase.index.covered.CoveredColumns;
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+
+public class TestCoveredColumns {
+
+  private static final byte[] fam = Bytes.toBytes("fam");
+  private static final byte[] qual = Bytes.toBytes("qual");
+
+  @Test
+  public void testCovering() {
+    ColumnReference ref = new ColumnReference(fam, qual);
+    CoveredColumns columns = new CoveredColumns();
+    assertEquals("Should have only found a single column to cover", 1, columns
+        .findNonCoveredColumns(Arrays.asList(ref)).size());
+
+    columns.addColumn(ref);
+    assertEquals("Shouldn't have any columns to cover", 0,
+      columns.findNonCoveredColumns(Arrays.asList(ref)).size());
+  }
+}
\ No newline at end of file