You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@phoenix.apache.org by JamesRTaylor <gi...@git.apache.org> on 2014/03/05 20:18:00 UTC

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

GitHub user JamesRTaylor opened a pull request:

    https://github.com/apache/incubator-phoenix/pull/14

    Use KeyValueBuilder where possible

    Using ClientKeyValue where possible. Looks like it's not going to work on the server-side, as the memstore expects to be able to get a backing buffer for the KeyValue (see below). Would it be possible to have this code in the memstore go through a KeyValue method to get the backing buffer? Then we could manufacture one for the ClientKeyValue impl.
    
    ```
    java.util.concurrent.ExecutionException: java.lang.RuntimeException: java.lang.UnsupportedOperationException: ClientKeyValue does not support a single backing buffer.
            at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
            at java.util.concurrent.FutureTask.get(FutureTask.java:83)
            at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.processBatchCallback(HConnectionManager.java:1708)
            at org.apache.hadoop.hbase.client.CoprocessorHConnection.processBatchCallback(CoprocessorHConnection.java:39)
            at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.processBatch(HConnectionManager.java:1560)
            at org.apache.hadoop.hbase.client.CoprocessorHConnection.processBatch(CoprocessorHConnection.java:39)
            at org.apache.hadoop.hbase.client.HTable.batch(HTable.java:779)
            at org.apache.hadoop.hbase.coprocessor.CoprocessorHost$Environment$HTableWrapper.batch(CoprocessorHost.java:522)
            at org.apache.phoenix.hbase.index.write.ParallelWriterIndexCommitter$1.call(ParallelWriterIndexCommitter.java:144)
            at org.apache.phoenix.hbase.index.write.ParallelWriterIndexCommitter$1.call(ParallelWriterIndexCommitter.java:1)
            at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
            at java.util.concurrent.FutureTask.run(FutureTask.java:138)
            at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
            at java.lang.Thread.run(Thread.java:695)
    Caused by: java.lang.RuntimeException: java.lang.UnsupportedOperationException: ClientKeyValue does not support a single backing buffer.
            at org.apache.hadoop.hbase.client.ServerCallable.withoutRetries(ServerCallable.java:235)
            at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation$3.call(HConnectionManager.java:1544)
            at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation$3.call(HConnectionManager.java:1532)
            ... 5 more
    Caused by: java.lang.UnsupportedOperationException: ClientKeyValue does not support a single backing buffer.
            at org.apache.phoenix.client.ClientKeyValue.getBuffer(ClientKeyValue.java:500)
            at org.apache.hadoop.hbase.regionserver.MemStore.maybeCloneWithAllocator(MemStore.java:259)
            at org.apache.hadoop.hbase.regionserver.MemStore.add(MemStore.java:207)
            at org.apache.hadoop.hbase.regionserver.Store.add(Store.java:512)
            at org.apache.hadoop.hbase.regionserver.HRegion.applyFamilyMapToMemstore(HRegion.java:2982)
            at org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(HRegion.java:2426)
            at org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2190)
            at org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3873)
            at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation$3$1.call(HConnectionManager.java:1537)
            at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation$3$1.call(HConnectionManager.java:1535)
            at org.apache.hadoop.hbase.client.ServerCallable.withoutRetries(ServerCallable.java:229)
            ... 7 more
    ```

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/JamesRTaylor/incubator-phoenix key-value-builder

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-phoenix/pull/14.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #14
    
