You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2017/11/18 19:44:24 UTC

[1/4] phoenix git commit: PHOENIX-4373 Local index variable length key can have trailing nulls while upserting

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 0a9475457 -> 250439c33


PHOENIX-4373 Local index variable length key can have trailing nulls while upserting


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

Branch: refs/heads/4.x-HBase-1.1
Commit: f431894148c29241bf311c7773183768f13af66a
Parents: 0a94754
Author: Vincent Poon <vi...@apache.org>
Authored: Tue Nov 14 10:47:46 2017 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Nov 18 11:42:12 2017 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/IndexToolIT.java | 40 ++++++++++++++++++++
 .../org/apache/phoenix/compile/ScanRanges.java  | 11 ++++--
 .../apache/phoenix/compile/UpsertCompiler.java  |  3 +-
 3 files changed, 50 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f4318941/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
index 913a147..a9128ab 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
@@ -209,6 +209,46 @@ public class IndexToolIT extends BaseTest {
         }
     }
 
+    @Test
+    public void testSaltedVariableLengthPK() throws Exception {
+        String schemaName = generateUniqueName();
+        String dataTableName = generateUniqueName();
+        String dataTableFullName = SchemaUtil.getTableName(schemaName, dataTableName);
+        String indexTableName = generateUniqueName();
+        try (Connection conn =
+                DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            String dataDDL =
+                    "CREATE TABLE " + dataTableFullName + "(\n"
+                            + "ID VARCHAR NOT NULL PRIMARY KEY,\n"
+                            + "\"info\".CAR_NUM VARCHAR(18) NULL,\n"
+                            + "\"info\".CAP_DATE VARCHAR NULL,\n" + "\"info\".ORG_ID BIGINT NULL,\n"
+                            + "\"info\".ORG_NAME VARCHAR(255) NULL\n" + ") SALT_BUCKETS=3";
+            conn.createStatement().execute(dataDDL);
+
+            String upsert =
+                    "UPSERT INTO " + dataTableFullName
+                            + "(ID,CAR_NUM,CAP_DATE,ORG_ID,ORG_NAME) VALUES('1','car1','2016-01-01 00:00:00',11,'orgname1')";
+            conn.createStatement().execute(upsert);
+            conn.commit();
+
+            String indexDDL =
+                    String.format(
+                        "CREATE %s INDEX %s on %s (\"info\".CAR_NUM,\"info\".CAP_DATE) ASYNC",
+                        (localIndex ? "LOCAL" : ""), indexTableName, dataTableFullName);
+            conn.createStatement().execute(indexDDL);
+
+            runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName);
+
+            ResultSet rs =
+                    conn.createStatement().executeQuery(
+                        "SELECT ORG_ID,CAP_DATE,CAR_NUM,ORG_NAME FROM " + dataTableFullName
+                                + " WHERE CAR_NUM='car1' AND CAP_DATE>='2016-01-01' AND CAP_DATE<='2016-05-02' LIMIT 10");
+            assertTrue(rs.next());
+            int orgId = rs.getInt(1);
+            assertEquals(11, orgId);
+        }
+    }
+
     public static void assertExplainPlan(boolean localIndex, String actualExplainPlan,
             String dataTableFullName, String indexTableFullName) {
         String expectedExplainPlan;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f4318941/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
index 1b94cff..817c1bc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
@@ -207,13 +207,18 @@ public class ScanRanges {
     }
     
     public static byte[] prefixKey(byte[] key, int keyOffset, byte[] prefixKey, int prefixKeyOffset) {
-        if (key.length > 0) {
-            byte[] newKey = new byte[key.length + prefixKeyOffset];
+        return prefixKey(key, keyOffset, key.length, prefixKey, prefixKeyOffset);
+    }
+
+    public static byte[] prefixKey(byte[] key, int keyOffset, int keyLength, byte[] prefixKey,
+            int prefixKeyOffset) {
+        if (keyLength > 0) {
+            byte[] newKey = new byte[keyLength + prefixKeyOffset];
             int totalKeyOffset = keyOffset + prefixKeyOffset;
             if (prefixKey.length >= totalKeyOffset) { // otherwise it's null padded
                 System.arraycopy(prefixKey, 0, newKey, 0, totalKeyOffset);
             }
-            System.arraycopy(key, keyOffset, newKey, totalKeyOffset, key.length - keyOffset);
+            System.arraycopy(key, keyOffset, newKey, totalKeyOffset, keyLength - keyOffset);
             return newKey;
         } 
         return key;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f4318941/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index 6445894..9eaaf62 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -162,7 +162,8 @@ public class UpsertCompiler {
                             .getRegionInfo().getEndKey().length] : region.getRegionInfo()
                             .getStartKey();
             if (regionPrefix.length != 0) {
-                ptr.set(ScanRanges.prefixKey(ptr.get(), 0, regionPrefix, regionPrefix.length));
+                ptr.set(ScanRanges.prefixKey(ptr.get(), 0, ptr.getLength(), regionPrefix,
+                    regionPrefix.length));
             }
         } 
         mutation.put(ptr, new RowMutationState(columnValues, statement.getConnection().getStatementExecutionCounter(), rowTsColInfo, onDupKeyBytes));


[2/4] phoenix git commit: PHOENIX-4381 Calculate the estimatedSize of MutationState incrementally

Posted by ja...@apache.org.
PHOENIX-4381 Calculate the estimatedSize of MutationState incrementally


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

Branch: refs/heads/4.x-HBase-1.1
Commit: b068aebf6c91bbb0a11c92e423cc287620eb0a13
Parents: f431894
Author: Thomas D'Silva <td...@apache.org>
Authored: Wed Nov 15 18:54:04 2017 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Nov 18 11:42:53 2017 -0800

----------------------------------------------------------------------
 .../apache/phoenix/execute/MutationState.java   | 15 ++++-
 .../org/apache/phoenix/util/KeyValueUtil.java   | 65 +++++++++-----------
 2 files changed, 43 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b068aebf/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 9c26575..9fc62c7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -127,6 +127,7 @@ public class MutationState implements SQLCloseable {
 
     private long sizeOffset;
     private int numRows = 0;
+    private long estimatedSize = 0;
     private int[] uncommittedStatementIndexes = EMPTY_STATEMENT_INDEX_ARRAY;
     private boolean isExternalTxContext = false;
     private Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> txMutations = Collections.emptyMap();
@@ -193,6 +194,7 @@ public class MutationState implements SQLCloseable {
             this.mutations.put(table, mutations);
         }
         this.numRows = mutations.size();
+        this.estimatedSize = KeyValueUtil.getEstimatedRowSize(table, mutations);
         throwIfTooBig();
     }
 
@@ -355,7 +357,6 @@ public class MutationState implements SQLCloseable {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.MAX_MUTATION_SIZE_EXCEEDED).build()
                     .buildException();
         }
-        long estimatedSize = KeyValueUtil.getEstimatedRowSize(mutations);
         if (estimatedSize > maxSizeBytes) {
             resetState();
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.MAX_MUTATION_SIZE_BYTES_EXCEEDED)
@@ -434,7 +435,12 @@ public class MutationState implements SQLCloseable {
         phoenixTransactionContext.join(newMutationState.getPhoenixTransactionContext());
 
         this.sizeOffset += newMutationState.sizeOffset;
+        int oldNumRows = this.numRows;
         joinMutationState(newMutationState.mutations, this.mutations);
+        // here we increment the estimated size by the fraction of new rows we added from the newMutationState 
+        if (newMutationState.numRows>0) {
+            this.estimatedSize += ((double)(this.numRows-oldNumRows)/newMutationState.numRows) * newMutationState.estimatedSize;
+        }
         if (!newMutationState.txMutations.isEmpty()) {
             if (txMutations.isEmpty()) {
                 txMutations = Maps.newHashMapWithExpectedSize(mutations.size());
@@ -969,6 +975,8 @@ public class MutationState implements SQLCloseable {
                 long mutationCommitTime = 0;
                 long numFailedMutations = 0;;
                 long startTime = 0;
+                long startNumRows = numRows;
+                long startEstimatedSize = estimatedSize;
                 do {
                     TableRef origTableRef = tableInfo.getOrigTableRef();
                     PTable table = origTableRef.getTable();
@@ -1005,8 +1013,8 @@ public class MutationState implements SQLCloseable {
                         for (List<Mutation> mutationBatch : mutationBatchList) {
                             hTable.batch(mutationBatch);
                             batchCount++;
+                            if (logger.isDebugEnabled()) logger.debug("Sent batch of " + mutationBatch.size() + " for " + Bytes.toString(htableName));
                         }
-                        if (logger.isDebugEnabled()) logger.debug("Sent batch of " + numMutations + " for " + Bytes.toString(htableName));
                         child.stop();
                         child.stop();
                         shouldRetry = false;
@@ -1016,6 +1024,8 @@ public class MutationState implements SQLCloseable {
                         
                         if (tableInfo.isDataTable()) {
                             numRows -= numMutations;
+                            // decrement estimated size by the fraction of rows we sent to hbase
+                            estimatedSize -= ((double)numMutations/startNumRows)*startEstimatedSize;
                         }
                         // Remove batches as we process them
                         mutations.remove(origTableRef);
@@ -1181,6 +1191,7 @@ public class MutationState implements SQLCloseable {
 
     private void resetState() {
         numRows = 0;
+        estimatedSize = 0;
         this.mutations.clear();
         resetTransactionalState();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b068aebf/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
index 4234df5..2dfe1b9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
@@ -192,46 +192,41 @@ public class KeyValueUtil {
      * @return estimated row size
      */
     public static long
-            getEstimatedRowSize(Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> mutations) {
+            getEstimatedRowSize(TableRef tableRef, Map<ImmutableBytesPtr, RowMutationState> mutations) {
         long size = 0;
-        // iterate over tables
-        for (Entry<TableRef, Map<ImmutableBytesPtr, RowMutationState>> tableEntry : mutations
-                .entrySet()) {
-            PTable table = tableEntry.getKey().getTable();
-            // iterate over rows
-            for (Entry<ImmutableBytesPtr, RowMutationState> rowEntry : tableEntry.getValue()
-                    .entrySet()) {
-                int rowLength = rowEntry.getKey().getLength();
-                Map<PColumn, byte[]> colValueMap = rowEntry.getValue().getColumnValues();
-                switch (table.getImmutableStorageScheme()) {
-                case ONE_CELL_PER_COLUMN:
-                    // iterate over columns
-                    for (Entry<PColumn, byte[]> colValueEntry : colValueMap.entrySet()) {
-                        PColumn pColumn = colValueEntry.getKey();
-                        size +=
-                                KeyValue.getKeyValueDataStructureSize(rowLength,
-                                    pColumn.getFamilyName().getBytes().length,
-                                    pColumn.getColumnQualifierBytes().length,
-                                    colValueEntry.getValue().length);
-                    }
-                    break;
-                case SINGLE_CELL_ARRAY_WITH_OFFSETS:
-                    // we store all the column values in a single key value that contains all the
-                    // column values followed by an offset array
+        PTable table = tableRef.getTable();
+        // iterate over rows
+        for (Entry<ImmutableBytesPtr, RowMutationState> rowEntry : mutations.entrySet()) {
+            int rowLength = rowEntry.getKey().getLength();
+            Map<PColumn, byte[]> colValueMap = rowEntry.getValue().getColumnValues();
+            switch (table.getImmutableStorageScheme()) {
+            case ONE_CELL_PER_COLUMN:
+                // iterate over columns
+                for (Entry<PColumn, byte[]> colValueEntry : colValueMap.entrySet()) {
+                    PColumn pColumn = colValueEntry.getKey();
                     size +=
-                            PArrayDataTypeEncoder.getEstimatedByteSize(table, rowLength,
-                                colValueMap);
-                    break;
+                            KeyValue.getKeyValueDataStructureSize(rowLength,
+                                pColumn.getFamilyName().getBytes().length,
+                                pColumn.getColumnQualifierBytes().length,
+                                colValueEntry.getValue().length);
                 }
-                // count the empty key value
-                Pair<byte[], byte[]> emptyKeyValueInfo =
-                        EncodedColumnsUtil.getEmptyKeyValueInfo(table);
+                break;
+            case SINGLE_CELL_ARRAY_WITH_OFFSETS:
+                // we store all the column values in a single key value that contains all the
+                // column values followed by an offset array
                 size +=
-                        KeyValue.getKeyValueDataStructureSize(rowLength,
-                            SchemaUtil.getEmptyColumnFamilyPtr(table).getLength(),
-                            emptyKeyValueInfo.getFirst().length,
-                            emptyKeyValueInfo.getSecond().length);
+                        PArrayDataTypeEncoder.getEstimatedByteSize(table, rowLength,
+                            colValueMap);
+                break;
             }
+            // count the empty key value
+            Pair<byte[], byte[]> emptyKeyValueInfo =
+                    EncodedColumnsUtil.getEmptyKeyValueInfo(table);
+            size +=
+                    KeyValue.getKeyValueDataStructureSize(rowLength,
+                        SchemaUtil.getEmptyColumnFamilyPtr(table).getLength(),
+                        emptyKeyValueInfo.getFirst().length,
+                        emptyKeyValueInfo.getSecond().length);
         }
         return size;
     }


[4/4] phoenix git commit: PHOENIX-4387 DefaultColumnValueIT failing in non-US build environments (Pedro Boado)

Posted by ja...@apache.org.
PHOENIX-4387 DefaultColumnValueIT failing in non-US build environments (Pedro Boado)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 250439c338fe22d68acba95d41695652ddf5a2b2
Parents: ce02d88
Author: James Taylor <jt...@salesforce.com>
Authored: Sat Nov 18 11:36:53 2017 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Nov 18 11:43:53 2017 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/DefaultColumnValueIT.java    | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/250439c3/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
index 62d79bc..59b15d8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
@@ -32,6 +32,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Time;
 import java.sql.Timestamp;
+import java.text.DecimalFormatSymbols;
 
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.util.ByteUtil;
@@ -44,6 +45,8 @@ public class DefaultColumnValueIT extends ParallelStatsDisabledIT {
     private String sharedTable1;
     private String sharedTable2;
 
+    private String DEFAULT_CURRENCY_SYMBOL = DecimalFormatSymbols.getInstance().getCurrencySymbol();
+
     @Before
     public void init() {
         sharedTable1 = generateUniqueName();
@@ -583,7 +586,7 @@ public class DefaultColumnValueIT extends ParallelStatsDisabledIT {
                 "pk INTEGER PRIMARY KEY,"
                 + "c1 INTEGER DEFAULT 1 + 9,"
                 + "c2 DOUBLE DEFAULT SQRT(91506.25),"
-                + "c3 DECIMAL DEFAULT TO_NUMBER('$123.33', '\u00A4###.##'),"
+                + "c3 DECIMAL DEFAULT TO_NUMBER('" + DEFAULT_CURRENCY_SYMBOL + "123.33', '\u00A4###.##'),"
                 + "c4 VARCHAR DEFAULT 'AB' || 'CD',"
                 + "c5 CHAR(5) DEFAULT 'E' || 'F',"
                 + "c6 INTEGER DEFAULT \"MONTH\"(TO_TIMESTAMP('2015-6-05'))"
@@ -599,7 +602,7 @@ public class DefaultColumnValueIT extends ParallelStatsDisabledIT {
                 "pk INTEGER NOT NULL,"
                 + "c1 INTEGER NOT NULL DEFAULT 1 + 9,"
                 + "c2 DOUBLE NOT NULL DEFAULT SQRT(91506.25),"
-                + "c3 DECIMAL NOT NULL DEFAULT TO_NUMBER('$123.33', '\u00A4###.##'),"
+                + "c3 DECIMAL NOT NULL DEFAULT TO_NUMBER('" + DEFAULT_CURRENCY_SYMBOL + "123.33', '\u00A4###.##'),"
                 + "c4 VARCHAR NOT NULL DEFAULT 'AB' || 'CD',"
                 + "c5 CHAR(5) NOT NULL DEFAULT 'E' || 'F',"
                 + "c6 INTEGER NOT NULL DEFAULT \"MONTH\"(TO_TIMESTAMP('2015-6-05')),"
@@ -1045,7 +1048,7 @@ public class DefaultColumnValueIT extends ParallelStatsDisabledIT {
                 "pk INTEGER PRIMARY KEY,"
                 + "c1 INTEGER DEFAULT 1 + 9,"
                 + "c2 DOUBLE DEFAULT SQRT(91506.25),"
-                + "c3 DECIMAL DEFAULT TO_NUMBER('$123.33', '\u00A4###.##'),"
+                + "c3 DECIMAL DEFAULT TO_NUMBER('" + DEFAULT_CURRENCY_SYMBOL + "123.33', '\u00A4###.##'),"
                 + "c4 VARCHAR DEFAULT 'AB' || 'CD',"
                 + "c5 CHAR(5) DEFAULT 'E' || 'F',"
                 + "c6 INTEGER DEFAULT \"MONTH\"(TO_TIMESTAMP('2015-6-05'))"


[3/4] phoenix git commit: PHOENIX-4384 Phoenix server jar doesn't include icu4j jars (Shehzaad Nakhoda)

Posted by ja...@apache.org.
PHOENIX-4384 Phoenix server jar doesn't include icu4j jars (Shehzaad Nakhoda)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: ce02d88ae7adc8dbdebd97caa45b92c33e6d1812
Parents: b068aeb
Author: James Taylor <jt...@salesforce.com>
Authored: Sat Nov 18 11:31:46 2017 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Nov 18 11:43:40 2017 -0800

----------------------------------------------------------------------
 phoenix-core/pom.xml   | 1 +
 phoenix-server/pom.xml | 3 +++
 pom.xml                | 3 ++-
 3 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ce02d88a/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 5bf8c75..30fcab3 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -474,6 +474,7 @@
     <dependency>
       <groupId>com.salesforce.i18n</groupId>
       <artifactId>i18n-util</artifactId>
+      <version>${i18n-util.version}</version>
     </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ce02d88a/phoenix-server/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index acba32b..59db2b7 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -135,6 +135,9 @@
                   <include>org.apache.thrift:libthrift</include>
                   <include>com.clearspring.analytics:stream</include>
                   <include>com.salesforce.i18n:i18n-util</include>
+                  <include>com.ibm.icu:icu4j</include>
+                  <include>com.ibm.icu:icu4j-charset</include>
+                  <include>com.ibm.icu:icu4j-localespi</include>
                 </includes>
                   <excludes>
                     <exclude>org.apache.phoenix:phoenix-server</exclude>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ce02d88a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e89d5f2..1901d86 100644
--- a/pom.xml
+++ b/pom.xml
@@ -105,6 +105,7 @@
     <scala.version>2.11.8</scala.version>
     <scala.binary.version>2.11</scala.binary.version>
     <stream.version>2.9.5</stream.version>
+    <i18n-util.version>1.0.1</i18n-util.version>
     <!-- Test Dependencies -->
     <mockito-all.version>1.8.5</mockito-all.version>
     <junit.version>4.12</junit.version>
@@ -930,7 +931,7 @@
       <dependency>
         <groupId>com.salesforce.i18n</groupId>
         <artifactId>i18n-util</artifactId>
-        <version>1.0.1</version>
+        <version>${i18n-util.version}</version>
       </dependency>
     </dependencies>
   </dependencyManagement>