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 2014/01/27 20:55:47 UTC

git commit: Adding UPPER, LOWER, and MD5 built-in functions

Updated Branches:
  refs/heads/master c5b802461 -> 64da9e222


Adding UPPER, LOWER, and MD5 built-in functions


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

Branch: refs/heads/master
Commit: 64da9e2225d9deefc2156180f4d1b7814582e150
Parents: c5b8024
Author: James Taylor <ja...@apache.org>
Authored: Mon Jan 27 11:55:36 2014 -0800
Committer: James Taylor <ja...@apache.org>
Committed: Mon Jan 27 11:55:36 2014 -0800

----------------------------------------------------------------------
 .../phoenix/expression/ExpressionType.java      |   9 +-
 .../expression/function/LowerFunction.java      |  60 +++++++++++
 .../expression/function/MD5Function.java        | 100 +++++++++++++++++++
 .../expression/function/UpperFunction.java      |  58 +++++++++++
 .../apache/phoenix/end2end/MD5FunctionTest.java |  85 ++++++++++++++++
 .../phoenix/end2end/VariableLengthPKTest.java   |  75 +++++++++++++-
 .../DescColumnSortOrderExpressionTest.java      |  36 ++++---
 7 files changed, 401 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/64da9e22/src/main/java/org/apache/phoenix/expression/ExpressionType.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/expression/ExpressionType.java b/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index c865842..40439d7 100644
--- a/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ b/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -21,6 +21,7 @@ package org.apache.phoenix.expression;
 
 import java.util.Map;
 
+import com.google.common.collect.Maps;
 import org.apache.phoenix.expression.function.CoalesceFunction;
 import org.apache.phoenix.expression.function.CountAggregateFunction;
 import org.apache.phoenix.expression.function.DistinctCountAggregateFunction;
@@ -28,6 +29,8 @@ import org.apache.phoenix.expression.function.IndexStateNameFunction;
 import org.apache.phoenix.expression.function.InvertFunction;
 import org.apache.phoenix.expression.function.LTrimFunction;
 import org.apache.phoenix.expression.function.LengthFunction;
+import org.apache.phoenix.expression.function.LowerFunction;
+import org.apache.phoenix.expression.function.MD5Function;
 import org.apache.phoenix.expression.function.MaxAggregateFunction;
 import org.apache.phoenix.expression.function.MinAggregateFunction;
 import org.apache.phoenix.expression.function.PercentRankAggregateFunction;
@@ -49,8 +52,7 @@ import org.apache.phoenix.expression.function.ToDateFunction;
 import org.apache.phoenix.expression.function.ToNumberFunction;
 import org.apache.phoenix.expression.function.TrimFunction;
 import org.apache.phoenix.expression.function.TruncFunction;