----
commit 6431373014dfcb5a59e24192741b02d269d477e0
Author: James Taylor <ja...@apache.org>
Date:   2014-03-05T19:11:19Z

    Use KeyValueBuilder where possible

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#discussion_r10326903
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java ---
    @@ -70,7 +71,9 @@ public void initialize(RegionCoprocessorEnvironment env) {
             // server
             conf.setIfUnset(IndexWriter.INDEX_FAILURE_POLICY_CONF_KEY,
                 PhoenixIndexFailurePolicy.class.getName());
    -        this.builder = KeyValueBuilder.get(env.getHBaseVersion());
    +        // We cannot use the ClientKeyValueBuilder because when these hit the memstore
    --- End diff --
    
    Can you give me a patch for this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#issuecomment-36812798
  
    Didn't understand your last comment. Please elaborate?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#discussion_r10328087
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java ---
    @@ -70,7 +71,9 @@ public void initialize(RegionCoprocessorEnvironment env) {
             // server
             conf.setIfUnset(IndexWriter.INDEX_FAILURE_POLICY_CONF_KEY,
                 PhoenixIndexFailurePolicy.class.getName());
    -        this.builder = KeyValueBuilder.get(env.getHBaseVersion());
    +        // We cannot use the ClientKeyValueBuilder because when these hit the memstore
    --- End diff --
    
    Would this just be a few lines of code? In ParallelWriterIndexCommitter:114 plus having a method on ClientKeyValue that copies the KeyValues and returns the new List<Mutation>?
    
          final List<Mutation> mutations = (List<Mutation>) entry.getValue();
          // Pass the list through to the KeyValueBuilder to potentially make copies of them, as
          // the memstore requires the KeyValue to have a backing buffer
         mutations = kvBuilder.copyMutationsIfNecessary(mutations);



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#discussion_r10316037
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/expression/aggregator/BaseDecimalStddevAggregator.java ---
    @@ -17,7 +17,9 @@
      */
    --- End diff --
    
    Ignore this file - will revert.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#discussion_r10326930
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/util/ColumnInfo.java ---
    @@ -40,7 +40,8 @@ public Integer getSqlType() {
     		return sqlType;
     	}
     	
    -	public String toString() {
    +	@Override
    +    public String toString() {
    --- End diff --
    
    Will fix - and put in a different commit - this shouldn't be here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by jyates <gi...@git.apache.org>.
Github user jyates commented on a diff in the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#discussion_r10326777
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java ---
    @@ -70,7 +71,9 @@ public void initialize(RegionCoprocessorEnvironment env) {
             // server
             conf.setIfUnset(IndexWriter.INDEX_FAILURE_POLICY_CONF_KEY,
                 PhoenixIndexFailurePolicy.class.getName());
    -        this.builder = KeyValueBuilder.get(env.getHBaseVersion());
    +        // We cannot use the ClientKeyValueBuilder because when these hit the memstore
    --- End diff --
    
    So doing this means that we loose a lot of the gains for indexing. If we fix up the kvs on write to the table instead, we can keep this


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by jyates <gi...@git.apache.org>.
Github user jyates commented on a diff in the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#discussion_r10326690
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValueBuilder.java ---
    @@ -1,78 +1,332 @@
     /*
    - * Licensed to the Apache Software Foundation (ASF) under one
    - * or more contributor license agreements.  See the NOTICE file
    - * distributed with this work for additional information
    - * regarding copyright ownership.  The ASF licenses this file
    - * to you under the Apache License, Version 2.0 (the
    - * "License"); you may not use this file except in compliance
    - * with the License.  You may obtain a copy of the License at
    - *
    - * http://www.apache.org/licenses/LICENSE-2.0
    - *
    - * Unless required by applicable law or agreed to in writing, software
    - * distributed under the License is distributed on an "AS IS" BASIS,
    - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    - * See the License for the specific language governing permissions and
    - * limitations under the License.
    + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
    + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
    + * License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by
    + * applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS 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.client;
     
     import org.apache.hadoop.hbase.KeyValue;
    +import org.apache.hadoop.hbase.KeyValue.KVComparator;
     import org.apache.hadoop.hbase.KeyValue.Type;
     import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
     import org.apache.hadoop.hbase.util.Bytes;
     import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
     
    +import com.google.common.primitives.Longs;
    +
     /**
    - * A {@link KeyValueBuilder} that builds {@link ClientKeyValue}, eliminating the extra byte copies
    - * inherent in the standard {@link KeyValue} implementation.
    + * A {@link KeyValueBuilder} that builds {@link ClientKeyValue}, eliminating the extra byte copies inherent in the
    + * standard {@link KeyValue} implementation.
      * <p>
      * This {@link KeyValueBuilder} is only supported in HBase 0.94.14+ (
    - * {@link PhoenixDatabaseMetaData#CLIENT_KEY_VALUE_BUILDER_THRESHOLD}), with the addition of
    - * HBASE-9834.
    + * {@link PhoenixDatabaseMetaData#CLIENT_KEY_VALUE_BUILDER_THRESHOLD}), with the addition of HBASE-9834.
      */
     public class ClientKeyValueBuilder extends KeyValueBuilder {
     
         public static final KeyValueBuilder INSTANCE = new ClientKeyValueBuilder();
     
    -  private ClientKeyValueBuilder() {
    -    // private ctor for singleton
    -  }
    +    private ClientKeyValueBuilder() {
    +        // private ctor for singleton
    +    }
     
    -  @Override
    -  public KeyValue buildPut(ImmutableBytesWritable row, ImmutableBytesWritable family,
    -      ImmutableBytesWritable qualifier, long ts, ImmutableBytesWritable value) {
    -    return new ClientKeyValue(row, family, qualifier, ts, Type.Put, value);
    -  }
    +    @Override
    +    public KeyValue buildPut(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +            ImmutableBytesWritable qualifier, long ts, ImmutableBytesWritable value) {
    +        return new ClientKeyValue(row, family, qualifier, ts, Type.Put, value);
    +    }
     
    -  @Override
    -  public KeyValue buildDeleteFamily(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +    @Override
    +    public KeyValue buildDeleteFamily(ImmutableBytesWritable row, ImmutableBytesWritable family,
                 ImmutableBytesWritable qualifier, long ts) {
             return new ClientKeyValue(row, family, qualifier, ts, Type.DeleteFamily, null);
    -  }
    +    }
     
    -  @Override
    -  public KeyValue buildDeleteColumns(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +    @Override
    +    public KeyValue buildDeleteColumns(ImmutableBytesWritable row, ImmutableBytesWritable family,
                 ImmutableBytesWritable qualifier, long ts) {
             return new ClientKeyValue(row, family, qualifier, ts, Type.DeleteColumn, null);
    -  }
    +    }
     
    -  @Override
    -  public KeyValue buildDeleteColumn(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +    @Override
    +    public KeyValue buildDeleteColumn(ImmutableBytesWritable row, ImmutableBytesWritable family,
                 ImmutableBytesWritable qualifier, long ts) {
             return new ClientKeyValue(row, family, qualifier, ts, Type.Delete, null);
    -  }
    +    }
     
    -  @Override
    -  public int compareQualifier(KeyValue kv, byte[] key, int offset, int length) {
    +    @Override
    +    public int compareQualifier(KeyValue kv, byte[] key, int offset, int length) {
             byte[] qual = kv.getQualifier();
             return Bytes.compareTo(qual, 0, qual.length, key, offset, length);
    -  }
    +    }
     
    -  @Override
    -  public void getValueAsPtr(KeyValue kv, ImmutableBytesWritable ptr) {
    -        ClientKeyValue ckv = (ClientKeyValue) kv;
    +    @Override
    +    public void getValueAsPtr(KeyValue kv, ImmutableBytesWritable ptr) {
    +        ClientKeyValue ckv = (ClientKeyValue)kv;
             ImmutableBytesWritable value = ckv.getRawValue();
             ptr.set(value.get(), value.getOffset(), value.getLength());
    -  }
    +    }
    +
    +    /**
    +     * 
    +     * Singleton ClientKeyValue version of KVComparator that doesn't assume we
    +     * have a backing buffer for the key value
    +     *
    +     */
    +    private static class ClientKeyValueComparator extends KVComparator {
    +        private ClientKeyValueComparator() { // Singleton
    +        }
    +        
    +        public int compareTypes(final KeyValue left, final KeyValue right) {
    +            return left.getType() - right.getType();
    +        }
    +
    +        public int compareMemstoreTimestamps(final KeyValue left, final KeyValue right) {
    +            // Descending order
    +            return Longs.compare(right.getMemstoreTS(), left.getMemstoreTS());
    +        }
    +
    +        @Override
    +        public int compareTimestamps(final KeyValue left, final KeyValue right) {
    +            // Descending order
    +            return Longs.compare(right.getTimestamp(), left.getTimestamp());
    +        }
    +
    +
    +        @Override
    +        public int compare(final KeyValue left, final KeyValue right) {
    +            int c = compareRows(left, right);
    +            if (c != 0) return c;
    +            c = Bytes.compareTo(left.getFamily(), right.getFamily());
    +            if (c != 0) return c;
    +            c = Bytes.compareTo(left.getQualifier(), right.getQualifier());
    +            c = compareTimestamps(left, right);
    +            if (c != 0) return c;
    +            c = compareTypes(left, right);
    +            if (c != 0) return c;            
    +            
    +            return compareMemstoreTimestamps(left, right);
    +        }
    +
    +        /**
    +         * @param left
    +         * @param right
    +         * @return Result comparing rows.
    +         */
    +        @Override
    +        public int compareRows(final KeyValue left, final KeyValue right) {
    +            return Bytes.compareTo(left.getRow(), right.getRow());
    +        }
    +
    +        /**
    +         * @param left
    +         * @param lrowlength Length of left row.
    +         * @param right
    +         * @param rrowlength Length of right row.
    +         * @return Result comparing rows.
    +         */
    +        @Override
    +        public int compareRows(final KeyValue left, final short lrowlength,
    +            final KeyValue right, final short rrowlength) {
    +            return Bytes.compareTo(left.getRow(), 0, lrowlength, right.getRow(), 0, rrowlength);
    +        }
    +
    +        /**
    +         * @param left
    +         * @param row - row key (arbitrary byte array)
    +         * @return RawComparator
    +         */
    +        @Override
    +        public int compareRows(final KeyValue left, final byte [] row) {
    +            return Bytes.compareTo(left.getRow(), row);
    +        }
    +
    +        @Override
    +        public int compareColumns(final KeyValue left, 
    +                final byte [] right, final int roffset, final int rlength, final int rfamilyLength) {
    +          // Compare family portion first.
    +          byte[] lcf = left.getFamily();
    +          int diff = Bytes.compareTo(lcf, 0, lcf.length,
    +            right, roffset, rfamilyLength);
    +          if (diff != 0) {
    +            return diff;
    +          }
    +          // Compare qualifier portion
    +          byte[] lcq = left.getQualifier();
    +          return Bytes.compareTo(lcq, 0, lcq.length,
    +            right, roffset + rfamilyLength, rlength - rfamilyLength);
    +
    +        }
    +
    +        /**
    +         * Compares the row and column of two keyvalues for equality
    +         * @param left
    +         * @param right
    +         * @return True if same row and column.
    +         */
    +        @Override
    +        public boolean matchingRowColumn(final KeyValue left, final KeyValue right) {
    +            if (compareRows(left, right) != 0) {
    +                return false;
    +            }
    +            if (Bytes.compareTo(left.getFamily(), right.getFamily()) != 0) {
    +                return false;
    +            }
    +            if (Bytes.compareTo(left.getQualifier(), right.getQualifier()) != 0) {
    +                return false;
    +            }
    +            return true;
    +        }
    +
    +        /**
    +         * @param left
    +         * @param right
    +         * @return True if rows match.
    +         */
    +        @Override
    +        public boolean matchingRows(final KeyValue left, final byte [] right) {
    +          return Bytes.equals(left.getRow(), right);
    +        }
    +
    +        /**
    +         * Compares the row of two keyvalues for equality
    +         * @param left
    +         * @param right
    +         * @return True if rows match.
    +         */
    +        @Override
    +        public boolean matchingRows(final KeyValue left, final KeyValue right) {
    +            return compareRows(left, right) == 0;
    +        }
    +
    +        /**
    +         * @param left
    +         * @param lrowlength
    +         * @param right
    +         * @param rrowlength
    +         * @return True if rows match.
    +         */
    +        @Override
    +        public boolean matchingRows(final KeyValue left, final short lrowlength,
    +            final KeyValue right, final short rrowlength) {
    +            return compareRows(left, lrowlength, right, rrowlength) == 0;
    +        }
    +
    +        @Override
    +        protected Object clone() throws CloneNotSupportedException {
    +          return this; // Makes no sense to clone this
    +        }
    +
    +        /**
    +         * @return Comparator that ignores timestamps; useful counting versions.
    +         */
    +        @Override
    +        public KVComparator getComparatorIgnoringTimestamps() {
    +            return IGNORE_TIMESTAMP_COMPARATOR;
    +        }
    +
    +        /**
    +         * @return Comparator that ignores key type; useful checking deletes
    +         */
    +        @Override
    +        public KVComparator getComparatorIgnoringType() {
    +            return IGNORE_TYPE_COMPARATOR;
    +        }
    +    }
    +    
    +    /**
    +     * 
    +     * Singleton ClientKeyValue version of KVComparator that ignores differences in the Type
    +     *
    +     */
    +    private static class IgnoreTypeClientKeyValueComparator extends ClientKeyValueComparator {
    +        private IgnoreTypeClientKeyValueComparator() { // Singleton
    +        }
    +        
    +        @Override
    +        public int compareTypes(final KeyValue left, final KeyValue right) {
    +            return 0;
    +        }
    +        
    +        @Override
    +        public KVComparator getComparatorIgnoringTimestamps() {
    +            return IGNORE_TIMESTAMP_AND_TYPE_COMPARATOR;
    +        }
    +    }
    +
    +
    +    /**
    +     * 
    +     * Singleton ClientKeyValue version of KVComparator that ignores differences in the Timestamp
    +     *
    +     */
    +    private static class IgnoreTimestampClientKeyValueComparator extends ClientKeyValueComparator {
    +        private IgnoreTimestampClientKeyValueComparator() { // Singleton
    +        }
    +        
    +        @Override
    +        public int compareMemstoreTimestamps(final KeyValue left, final KeyValue right) {
    +            return 0;
    +        }
    +
    +        @Override
    +        public int compareTimestamps(final KeyValue left, final KeyValue right) {
    +            return 0;
    +        }
    +        
    +        @Override
    +        public KVComparator getComparatorIgnoringType() {
    +            return IGNORE_TYPE_COMPARATOR;
    +        }
    +    }
    +    
    +    /**
    +     * 
    +     * Singleton ClientKeyValue version of KVComparator that ignores differences in the Timestamp and Type
    +     *
    +     */
    +    private static final class IgnoreTimestampAndTypeClientKeyValueComparator extends IgnoreTimestampClientKeyValueComparator {
    +        private IgnoreTimestampAndTypeClientKeyValueComparator() { // Singleton
    +        }
    +        
    +        @Override
    +        public int compareTypes(final KeyValue left, final KeyValue right) {
    +            return 0;
    +        }
    +
    +        @Override
    +        public KVComparator getComparatorIgnoringTimestamps() {
    +            return this;
    +        }
    +
    +        @Override
    +        public KVComparator getComparatorIgnoringType() {
    +            return this;
    +        }
    +    }
    +
    +    private static final KVComparator COMPARATOR = new ClientKeyValueComparator();
    +    private static final KVComparator IGNORE_TYPE_COMPARATOR = new IgnoreTypeClientKeyValueComparator();
    --- End diff --
    
    Are all these necessary? We only  use one internally, but I guess it can't hurt


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#discussion_r10315976
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/client/GenericKeyValueBuilder.java ---
    @@ -27,55 +20,69 @@
     import org.apache.hadoop.hbase.util.Bytes;
     
     /**
    - * {@link KeyValueBuilder} that does simple byte[] copies to build the underlying key-value. This is
    - * exactly the same behavior as currently used in {@link Delete} and {@link Put}.
    + * {@link KeyValueBuilder} that does simple byte[] copies to build the underlying key-value. This is exactly the same
    + * behavior as currently used in {@link Delete} and {@link Put}.
      */
     public class GenericKeyValueBuilder extends KeyValueBuilder {
     
    -  public static final KeyValueBuilder INSTANCE = new GenericKeyValueBuilder();
    +    public static final KeyValueBuilder INSTANCE = new GenericKeyValueBuilder();
     
    -  private GenericKeyValueBuilder() {
    -    // private ctor for singleton
    -  }
    +    private GenericKeyValueBuilder() {
    +        // private ctor for singleton
    +    }
     
    -  @Override
    -  public KeyValue buildPut(ImmutableBytesWritable row, ImmutableBytesWritable family,
    -      ImmutableBytesWritable qualifier, long ts, ImmutableBytesWritable value) {
    -    return build(row, family, qualifier, ts, Type.Put, value);
    -  }
    +    @Override
    +    public KeyValue buildPut(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +            ImmutableBytesWritable qualifier, long ts, ImmutableBytesWritable value) {
    +        return build(row, family, qualifier, ts, Type.Put, value);
    +    }
     
    -  @Override
    -  public KeyValue buildDeleteFamily(ImmutableBytesWritable row, ImmutableBytesWritable family,
    -      ImmutableBytesWritable qualifier, long ts) {
    -    return build(row, family, qualifier, ts, Type.DeleteFamily, null);
    -  }
    +    @Override
    +    public KeyValue buildDeleteFamily(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +            ImmutableBytesWritable qualifier, long ts) {
    +        return build(row, family, qualifier, ts, Type.DeleteFamily, null);
    +    }
     
    -  @Override
    -  public KeyValue buildDeleteColumns(ImmutableBytesWritable row, ImmutableBytesWritable family,
    -      ImmutableBytesWritable qualifier, long ts) {
    -    return build(row, family, qualifier, ts, Type.DeleteColumn, null);
    -  }
    +    @Override
    +    public KeyValue buildDeleteColumns(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +            ImmutableBytesWritable qualifier, long ts) {
    +        return build(row, family, qualifier, ts, Type.DeleteColumn, null);
    +    }
     
    -  @Override
    -  public KeyValue buildDeleteColumn(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +    @Override
    +    public KeyValue buildDeleteColumn(ImmutableBytesWritable row, ImmutableBytesWritable family,
                 ImmutableBytesWritable qualifier, long ts) {
    -    return build(row, family, qualifier, ts, Type.Delete, null);
    -  }
    +        return build(row, family, qualifier, ts, Type.Delete, null);
    +    }
    +
    +    private KeyValue build(ImmutableBytesWritable row, ImmutableBytesWritable family, ImmutableBytesWritable qualifier,
    +            long ts, KeyValue.Type type, ImmutableBytesWritable value) {
    +        return new KeyValue(copyBytesIfNecessary(row), copyBytesIfNecessary(family), copyBytesIfNecessary(qualifier),
    +                ts, type, value == null ? null : copyBytesIfNecessary(value));
    +    }
    +
    +    @Override
    +    public int compareQualifier(KeyValue kv, byte[] key, int offset, int length) {
    +        return Bytes.compareTo(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength(), key, offset, length);
    +    }
    +
    +    @Override
    +    public int compareFamily(KeyValue kv, byte[] key, int offset, int length) {
    +        return Bytes.compareTo(kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(), key, offset, length);
    +    }
     
    -  private KeyValue build(ImmutableBytesWritable row, ImmutableBytesWritable family,
    --- End diff --
    
    Whitespace only changes below here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by jyates <gi...@git.apache.org>.
Github user jyates commented on the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#issuecomment-36813231
  
    > Didn't understand your last comment. Please elaborate?
    
    You mean this one:
    > Some issues around license formatting though :)
    
    Because the licenses are getting compressed together, when there needs to be a lot of whitespace. Orrrrr, you are making fun of me, in which case, yeah, I get it, I'll be better in the future :)
    
    Or this:
    > This shouldn't have the index writing issue (that goes directly to the local HRegion, if its on this server, whose memstore will barf on the ClientKeyValue) as the codec isn't using the clientkeyvaluebuilder.
    
    You'd emailed me about an issue with the index writes still using the ClientKeyValue and that breaking when it did a local write (HRegion present on the same server) since the MemStore needs to read KeyValues. With using the GenericKeyValueBuilder everywhere on the server (including the PhoenixIndexCodec) this shouldn't be a problem anymore, right? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#discussion_r10315962
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/client/GenericKeyValueBuilder.java ---
    @@ -27,55 +20,69 @@
     import org.apache.hadoop.hbase.util.Bytes;
     
     /**
    - * {@link KeyValueBuilder} that does simple byte[] copies to build the underlying key-value. This is
    - * exactly the same behavior as currently used in {@link Delete} and {@link Put}.
    + * {@link KeyValueBuilder} that does simple byte[] copies to build the underlying key-value. This is exactly the same
    + * behavior as currently used in {@link Delete} and {@link Put}.
      */
     public class GenericKeyValueBuilder extends KeyValueBuilder {
     
    -  public static final KeyValueBuilder INSTANCE = new GenericKeyValueBuilder();
    +    public static final KeyValueBuilder INSTANCE = new GenericKeyValueBuilder();
     
    -  private GenericKeyValueBuilder() {
    -    // private ctor for singleton
    -  }
    +    private GenericKeyValueBuilder() {
    +        // private ctor for singleton
    +    }
     
    -  @Override
    -  public KeyValue buildPut(ImmutableBytesWritable row, ImmutableBytesWritable family,
    -      ImmutableBytesWritable qualifier, long ts, ImmutableBytesWritable value) {
    -    return build(row, family, qualifier, ts, Type.Put, value);
    -  }
    +    @Override
    +    public KeyValue buildPut(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +            ImmutableBytesWritable qualifier, long ts, ImmutableBytesWritable value) {
    +        return build(row, family, qualifier, ts, Type.Put, value);
    +    }
     
    -  @Override
    -  public KeyValue buildDeleteFamily(ImmutableBytesWritable row, ImmutableBytesWritable family,
    -      ImmutableBytesWritable qualifier, long ts) {
    -    return build(row, family, qualifier, ts, Type.DeleteFamily, null);
    -  }
    +    @Override
    +    public KeyValue buildDeleteFamily(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +            ImmutableBytesWritable qualifier, long ts) {
    +        return build(row, family, qualifier, ts, Type.DeleteFamily, null);
    +    }
     
    -  @Override
    -  public KeyValue buildDeleteColumns(ImmutableBytesWritable row, ImmutableBytesWritable family,
    -      ImmutableBytesWritable qualifier, long ts) {
    -    return build(row, family, qualifier, ts, Type.DeleteColumn, null);
    -  }
    +    @Override
    +    public KeyValue buildDeleteColumns(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +            ImmutableBytesWritable qualifier, long ts) {
    +        return build(row, family, qualifier, ts, Type.DeleteColumn, null);
    +    }
     
    -  @Override
    -  public KeyValue buildDeleteColumn(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +    @Override
    +    public KeyValue buildDeleteColumn(ImmutableBytesWritable row, ImmutableBytesWritable family,
                 ImmutableBytesWritable qualifier, long ts) {
    -    return build(row, family, qualifier, ts, Type.Delete, null);
    -  }
    +        return build(row, family, qualifier, ts, Type.Delete, null);
    +    }
    +
    +    private KeyValue build(ImmutableBytesWritable row, ImmutableBytesWritable family, ImmutableBytesWritable qualifier,
    +            long ts, KeyValue.Type type, ImmutableBytesWritable value) {
    +        return new KeyValue(copyBytesIfNecessary(row), copyBytesIfNecessary(family), copyBytesIfNecessary(qualifier),
    +                ts, type, value == null ? null : copyBytesIfNecessary(value));
    +    }
    +
    +    @Override
    --- End diff --
    
    These two methods are new


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by jyates <gi...@git.apache.org>.
Github user jyates commented on the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#issuecomment-36812642
  
    Generally looks alright. This shouldn't have the index writing issue (that goes directly to the local HRegion, if its on this server, whose memstore will barf on the ClientKeyValue) as the codec isn't using the clientkeyvaluebuilder.
    
    Some issues around license formatting though  :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#discussion_r10315878
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValueBuilder.java ---
    @@ -1,78 +1,332 @@
     /*
    - * Licensed to the Apache Software Foundation (ASF) under one
    - * or more contributor license agreements.  See the NOTICE file
    - * distributed with this work for additional information
    - * regarding copyright ownership.  The ASF licenses this file
    - * to you under the Apache License, Version 2.0 (the
    - * "License"); you may not use this file except in compliance
    - * with the License.  You may obtain a copy of the License at
    - *
    - * http://www.apache.org/licenses/LICENSE-2.0
    - *
    - * Unless required by applicable law or agreed to in writing, software
    - * distributed under the License is distributed on an "AS IS" BASIS,
    - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    - * See the License for the specific language governing permissions and
    - * limitations under the License.
    + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
    --- End diff --
    
    fixed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#discussion_r10315916
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValueBuilder.java ---
    @@ -1,78 +1,332 @@
     /*
    - * Licensed to the Apache Software Foundation (ASF) under one
    - * or more contributor license agreements.  See the NOTICE file
    - * distributed with this work for additional information
    - * regarding copyright ownership.  The ASF licenses this file
    - * to you under the Apache License, Version 2.0 (the
    - * "License"); you may not use this file except in compliance
    - * with the License.  You may obtain a copy of the License at
    - *
    - * http://www.apache.org/licenses/LICENSE-2.0
    - *
    - * Unless required by applicable law or agreed to in writing, software
    - * distributed under the License is distributed on an "AS IS" BASIS,
    - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    - * See the License for the specific language governing permissions and
    - * limitations under the License.
    + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
    + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
    + * License. You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by
    + * applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS 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.client;
     
     import org.apache.hadoop.hbase.KeyValue;
    +import org.apache.hadoop.hbase.KeyValue.KVComparator;
     import org.apache.hadoop.hbase.KeyValue.Type;
     import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
     import org.apache.hadoop.hbase.util.Bytes;
     import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
     
    +import com.google.common.primitives.Longs;
    +
     /**
    - * A {@link KeyValueBuilder} that builds {@link ClientKeyValue}, eliminating the extra byte copies
    - * inherent in the standard {@link KeyValue} implementation.
    + * A {@link KeyValueBuilder} that builds {@link ClientKeyValue}, eliminating the extra byte copies inherent in the
    + * standard {@link KeyValue} implementation.
      * <p>
      * This {@link KeyValueBuilder} is only supported in HBase 0.94.14+ (
    - * {@link PhoenixDatabaseMetaData#CLIENT_KEY_VALUE_BUILDER_THRESHOLD}), with the addition of
    - * HBASE-9834.
    + * {@link PhoenixDatabaseMetaData#CLIENT_KEY_VALUE_BUILDER_THRESHOLD}), with the addition of HBASE-9834.
      */
     public class ClientKeyValueBuilder extends KeyValueBuilder {
     
         public static final KeyValueBuilder INSTANCE = new ClientKeyValueBuilder();
     
    -  private ClientKeyValueBuilder() {
    -    // private ctor for singleton
    -  }
    +    private ClientKeyValueBuilder() {
    +        // private ctor for singleton
    +    }
     
    -  @Override
    -  public KeyValue buildPut(ImmutableBytesWritable row, ImmutableBytesWritable family,
    -      ImmutableBytesWritable qualifier, long ts, ImmutableBytesWritable value) {
    -    return new ClientKeyValue(row, family, qualifier, ts, Type.Put, value);
    -  }
    +    @Override
    +    public KeyValue buildPut(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +            ImmutableBytesWritable qualifier, long ts, ImmutableBytesWritable value) {
    +        return new ClientKeyValue(row, family, qualifier, ts, Type.Put, value);
    +    }
     
    -  @Override
    -  public KeyValue buildDeleteFamily(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +    @Override
    +    public KeyValue buildDeleteFamily(ImmutableBytesWritable row, ImmutableBytesWritable family,
                 ImmutableBytesWritable qualifier, long ts) {
             return new ClientKeyValue(row, family, qualifier, ts, Type.DeleteFamily, null);
    -  }
    +    }
     
    -  @Override
    -  public KeyValue buildDeleteColumns(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +    @Override
    +    public KeyValue buildDeleteColumns(ImmutableBytesWritable row, ImmutableBytesWritable family,
                 ImmutableBytesWritable qualifier, long ts) {
             return new ClientKeyValue(row, family, qualifier, ts, Type.DeleteColumn, null);
    -  }
    +    }
     
    -  @Override
    -  public KeyValue buildDeleteColumn(ImmutableBytesWritable row, ImmutableBytesWritable family,
    +    @Override
    +    public KeyValue buildDeleteColumn(ImmutableBytesWritable row, ImmutableBytesWritable family,
                 ImmutableBytesWritable qualifier, long ts) {
             return new ClientKeyValue(row, family, qualifier, ts, Type.Delete, null);
    -  }
    +    }
     
    -  @Override
    -  public int compareQualifier(KeyValue kv, byte[] key, int offset, int length) {
    +    @Override
    +    public int compareQualifier(KeyValue kv, byte[] key, int offset, int length) {
             byte[] qual = kv.getQualifier();
             return Bytes.compareTo(qual, 0, qual.length, key, offset, length);
    -  }
    +    }
     
    -  @Override
    -  public void getValueAsPtr(KeyValue kv, ImmutableBytesWritable ptr) {
    -        ClientKeyValue ckv = (ClientKeyValue) kv;
    +    @Override
    +    public void getValueAsPtr(KeyValue kv, ImmutableBytesWritable ptr) {
    +        ClientKeyValue ckv = (ClientKeyValue)kv;
             ImmutableBytesWritable value = ckv.getRawValue();
             ptr.set(value.get(), value.getOffset(), value.getLength());
    -  }
    +    }
    --- End diff --
    
    Above here, are whitespace only changes. Please ignore.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by jyates <gi...@git.apache.org>.
Github user jyates commented on a diff in the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#discussion_r10313963
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/client/ClientKeyValueBuilder.java ---
    @@ -1,78 +1,332 @@
     /*
    - * Licensed to the Apache Software Foundation (ASF) under one
    - * or more contributor license agreements.  See the NOTICE file
    - * distributed with this work for additional information
    - * regarding copyright ownership.  The ASF licenses this file
    - * to you under the Apache License, Version 2.0 (the
    - * "License"); you may not use this file except in compliance
    - * with the License.  You may obtain a copy of the License at
    - *
    - * http://www.apache.org/licenses/LICENSE-2.0
    - *
    - * Unless required by applicable law or agreed to in writing, software
    - * distributed under the License is distributed on an "AS IS" BASIS,
    - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    - * See the License for the specific language governing permissions and
    - * limitations under the License.
    + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
    --- End diff --
    
    nit: mangled the header


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by jyates <gi...@git.apache.org>.
Github user jyates commented on a diff in the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#discussion_r10326792
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/util/ColumnInfo.java ---
    @@ -40,7 +40,8 @@ public Integer getSqlType() {
     		return sqlType;
     	}
     	
    -	public String toString() {
    +	@Override
    +    public String toString() {
    --- End diff --
    
    that spacing doesn't look right


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#issuecomment-36813908
  
    Yes, I'm using GenericKeyValueBuilder on server because it breaks otherwise. No issues anymore, but pretty limited value as it only helps on the client side now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by jyates <gi...@git.apache.org>.
Github user jyates commented on a diff in the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#discussion_r10327006
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java ---
    @@ -70,7 +71,9 @@ public void initialize(RegionCoprocessorEnvironment env) {
             // server
             conf.setIfUnset(IndexWriter.INDEX_FAILURE_POLICY_CONF_KEY,
                 PhoenixIndexFailurePolicy.class.getName());
    -        this.builder = KeyValueBuilder.get(env.getHBaseVersion());
    +        // We cannot use the ClientKeyValueBuilder because when these hit the memstore
    --- End diff --
    
    Not at the moment...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by jyates <gi...@git.apache.org>.
Github user jyates commented on the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#issuecomment-36813973
  
    We can fix it later. Lets get it used in basic cases and go from there (when I, or someone else, has time)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-phoenix pull request: Use KeyValueBuilder where possible

Posted by JamesRTaylor <gi...@git.apache.org>.
Github user JamesRTaylor commented on a diff in the pull request:

    https://github.com/apache/incubator-phoenix/pull/14#discussion_r10327592
  
    --- Diff: phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java ---
    @@ -70,7 +71,9 @@ public void initialize(RegionCoprocessorEnvironment env) {
             // server
             conf.setIfUnset(IndexWriter.INDEX_FAILURE_POLICY_CONF_KEY,
                 PhoenixIndexFailurePolicy.class.getName());
    -        this.builder = KeyValueBuilder.get(env.getHBaseVersion());
    +        // We cannot use the ClientKeyValueBuilder because when these hit the memstore
    --- End diff --
    
    How about the start of a patch, as I'm not sure where the change in your code is required?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---