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:47 UTC

[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

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