-
-import com.google.common.collect.Maps;
+import org.apache.phoenix.expression.function.UpperFunction;
 
 /**
  * 
@@ -103,6 +105,8 @@ public enum ExpressionType {
     LengthFunction(LengthFunction.class),
     LTrimFunction(LTrimFunction.class),
     RTrimFunction(RTrimFunction.class),
+    UpperFunction(UpperFunction.class),
+    LowerFunction(LowerFunction.class),
     TrimFunction(TrimFunction.class),
     DistinctCountAggregateFunction(DistinctCountAggregateFunction.class),
     PercentileContAggregateFunction(PercentileContAggregateFunction.class),
@@ -115,6 +119,7 @@ public enum ExpressionType {
     DoubleMultiplyExpression(DoubleMultiplyExpression.class),
     DoubleDivideExpression(DoubleDivideExpression.class),
     RowValueConstructorExpression(RowValueConstructorExpression.class),
+    MD5Function(MD5Function.class),
     SqlTableType(SqlTableType.class),
     CeilingDecimalExpression(CeilingDecimalExpression.class),
     CeilingTimestampExpression(CeilingTimestampExpression.class),

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/64da9e22/src/main/java/org/apache/phoenix/expression/function/LowerFunction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/expression/function/LowerFunction.java b/src/main/java/org/apache/phoenix/expression/function/LowerFunction.java
new file mode 100644
index 0000000..aaa21a3
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/expression/function/LowerFunction.java
@@ -0,0 +1,60 @@
+package org.apache.phoenix.expression.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+@FunctionParseNode.BuiltInFunction(name=LowerFunction.NAME,  args={
+        @FunctionParseNode.Argument(allowedTypes={PDataType.VARCHAR})} )
+public class LowerFunction extends ScalarFunction {
+    public static final String NAME = "LOWER";
+
+    public LowerFunction() {
+    }
+
+    public LowerFunction(List<Expression> children) throws SQLException {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (!getStrExpression().evaluate(tuple, ptr)) {
+            return false;
+        }
+
+        String sourceStr = (String)PDataType.VARCHAR.toObject(ptr, getStrExpression().getColumnModifier());
+
+        if (sourceStr == null) {
+            return true;
+        }
+
+        ptr.set(PDataType.VARCHAR.toBytes(sourceStr.toLowerCase()));
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return getStrExpression().getDataType();
+    }
+
+    @Override
+    public boolean isNullable() {
+        return getStrExpression().isNullable();
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    private Expression getStrExpression() {
+        return children.get(0);
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/64da9e22/src/main/java/org/apache/phoenix/expression/function/MD5Function.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/expression/function/MD5Function.java b/src/main/java/org/apache/phoenix/expression/function/MD5Function.java
new file mode 100644
index 0000000..32ef642
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/expression/function/MD5Function.java
@@ -0,0 +1,100 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode.Argument;
+import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+@BuiltInFunction(name = MD5Function.NAME,  args={@Argument()})
+public class MD5Function extends ScalarFunction {
+  public static final String NAME = "MD5";
+  public static final Integer LENGTH = 16;
+
+  private final MessageDigest messageDigest;
+
+  public MD5Function() throws SQLException {
+      try {
+          messageDigest = MessageDigest.getInstance("MD5");
+        } catch (NoSuchAlgorithmException e) {
+          throw new SQLException(e);
+        }      
+  }
+  
+  public MD5Function(List<Expression> children) throws SQLException {
+    super(children);
+    try {
+      messageDigest = MessageDigest.getInstance("MD5");
+    } catch (NoSuchAlgorithmException e) {
+      throw new RuntimeException(e);
+    }
+  }
+  
+  @Override
+  public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+    if (!getChildExpression().evaluate(tuple, ptr)) {
+      return false;
+    }
+
+    // Update the digest value
+    messageDigest.update(ptr.get(), ptr.getOffset(), ptr.getLength());
+    // Get the digest bytes (note this resets the messageDigest as well)
+    ptr.set(messageDigest.digest());
+    return true;
+  }
+
+  @Override
+  public PDataType getDataType() {
+    return PDataType.BINARY;
+  }
+
+  @Override
+  public Integer getMaxLength() {
+    return LENGTH;
+  }
+
+  @Override
+  public Integer getByteSize() {
+    return LENGTH;
+  }
+
+  @Override
+  public boolean isNullable() {
+    return getChildExpression().isNullable();
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  private Expression getChildExpression() {
+    return children.get(0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/64da9e22/src/main/java/org/apache/phoenix/expression/function/UpperFunction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/phoenix/expression/function/UpperFunction.java b/src/main/java/org/apache/phoenix/expression/function/UpperFunction.java
new file mode 100644
index 0000000..af4c39c
--- /dev/null
+++ b/src/main/java/org/apache/phoenix/expression/function/UpperFunction.java
@@ -0,0 +1,58 @@
+package org.apache.phoenix.expression.function;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+@FunctionParseNode.BuiltInFunction(name=UpperFunction.NAME,  args={
+        @FunctionParseNode.Argument(allowedTypes={PDataType.VARCHAR})} )
+public class UpperFunction extends ScalarFunction {
+    public static final String NAME = "UPPER";
+
+    public UpperFunction() {
+    }
+
+    public UpperFunction(List<Expression> children) throws SQLException {
+        super(children);
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (!getStrExpression().evaluate(tuple, ptr)) {
+            return false;
+        }
+
+        String sourceStr = (String)PDataType.VARCHAR.toObject(ptr, getStrExpression().getColumnModifier());
+        if (sourceStr == null) {
+            return true;
+        }
+
+        ptr.set(PDataType.VARCHAR.toBytes(sourceStr.toUpperCase()));
+        return true;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return getStrExpression().getDataType();
+    }
+
+    @Override
+    public boolean isNullable() {
+        return getStrExpression().isNullable();
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    private Expression getStrExpression() {
+        return children.get(0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/64da9e22/src/test/java/org/apache/phoenix/end2end/MD5FunctionTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/phoenix/end2end/MD5FunctionTest.java b/src/test/java/org/apache/phoenix/end2end/MD5FunctionTest.java
new file mode 100644
index 0000000..c4128d0
--- /dev/null
+++ b/src/test/java/org/apache/phoenix/end2end/MD5FunctionTest.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.security.MessageDigest;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+
+import org.junit.Test;
+
+public class MD5FunctionTest extends BaseHBaseManagedTimeTest {
+  
+  @Test
+  public void testRetrieve() throws Exception {
+      String testString = "mwalsh";
+      
+      Connection conn = DriverManager.getConnection(getUrl());
+      String ddl = "CREATE TABLE IF NOT EXISTS MD5_RETRIEVE_TEST (pk VARCHAR NOT NULL PRIMARY KEY)";
+      conn.createStatement().execute(ddl);
+      String dml = String.format("UPSERT INTO MD5_RETRIEVE_TEST VALUES('%s')", testString);
+      conn.createStatement().execute(dml);
+      conn.commit();
+      
+      ResultSet rs = conn.createStatement().executeQuery("SELECT MD5(pk) FROM MD5_RETRIEVE_TEST");
+      assertTrue(rs.next());
+      byte[] first = MessageDigest.getInstance("MD5").digest(testString.getBytes());
+      byte[] second = rs.getBytes(1);
+      assertArrayEquals(first, second);
+      assertFalse(rs.next());
+  }      
+  
+  @Test
+  public void testUpsert() throws Exception {
+      String testString1 = "mwalsh1";
+      String testString2 = "mwalsh2";
+      
+      Connection conn = DriverManager.getConnection(getUrl());
+      String ddl = "CREATE TABLE IF NOT EXISTS MD5_UPSERT_TEST (k1 binary(16) NOT NULL,k2 binary(16) NOT NULL  CONSTRAINT pk PRIMARY KEY (k1, k2))";
+      conn.createStatement().execute(ddl);
+      String dml = String.format("UPSERT INTO MD5_UPSERT_TEST VALUES(md5('%s'),md5('%s'))", testString1, testString2);
+      conn.createStatement().execute(dml);
+      conn.commit();
+      
+      ResultSet rs = conn.createStatement().executeQuery("SELECT k1,k2 FROM MD5_UPSERT_TEST");
+      assertTrue(rs.next());
+      byte[] pk1 = MessageDigest.getInstance("MD5").digest(testString1.getBytes());
+      byte[] pk2 = MessageDigest.getInstance("MD5").digest(testString2.getBytes());
+      assertArrayEquals(pk1, rs.getBytes(1));
+      assertArrayEquals(pk2, rs.getBytes(2));
+      assertFalse(rs.next());
+      PreparedStatement stmt = conn.prepareStatement("SELECT k1,k2 FROM MD5_UPSERT_TEST WHERE k1=md5(?)");
+      stmt.setString(1, testString1);
+      rs = stmt.executeQuery();
+      assertTrue(rs.next());
+      byte[] second1 = rs.getBytes(1);
+      byte[] second2 = rs.getBytes(2);
+      assertArrayEquals(pk1, second1);
+      assertArrayEquals(pk2, second2);
+      assertFalse(rs.next());
+  }                                                           
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/64da9e22/src/test/java/org/apache/phoenix/end2end/VariableLengthPKTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/phoenix/end2end/VariableLengthPKTest.java b/src/test/java/org/apache/phoenix/end2end/VariableLengthPKTest.java
index 552f1a7..9fae189 100644
--- a/src/test/java/org/apache/phoenix/end2end/VariableLengthPKTest.java
+++ b/src/test/java/org/apache/phoenix/end2end/VariableLengthPKTest.java
@@ -43,10 +43,11 @@ import java.text.Format;
 import java.text.ParseException;
 import java.util.Properties;
 
+import org.junit.Test;
+
 import org.apache.phoenix.schema.ConstraintViolationException;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
-import org.junit.Test;
 
 
 public class VariableLengthPKTest extends BaseClientMangedTimeTest {
@@ -1661,6 +1662,78 @@ public class VariableLengthPKTest extends BaseClientMangedTimeTest {
     }
 
     @Test
+    public void testUpperFunction() throws Exception {
+        long ts = nextTimestamp();
+        String query[] = {
+                "SELECT upper('abc') FROM BTABLE LIMIT 1",
+                "SELECT upper('Abc') FROM BTABLE LIMIT 1",
+                "SELECT upper('ABC') FROM BTABLE LIMIT 1",
+                "SELECT upper('ĎďĒ') FROM BTABLE LIMIT 1",
+                "SELECT upper('ß') FROM BTABLE LIMIT 1",
+        };
+        String result[] = {
+                "ABC",
+                "ABC",
+                "ABC",
+                "ĎĎĒ",
+                "SS",
+        };
+        assertEquals(query.length, result.length);
+        String url = PHOENIX_JDBC_URL + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            initTableValues(null, ts);
+            for (int i = 0; i < query.length; i++) {
+                PreparedStatement statement = conn.prepareStatement(query[i]);
+                ResultSet rs = statement.executeQuery();
+                assertTrue(rs.next());
+                assertEquals(query[i],result[i], rs.getString(1));
+                assertFalse(rs.next());
+            }
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testLowerFunction() throws Exception {
+        long ts = nextTimestamp();
+        String query[] = {
+                "SELECT lower('abc') FROM BTABLE LIMIT 1",
+                "SELECT lower('Abc') FROM BTABLE LIMIT 1",
+                "SELECT lower('ABC') FROM BTABLE LIMIT 1",
+                "SELECT lower('ĎďĒ') FROM BTABLE LIMIT 1",
+                "SELECT lower('ß') FROM BTABLE LIMIT 1",
+                "SELECT lower('SS') FROM BTABLE LIMIT 1",
+        };
+        String result[] = {
+                "abc",
+                "abc",
+                "abc",
+                "ďďē",
+                "ß",
+                "ss",
+        };
+        assertEquals(query.length, result.length);
+        String url = PHOENIX_JDBC_URL + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            initTableValues(null, ts);
+            for (int i = 0; i < query.length; i++) {
+                PreparedStatement statement = conn.prepareStatement(query[i]);
+                ResultSet rs = statement.executeQuery();
+                assertTrue(rs.next());
+                assertEquals(query[i],result[i], rs.getString(1));
+                assertFalse(rs.next());
+            }
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
     public void testRTrimFunction() throws Exception {
         long ts = nextTimestamp();
         String query[] = {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/64da9e22/src/test/java/org/apache/phoenix/expression/DescColumnSortOrderExpressionTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/phoenix/expression/DescColumnSortOrderExpressionTest.java b/src/test/java/org/apache/phoenix/expression/DescColumnSortOrderExpressionTest.java
index 9fbcce0..d422b67 100644
--- a/src/test/java/org/apache/phoenix/expression/DescColumnSortOrderExpressionTest.java
+++ b/src/test/java/org/apache/phoenix/expression/DescColumnSortOrderExpressionTest.java
@@ -25,31 +25,17 @@ import static org.junit.Assert.assertTrue;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.sql.Date;
-import java.util.Calendar;
-import java.util.GregorianCalendar;
-import java.util.List;
+import java.util.*;
 
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.phoenix.expression.function.FunctionArgumentType;
-import org.apache.phoenix.expression.function.LTrimFunction;
-import org.apache.phoenix.expression.function.LengthFunction;
-import org.apache.phoenix.expression.function.RTrimFunction;
-import org.apache.phoenix.expression.function.RegexpReplaceFunction;
-import org.apache.phoenix.expression.function.RegexpSubstrFunction;
-import org.apache.phoenix.expression.function.RoundFunction;
-import org.apache.phoenix.expression.function.SqlTypeNameFunction;
-import org.apache.phoenix.expression.function.SubstrFunction;
-import org.apache.phoenix.expression.function.ToCharFunction;
-import org.apache.phoenix.expression.function.ToDateFunction;
-import org.apache.phoenix.expression.function.ToNumberFunction;
-import org.apache.phoenix.expression.function.TrimFunction;
-import org.apache.phoenix.schema.ColumnModifier;
-import org.apache.phoenix.schema.PDataType;
-import org.apache.phoenix.util.DateUtil;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
+import org.apache.phoenix.expression.function.*;
+import org.apache.phoenix.schema.ColumnModifier;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.util.DateUtil;
 
 /**
  * @author simontoens
@@ -96,6 +82,18 @@ public class DescColumnSortOrderExpressionTest {
     }
     
     @Test
+    public void lower() throws Exception {
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral("BLAH", PDataType.CHAR));
+        evaluateAndAssertResult(new LowerFunction(args), "blah");        
+    }
+    
+    @Test
+    public void upper() throws Exception {
+        List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah", PDataType.CHAR));
+        evaluateAndAssertResult(new UpperFunction(args), "BLAH");        
+    }
+    
+    @Test
     public void length() throws Exception {
         List<Expression> args = Lists.newArrayList(getInvertedLiteral("blah", PDataType.CHAR));
         evaluateAndAssertResult(new LengthFunction(args), 4);