PHOENIX-4237 Allow sorting on (Java) collation keys for non-English locales 
(Shehzaad Nakhoda)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 81019c644a1085f81ab1f84af46e411660320171
Parents: 5820ff4
Author: James Taylor <jtay...@salesforce.com>
Authored: Fri Nov 3 09:17:29 2017 -0700
Committer: James Taylor <jtay...@salesforce.com>
Committed: Wed Nov 15 10:46:41 2017 -0800

----------------------------------------------------------------------
 LICENSE                                         |  43 ++--
 phoenix-core/pom.xml                            |   5 +
 .../phoenix/end2end/CollationKeyFunctionIT.java | 181 ++++++++++++++
 .../phoenix/expression/ExpressionType.java      |   4 +-
 .../function/CollationKeyFunction.java          | 199 +++++++++++++++
 .../apache/phoenix/jdbc/PhoenixConnection.java  |   3 +
 .../apache/phoenix/util/VarBinaryFormatter.java |  52 ++++
 .../function/CollationKeyFunctionTest.java      | 243 +++++++++++++++++++
 phoenix-server/pom.xml                          |   1 +
 9 files changed, 713 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/81019c64/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
index 08e5e10..7bd8ad1 100644
--- a/LICENSE
+++ b/LICENSE
@@ -236,23 +236,32 @@ Font Awesome fonts (http://fontawesome.io/)
 
 3-Clause BSD License:
 
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in
-all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+1. Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright
+notice, this list of conditions and the following disclaimer in the
+documentation and/or other materials provided with the distribution.
+
+3. Neither the name of the copyright holder nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 
 ---
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/81019c64/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 6f3adb4..f82cddc 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -471,5 +471,10 @@
       <artifactId>stream</artifactId>
       <version>${stream.version}</version>
     </dependency>
+     <dependency>
+      <groupId>com.salesforce.i18n</groupId>
+      <artifactId>i18n-util</artifactId>
+      <version>1.0.1</version>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/81019c64/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
new file mode 100644
index 0000000..efbab64
--- /dev/null
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.end2end;
+
+import static org.apache.phoenix.util.TestUtil.closeStmtAndConn;
+import static org.junit.Assert.assertEquals;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.text.Collator;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * End2End test that tests the COLLATION_KEY in an ORDER BY clause
+ * 
+ */
+public class CollationKeyFunctionIT extends ParallelStatsDisabledIT {
+
+       private String tableName;
+       private String[] dataArray = new String[] {
+                       // (0-6) chinese characters
+                       "\u963f", "\u55c4", "\u963e", "\u554a", "\u4ec8", 
"\u3d9a", "\u9f51",
+                       // (7-13) western characters, some with accent
+                       "a", "b", "ä", "A", "a", "ä", "A" };
+
+       @Before
+       public void initAndPopulateTable() throws Exception {
+               Connection conn = null;
+               PreparedStatement stmt = null;
+               tableName = generateUniqueName();
+               try {
+                       conn = DriverManager.getConnection(getUrl());
+                       String ddl = "CREATE TABLE " + tableName + " (id 
INTEGER PRIMARY KEY, data VARCHAR)";
+                       conn.createStatement().execute(ddl);
+
+                       // insert dataArray into the table, with the index into 
the array as
+                       // the id
+                       for (int i = 0; i < dataArray.length; i++) {
+                               PreparedStatement ps = 
conn.prepareStatement("upsert into " + tableName + " values(?, ?)");
+                               ps.setInt(1, i);
+                               ps.setString(2, dataArray[i]);
+                               ps.executeUpdate();
+                       }
+                       conn.commit();
+               } finally {
+                       closeStmtAndConn(stmt, conn);
+               }
+       }
+
+       @Test
+       public void testZhSort() throws Exception {
+               queryWithCollKeyDefaultArgsWithExpectedOrder("zh", 0, 6, new 
Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+       }
+
+       @Test
+       public void testZhTwSort() throws Exception {
+               queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW", 0, 6, new 
Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+       }
+
+       @Test
+       public void testZhTwStrokeSort() throws Exception {
+               queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW_STROKE", 0, 
6, new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
+       }
+
+       @Test
+       public void testZhStrokeSort() throws Exception {
+               queryWithCollKeyDefaultArgsWithExpectedOrder("zh__STROKE", 0, 
6, new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+       }
+
+       @Test
+       public void testZhPinyinSort() throws Exception {
+               queryWithCollKeyDefaultArgsWithExpectedOrder("zh__PINYIN", 0, 
6, new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+       }
+
+       @Test
+       public void testUpperCaseSort() throws Exception {
+               queryWithCollKeyUpperCaseWithExpectedOrder("en", 7, 13, new 
Integer[] { 7, 10, 11, 13, 9, 12, 8 });
+       }
+
+       @Test
+       public void testPrimaryStrengthSort() throws Exception {
+               queryWithCollKeyWithStrengthWithExpectedOrder("en", 
Collator.PRIMARY, false, 7, 13,
+                               new Integer[] { 7, 9, 10, 11, 12, 13, 8 });
+       }
+       
+       @Test
+       public void testSecondaryStrengthSort() throws Exception {
+               queryWithCollKeyWithStrengthWithExpectedOrder("en", 
Collator.SECONDARY, false, 7, 13,
+                               new Integer[] { 7, 10, 11, 13, 9, 12, 8 });
+       }
+
+       @Test
+       public void testTertiaryStrengthSort() throws Exception {
+               queryWithCollKeyWithStrengthWithExpectedOrder("en", 
Collator.TERTIARY, false, 7, 13,
+                               new Integer[] { 7, 11, 10, 13, 9, 12, 8 });
+       }
+
+       @Test
+       public void testTertiaryStrengthSortDesc() throws Exception {
+               queryWithCollKeyWithStrengthWithExpectedOrder("en", 
Collator.TERTIARY, true, 7, 13,
+                               new Integer[] { 8, 12, 9, 13, 10, 11, 7 });
+       }
+
+       
+       /**
+        * Issue a query ordered by the collation key (with COLLATION_KEY called
+        * with default args) of the data column according to the provided
+        * localeString, and compare the ID and data columns to the expected 
order.
+        * 
+        * @param expectedIndexOrder
+        *            an array of indexes into the dataArray in the order we 
expect.
+        *            This is the same as the ID column
+        * @throws SQLException
+        */
+       private void queryWithCollKeyDefaultArgsWithExpectedOrder(String 
localeString, Integer beginIndex, Integer endIndex,
+                       Integer[] expectedIndexOrder) throws Exception {
+               String query = String.format(
+                               "SELECT id, data FROM %s WHERE ID BETWEEN %d 
AND %d ORDER BY COLLATION_KEY(data, '%s')", tableName,
+                               beginIndex, endIndex, localeString);
+               queryWithExpectedOrder(query, expectedIndexOrder);
+       }
+
+       /**
+        * Same as above, except the upperCase collator argument is set to true
+        */
+       private void queryWithCollKeyUpperCaseWithExpectedOrder(String 
localeString, Integer beginIndex, Integer endIndex,
+                       Integer[] expectedIndexOrder) throws Exception {
+               String query = String.format(
+                               "SELECT id, data FROM %s WHERE ID BETWEEN %d 
AND %d ORDER BY COLLATION_KEY(data, '%s', true), id",
+                               tableName, beginIndex, endIndex, localeString);
+               queryWithExpectedOrder(query, expectedIndexOrder);
+       }
+
+       /**
+        * Same as above, except the collator strength is set
+        */
+       private void queryWithCollKeyWithStrengthWithExpectedOrder(String 
localeString, Integer strength, boolean isDescending,
+                       Integer beginIndex, Integer endIndex, Integer[] 
expectedIndexOrder) throws Exception {
+               String sortOrder = isDescending ? "DESC" : "";
+               
+               String query = String.format(
+                               "SELECT id, data FROM %s WHERE ID BETWEEN %d 
AND %d ORDER BY COLLATION_KEY(data, '%s', false, %d) %s, id %s",
+                               tableName, beginIndex, endIndex, localeString, 
strength, sortOrder, sortOrder);
+               queryWithExpectedOrder(query, expectedIndexOrder);
+       }
+
+       private void queryWithExpectedOrder(String query, Integer[] 
expectedIndexOrder) throws Exception {
+               Connection conn = DriverManager.getConnection(getUrl());
+               PreparedStatement ps = conn.prepareStatement(query);
+               ResultSet rs = ps.executeQuery();
+               int i = 0;
+               while (rs.next()) {
+                       int expectedId = expectedIndexOrder[i];
+                       assertEquals("For row " + i + ": The ID did not match 
the expected index", expectedId, rs.getInt(1));
+                       assertEquals("For row " + i + ": The data did not match 
the expected entry from the data array",
+                                       dataArray[expectedId], rs.getString(2));
+                       i++;
+               }
+               assertEquals("The result set returned a different number of 
rows from the data array", expectedIndexOrder.length, i);
+       }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/81019c64/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index 4f26e87..9a53eb1 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -42,6 +42,7 @@ import 
org.apache.phoenix.expression.function.CeilTimestampExpression;
 import org.apache.phoenix.expression.function.CeilWeekExpression;
 import org.apache.phoenix.expression.function.CeilYearExpression;
 import org.apache.phoenix.expression.function.CoalesceFunction;
+import org.apache.phoenix.expression.function.CollationKeyFunction;
 import org.apache.phoenix.expression.function.ConvertTimezoneFunction;
 import org.apache.phoenix.expression.function.CountAggregateFunction;
 import org.apache.phoenix.expression.function.DayOfMonthFunction;
@@ -294,7 +295,8 @@ public enum ExpressionType {
     ArrayColumnExpression(SingleCellColumnExpression.class),
     FirstValuesFunction(FirstValuesFunction.class),
     LastValuesFunction(LastValuesFunction.class),
-    
DistinctCountHyperLogLogAggregateFunction(DistinctCountHyperLogLogAggregateFunction.class);
+    
DistinctCountHyperLogLogAggregateFunction(DistinctCountHyperLogLogAggregateFunction.class),
+    CollationKeyFunction(CollationKeyFunction.class);
 
     ExpressionType(Class<? extends Expression> clazz) {
         this.clazz = clazz;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/81019c64/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
new file mode 100644
index 0000000..827f70a
--- /dev/null
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.expression.function;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.text.Collator;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.commons.lang.BooleanUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.VarBinaryFormatter;
+
+import com.force.db.i18n.LinguisticSort;
+import com.force.i18n.LocaleUtils;
+
+/**
+ * A Phoenix Function that calculates a collation key for an input string based
+ * on a caller-provided locale and collator strength and decomposition 
settings.
+ * 
+ * The locale should be specified as xx_yy_variant where xx is the ISO 639-1
+ * 2-letter language code, yy is the the ISO 3166 2-letter country code. Both
+ * countryCode and variant are optional. For example, zh_TW_STROKE, zh_TW and 
zh
+ * are all valid locale representations. Note the language code, country code
+ * and variant are used as arguments to the constructor of java.util.Locale.
+ *
+ * This function uses the open-source i18n-util package to obtain the collators
+ * it needs from the provided locale.
+ *
+ * The LinguisticSort implementation in i18n-util encapsulates sort-related
+ * functionality for a substantive list of locales. For each locale, it 
provides
+ * a collator and an Oracle-specific database function that can be used to sort
+ * strings according to the natural language rules of that locale.
+ *
+ * This function uses the collator returned by LinguisticSort.getCollator to
+ * produce a collation key for its input string. A user can expect that the
+ * sorting semantics of this function for a given locale is equivalent to the
+ * sorting behaviour of an Oracle query that is constructed using the Oracle
+ * functions returned by LinguisticSort for that locale.
+ *
+ * The optional third argument to the function is a boolean that specifies
+ * whether to use the upper-case collator (case-insensitive) returned by
+ * LinguisticSort.getUpperCaseCollator.
+ *
+ * The optional fourth and fifth arguments are used to set respectively the
+ * strength and composition of the collator returned by LinguisticSort using 
the
+ * setStrength and setDecomposition methods of java.text.Collator.
+ * 
+ */
+@FunctionParseNode.BuiltInFunction(name = CollationKeyFunction.NAME, args = {
+               // input string
+               @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }),
+               // ISO Code for Locale
+               @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }, 
isConstant = true),
+               // whether to use special upper case collator
+               @FunctionParseNode.Argument(allowedTypes = { PBoolean.class }, 
defaultValue = "false", isConstant = true),
+               // collator strength
+               @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true),
+               // collator decomposition
+               @FunctionParseNode.Argument(allowedTypes = { PInteger.class }, 
defaultValue = "null", isConstant = true) })
+public class CollationKeyFunction extends ScalarFunction {
+
+       private static final Log LOG = 
LogFactory.getLog(CollationKeyFunction.class);
+
+       public static final String NAME = "COLLATION_KEY";
+
+       private Collator collator;
+
+       public CollationKeyFunction() {
+       }
+
+       public CollationKeyFunction(List<Expression> children) throws 
SQLException {
+               super(children);
+               initialize();
+       }
+
+       @Override
+       public void readFields(DataInput input) throws IOException {
+               super.readFields(input);
+               initialize();
+       }
+
+       @Override
+       public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+               Expression expression = getChildren().get(0);
+               if (!expression.evaluate(tuple, ptr)) {
+                       return false;
+               }
+               String inputString = (String) PVarchar.INSTANCE.toObject(ptr, 
expression.getSortOrder());
+               if (LOG.isTraceEnabled()) {
+                       LOG.trace("CollationKey inputString: " + inputString);
+               }
+               byte[] collationKeyByteArray = 
collator.getCollationKey(inputString).toByteArray();
+
+               if (LOG.isTraceEnabled()) {
+                       LOG.trace("CollationKey bytes: " + 
VarBinaryFormatter.INSTANCE.format(collationKeyByteArray));
+               }
+
+               ptr.set(collationKeyByteArray);
+               return true;
+       }
+
+       private void initialize() {
+               String localeISOCode = getLiteralValue(1, String.class);
+               Boolean useSpecialUpperCaseCollator = getLiteralValue(2, 
Boolean.class);
+               Integer collatorStrength = getLiteralValue(3, Integer.class);
+               Integer collatorDecomposition = getLiteralValue(4, 
Integer.class);
+
+               if (LOG.isTraceEnabled()) {
+                       StringBuilder logInputsMessage = new StringBuilder();
+                       logInputsMessage.append("Input (literal) 
arguments:").append("localeISOCode: " + localeISOCode)
+                                       .append(", useSpecialUpperCaseCollator: 
" + useSpecialUpperCaseCollator)
+                                       .append(", collatorStrength: " + 
collatorStrength)
+                                       .append(", collatorDecomposition: " + 
collatorDecomposition);
+                       LOG.trace(logInputsMessage);
+               }
+
+               Locale locale = 
LocaleUtils.get().getLocaleByIsoCode(localeISOCode);
+
+               if (LOG.isTraceEnabled()) {
+                       LOG.trace(String.format("Locale: " + 
locale.toLanguageTag()));
+               }
+
+               LinguisticSort linguisticSort = LinguisticSort.get(locale);
+
+               collator = BooleanUtils.isTrue(useSpecialUpperCaseCollator) ? 
linguisticSort.getUpperCaseCollator(false)
+                               : linguisticSort.getCollator();
+
+               if (collatorStrength != null) {
+                       collator.setStrength(collatorStrength);
+               }
+
+               if (collatorDecomposition != null) {
+                       collator.setDecomposition(collatorDecomposition);
+               }
+
+               if (LOG.isTraceEnabled()) {
+                       LOG.trace(String.format("Collator: [strength: %d, 
decomposition: %d], Special-Upper-Case: %s",
+                                       collator.getStrength(), 
collator.getDecomposition(),
+                                       
BooleanUtils.isTrue(useSpecialUpperCaseCollator)));
+               }
+       }
+
+       @Override
+       public PDataType getDataType() {
+               return PVarbinary.INSTANCE;
+       }
+
+       @Override
+       public String getName() {
+               return NAME;
+       }
+
+       @Override
+       public boolean isThreadSafe() {
+               // ICU4J Collators are not thread-safe unless they are frozen.
+               // TODO: Look into calling freeze() on them to be able return 
true here.
+               return false;
+       }
+
+       private <T> T getLiteralValue(int childIndex, Class<T> type) {
+               Expression expression = getChildren().get(childIndex);
+               if (LOG.isDebugEnabled()) {
+                       LOG.debug("child: " + childIndex + ", expression: " + 
expression);
+               }
+               // It's safe to assume expression is a LiteralExpression since
+               // only arguments marked as isConstant = true should be handled 
through
+               // this method.
+               return type.cast(((LiteralExpression) expression).getValue());
+       }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/81019c64/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java 
b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index 730f754..4555190 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -105,6 +105,7 @@ import org.apache.phoenix.schema.types.PTimestamp;
 import org.apache.phoenix.schema.types.PUnsignedDate;
 import org.apache.phoenix.schema.types.PUnsignedTime;
 import org.apache.phoenix.schema.types.PUnsignedTimestamp;
+import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.trace.util.Tracing;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
 import org.apache.phoenix.util.DateUtil;
@@ -116,6 +117,7 @@ import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SQLCloseable;
 import org.apache.phoenix.util.SQLCloseables;
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.VarBinaryFormatter;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
@@ -336,6 +338,7 @@ public class PhoenixConnection implements Connection, 
MetaDataMutated, SQLClosea
         formatters.put(PUnsignedTimestamp.INSTANCE, timestampFormat);
         formatters.put(PDecimal.INSTANCE,
                 FunctionArgumentType.NUMERIC.getFormatter(numberPattern));
+        formatters.put(PVarbinary.INSTANCE, VarBinaryFormatter.INSTANCE);
         // We do not limit the metaData on a connection less than the global
         // one,
         // as there's not much that will be cached here.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/81019c64/phoenix-core/src/main/java/org/apache/phoenix/util/VarBinaryFormatter.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/util/VarBinaryFormatter.java 
b/phoenix-core/src/main/java/org/apache/phoenix/util/VarBinaryFormatter.java
new file mode 100644
index 0000000..7f0d030
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/VarBinaryFormatter.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.util;
+
+import java.text.FieldPosition;
+import java.text.Format;
+import java.text.ParsePosition;
+
+import org.apache.commons.codec.binary.Hex;
+
+/**
+ * A formatter that formats a byte array to a hexadecimal string
+ * (with each byte converted to a 2-digit hex sequence)
+ *
+ * @author snakhoda-sfdc
+ */
+public class VarBinaryFormatter extends Format {
+
+       private static final long serialVersionUID = -7940880118392024750L;
+
+       public static final VarBinaryFormatter INSTANCE = new 
VarBinaryFormatter();
+
+       @Override
+       public StringBuffer format(Object obj, StringBuffer toAppendTo, 
FieldPosition pos) {
+               if (!(obj instanceof byte[])) {
+                       throw new IllegalArgumentException("VarBinaryFormatter 
can only format byte arrays");
+               }
+               String hexString = Hex.encodeHexString((byte[]) obj);
+               toAppendTo.append(hexString);
+               return toAppendTo;
+       }
+
+       @Override
+       public Object parseObject(String source, ParsePosition pos) {
+               return new UnsupportedOperationException();
+       }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/81019c64/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
new file mode 100644
index 0000000..f57a937
--- /dev/null
+++ 
b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.expression.function;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.fail;
+
+import java.text.Collator;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.lang.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.UnsignedBytes;
+
+/**
+ * "Unit" tests for CollationKeyFunction
+ * 
+ */
+public class CollationKeyFunctionTest {
+
+       private static String[] chineseChars = new String[] { "\u963f", 
"\u55c4", "\u963e", "\u554a", "\u4ec8", "\u3d9a",
+                       "\u9f51" };
+
+       private static Comparator<byte[]> collationKeyComparator = 
UnsignedBytes.lexicographicalComparator();
+
+       private static Comparator<ByteArrayAndInteger> 
collationKeyAndIndexComparator = new Comparator<ByteArrayAndInteger>() {
+               @Override
+               public int compare(ByteArrayAndInteger o1, ByteArrayAndInteger 
o2) {
+                       int compareResult = 
collationKeyComparator.compare(o1.byteArray, o2.byteArray);
+                       if (compareResult == 0) {
+                               compareResult = 
o1.integer.compareTo(o2.integer);
+                       }
+                       return compareResult;
+               }
+       };
+
+       private static class ByteArrayAndInteger {
+
+               private ByteArrayAndInteger(byte[] byteArray, Integer integer) {
+                       super();
+                       this.byteArray = byteArray;
+                       this.integer = integer;
+               }
+
+               byte[] byteArray;
+               Integer integer;
+
+               public String toString() {
+                       return ToStringBuilder.reflectionToString(this);
+               }
+
+               public static ByteArrayAndInteger 
findFirstIntegerMatch(List<ByteArrayAndInteger> list,
+                               Integer matchingInteger) {
+                       for (ByteArrayAndInteger entry : list) {
+                               if (entry.integer.equals(matchingInteger)) {
+                                       return entry;
+                               }
+                       }
+                       return null;
+               }
+       }
+
+       @Test
+       public void testZhSort() throws Exception {
+               testSortOrderNoEquals(chineseChars, "zh", Boolean.FALSE, null, 
null, new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+       }
+
+       @Test
+       public void testZhTwSort() throws Exception {
+               testSortOrderNoEquals(chineseChars, "zh_TW", Boolean.FALSE, 
null, null, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+       }
+
+       @Test
+       public void testZhTwStrokeSort() throws Exception {
+               testSortOrderNoEquals(chineseChars, "zh_TW_STROKE", 
Boolean.FALSE, null, null,
+                               new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
+       }
+
+       @Test
+       public void testZhStrokeSort() throws Exception {
+               testSortOrderNoEquals(chineseChars, "zh__STROKE", 
Boolean.FALSE, null, null,
+                               new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+       }
+
+       @Test
+       public void testZhPinyinSort() throws Exception {
+               testSortOrderNoEquals(chineseChars, "zh__PINYIN", 
Boolean.FALSE, null, null,
+                               new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+       }
+
+       @Test
+       public void testUpperCaseCollationKeyBytes() throws Exception {
+               testCollationKeysEqual(new String[] { "abcdef", "ABCDEF", 
"aBcDeF" }, "en", Boolean.TRUE, null, null);
+       }
+
+       @Test
+       public void testEqualCollationKeysForPrimaryStrength() throws Exception 
{
+               // "a", "A", "ä" are considered equivalent
+               testCollationKeysEqual(new String[] { "a", "A", "ä" }, "en", 
Boolean.FALSE, Collator.PRIMARY, null);
+               testSortOrderNoEquals(new String[] { "b", "a" }, "en", 
Boolean.FALSE, Collator.PRIMARY, null,
+                               new Integer[] { 1, 0 });
+
+       }
+
+       @Test
+       public void testCollationKeyBytesForSecondaryStrength() throws 
Exception {
+               // "a" and "A" are considered equivalent but not "ä"
+               testCollationKeysEqual(new String[] { "a", "A" }, "en", 
Boolean.FALSE, Collator.SECONDARY, null);
+               testSortOrderNoEquals(new String[] { "b", "a", "ä" }, "en", 
Boolean.FALSE, Collator.SECONDARY, null,
+                               new Integer[] { 1, 2, 0 });
+       }
+
+       @Test
+       public void testCollationKeyBytesForTertiaryStrength() throws Exception 
{
+               // none of these are considered equivalent
+               testSortOrderNoEquals(new String[] { "b", "a", "ä", "A" }, 
"en", Boolean.FALSE, Collator.TERTIARY, null,
+                               new Integer[] { 1, 3, 2, 0 });
+       }
+
+       /**
+        * Just test that changing the decomposition mode works for basic 
sorting.
+        * TODO: Actually test for the accented characters and languages where 
this
+        * actually matters.
+        */
+       @Test
+       public void testCollationKeyBytesForFullDecomposition() throws 
Exception {
+               testCollationKeysEqual(new String[] { "a", "A" }, "en", 
Boolean.FALSE, null, Collator.FULL_DECOMPOSITION);
+       }
+
+       /** HELPER METHODS **/
+       private void testSortOrderNoEquals(String[] inputStrings, String 
locale, Boolean uppercaseCollator,
+                       Integer strength, Integer decomposition, Integer[] 
expectedOrder) throws Exception {
+               List<ByteArrayAndInteger> sortedCollationKeysAndIndexes = 
calculateCollationKeys(inputStrings, locale,
+                               uppercaseCollator, strength, decomposition);
+               Collections.sort(sortedCollationKeysAndIndexes, 
collationKeyAndIndexComparator);
+               testCollationKeysNotEqual(inputStrings, 
sortedCollationKeysAndIndexes);
+
+               Integer[] sortedIndexes = new 
Integer[sortedCollationKeysAndIndexes.size()];
+               for (int i = 0; i < sortedIndexes.length; i++) {
+                       sortedIndexes[i] = 
sortedCollationKeysAndIndexes.get(i).integer;
+               }
+               assertArrayEquals(expectedOrder, sortedIndexes);
+       }
+
+       private List<ByteArrayAndInteger> calculateCollationKeys(String[] 
inputStrings, String locale,
+                       Boolean upperCaseCollator, Integer strength, Integer 
decomposition) throws Exception {
+               List<ByteArrayAndInteger> collationKeysAndIndexes = 
Lists.newArrayList();
+               for (int i = 0; i < inputStrings.length; i++) {
+                       byte[] thisCollationKeyBytes = 
callFunction(inputStrings[i], locale, upperCaseCollator, strength,
+                                       decomposition, SortOrder.ASC);
+                       collationKeysAndIndexes.add(new 
ByteArrayAndInteger(thisCollationKeyBytes, i));
+               }
+               return collationKeysAndIndexes;
+       }
+
+       private void testCollationKeysEqual(String[] inputStrings, String 
locale, Boolean upperCaseCollator,
+                       Integer strength, Integer decomposition) throws 
Exception {
+               List<ByteArrayAndInteger> collationKeysAndIndexes = 
calculateCollationKeys(inputStrings, locale,
+                               upperCaseCollator, strength, decomposition);
+
+               for (int i = 0, j = 1; i < inputStrings.length && j < 
inputStrings.length; i++, j++) {
+                       byte[] iByteArray = 
ByteArrayAndInteger.findFirstIntegerMatch(collationKeysAndIndexes, i).byteArray;
+                       byte[] jByteArray = 
ByteArrayAndInteger.findFirstIntegerMatch(collationKeysAndIndexes, j).byteArray;
+                       boolean isPairEqual = 
collationKeyComparator.compare(iByteArray, jByteArray) == 0;
+                       if (!isPairEqual) {
+                               fail(String.format("Collation keys for 
inputStrings [%s] and [%s] ([%s], [%s]) were not equal",
+                                               inputStrings[i], 
inputStrings[j], Hex.encodeHexString(iByteArray),
+                                               
Hex.encodeHexString(jByteArray)));
+                       }
+               }
+       }
+
+       private void testCollationKeysNotEqual(String[] inputStrings, 
List<ByteArrayAndInteger> collationKeysAndIndexes)
+                       throws Exception {
+               for (int i = 0; i < inputStrings.length; i++) {
+                       for (int j = i + 1; j < inputStrings.length; j++) {
+                               byte[] iByteArray = 
ByteArrayAndInteger.findFirstIntegerMatch(collationKeysAndIndexes, i).byteArray;
+                               byte[] jByteArray = 
ByteArrayAndInteger.findFirstIntegerMatch(collationKeysAndIndexes, j).byteArray;
+                               boolean isPairEqual = 
collationKeyComparator.compare(iByteArray, jByteArray) == 0;
+                               if (isPairEqual) {
+                                       fail(String.format("Collation keys for 
inputStrings [%s] and [%s] ([%s], [%s]) were equal",
+                                                       inputStrings[i], 
inputStrings[j], Hex.encodeHexString(iByteArray),
+                                                       
Hex.encodeHexString(jByteArray)));
+                               }
+                       }
+               }
+       }
+
+       private static byte[] callFunction(String inputStr, String 
localeIsoCode, Boolean upperCaseCollator,
+                       Integer strength, Integer decomposition, SortOrder 
sortOrder) throws Exception {
+               LiteralExpression inputStrLiteral, localeIsoCodeLiteral, 
upperCaseBooleanLiteral, strengthLiteral,
+                               decompositionLiteral;
+               inputStrLiteral = LiteralExpression.newConstant(inputStr, 
PVarchar.INSTANCE, sortOrder);
+               localeIsoCodeLiteral = 
LiteralExpression.newConstant(localeIsoCode, PVarchar.INSTANCE, sortOrder);
+               upperCaseBooleanLiteral = 
LiteralExpression.newConstant(upperCaseCollator, PBoolean.INSTANCE, sortOrder);
+               strengthLiteral = LiteralExpression.newConstant(strength, 
PInteger.INSTANCE, sortOrder);
+               decompositionLiteral = 
LiteralExpression.newConstant(decomposition, PInteger.INSTANCE, sortOrder);
+               return callFunction(inputStrLiteral, localeIsoCodeLiteral, 
upperCaseBooleanLiteral, strengthLiteral,
+                               decompositionLiteral);
+
+       }
+
+       private static byte[] callFunction(LiteralExpression inputStrLiteral, 
LiteralExpression localeIsoCodeLiteral,
+                       LiteralExpression upperCaseBooleanLiteral, 
LiteralExpression strengthLiteral,
+                       LiteralExpression decompositionLiteral) throws 
Exception {
+               List<Expression> expressions = Lists.newArrayList((Expression) 
inputStrLiteral,
+                               (Expression) localeIsoCodeLiteral, (Expression) 
upperCaseBooleanLiteral, (Expression) strengthLiteral,
+                               (Expression) decompositionLiteral);
+               Expression collationKeyFunction = new 
CollationKeyFunction(expressions);
+               ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+               boolean ret = collationKeyFunction.evaluate(null, ptr);
+               byte[] result = ret
+                               ? (byte[]) 
collationKeyFunction.getDataType().toObject(ptr, 
collationKeyFunction.getSortOrder()) : null;
+               return result;
+       }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/81019c64/phoenix-server/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index 3576425..67832ad 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -134,6 +134,7 @@
                   <include>io.dropwizard.metrics:metrics-core</include>
                   <include>org.apache.thrift:libthrift</include>
                   <include>com.clearspring.analytics:stream</include>
+                  <include>com.salesforce.i18n:i18n-util</include>
                 </includes>
                   <excludes>
                     <exclude>org.apache.phoenix:phoenix-server</exclude>

Reply via email to