You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2015/04/16 16:31:48 UTC

[07/50] [abbrv] phoenix git commit: PHOENIX-1722 Speedup CONVERT_TZ function (Vaclav Loffelmann)

PHOENIX-1722 Speedup CONVERT_TZ function (Vaclav Loffelmann)


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

Branch: refs/heads/calcite
Commit: b9002b7caa54c4fde04b27fe6963719a8d821d7c
Parents: a7d7dfb
Author: Samarth <sa...@salesforce.com>
Authored: Fri Mar 27 14:58:40 2015 -0700
Committer: Samarth <sa...@salesforce.com>
Committed: Fri Mar 27 14:58:40 2015 -0700

----------------------------------------------------------------------
 .../end2end/ConvertTimezoneFunctionIT.java      | 24 +++++-
 .../apache/phoenix/cache/JodaTimezoneCache.java | 84 ++++++++++++++++++++
 .../function/ConvertTimezoneFunction.java       | 38 +++------
 .../function/TimezoneOffsetFunction.java        | 25 ++----
 .../phoenix/cache/JodaTimezoneCacheTest.java    | 51 ++++++++++++
 pom.xml                                         |  2 +-
 6 files changed, 173 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b9002b7c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java
index d89a03b..f415dc6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java
@@ -23,8 +23,10 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Statement;
 
 import org.apache.phoenix.exception.SQLExceptionCode;
+import static org.junit.Assert.assertFalse;
 import org.junit.Test;
 
 /**
@@ -129,7 +131,7 @@ public class ConvertTimezoneFunctionIT extends BaseHBaseManagedTimeIT {
         try {
             ResultSet rs = conn.createStatement().executeQuery(
                     "SELECT k1, dates, CONVERT_TZ(dates, 'UNKNOWN_TIMEZONE', 'America/Adak') FROM TIMEZONE_OFFSET_TEST");
-    
+
             rs.next();
             rs.getDate(3).getTime();
             fail();
@@ -137,4 +139,24 @@ public class ConvertTimezoneFunctionIT extends BaseHBaseManagedTimeIT {
             assertEquals(SQLExceptionCode.ILLEGAL_DATA.getErrorCode(), e.getErrorCode());
         }
     }
+
+	@Test
+	public void testConvertMultipleRecords() throws Exception {
+		Connection conn = DriverManager.getConnection(getUrl());
+		String ddl = "CREATE TABLE IF NOT EXISTS TIMEZONE_OFFSET_TEST (k1 INTEGER NOT NULL, dates DATE CONSTRAINT pk PRIMARY KEY (k1))";
+		Statement stmt = conn.createStatement();
+		stmt.execute(ddl);
+		stmt.execute("UPSERT INTO TIMEZONE_OFFSET_TEST (k1, dates) VALUES (1, TO_DATE('2014-03-01 00:00:00'))");
+		stmt.execute("UPSERT INTO TIMEZONE_OFFSET_TEST (k1, dates) VALUES (2, TO_DATE('2014-03-01 00:00:00'))");
+		conn.commit();
+
+		ResultSet rs = stmt.executeQuery(
+				"SELECT k1, dates, CONVERT_TZ(dates, 'UTC', 'America/Adak') FROM TIMEZONE_OFFSET_TEST");
+
+		assertTrue(rs.next());
+		assertEquals(1393596000000L, rs.getDate(3).getTime()); //Fri, 28 Feb 2014 14:00:00
+		assertTrue(rs.next());
+		assertEquals(1393596000000L, rs.getDate(3).getTime()); //Fri, 28 Feb 2014 14:00:00
+		assertFalse(rs.next());
+	}
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b9002b7c/phoenix-core/src/main/java/org/apache/phoenix/cache/JodaTimezoneCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/JodaTimezoneCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/JodaTimezoneCache.java
new file mode 100644
index 0000000..54904d7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/JodaTimezoneCache.java
@@ -0,0 +1,84 @@
+/*
+ * Copyright 2015 Apache Software Foundation.
+ *
+ * Licensed 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.cache;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.IllegalDataException;
+import org.joda.time.DateTimeZone;
+
+public class JodaTimezoneCache {
+
+    public static final int CACHE_EXPRIRE_TIME_MINUTES = 10;
+    private static final LoadingCache<ByteBuffer, DateTimeZone> cachedJodaTimeZones = createTimezoneCache();
+
+    /**
+     * Returns joda's DateTimeZone instance from cache or create new instance and cache it.
+     *
+     * @param timezoneId Timezone Id as accepted by {@code DateTimeZone.forID()}. E.g. Europe/Isle_of_Man
+     * @return joda's DateTimeZone instance
+     * @throws IllegalDataException if unknown timezone id is passed
+     */
+    public static DateTimeZone getInstance(ByteBuffer timezoneId) {
+        try {
+            return cachedJodaTimeZones.get(timezoneId);
+        } catch (ExecutionException ex) {
+            throw new IllegalDataException(ex);
+        } catch (UncheckedExecutionException e) {
+            throw new IllegalDataException("Unknown timezone " + Bytes.toString(timezoneId.array()));
+        }
+    }
+
+    /**
+     * Returns joda's DateTimeZone instance from cache or create new instance and cache it.
+     *
+     * @param timezoneId Timezone Id as accepted by {@code DateTimeZone.forID()}. E.g. Europe/Isle_of_Man
+     * @return joda's DateTimeZone instance
+     * @throws IllegalDataException if unknown timezone id is passed
+     */
+    public static DateTimeZone getInstance(ImmutableBytesWritable timezoneId) {
+        return getInstance(ByteBuffer.wrap(timezoneId.copyBytes()));
+    }
+
+    /**
+     * Returns joda's DateTimeZone instance from cache or create new instance and cache it.
+     *
+     * @param timezoneId Timezone Id as accepted by {@code DateTimeZone.forID()}. E.g. Europe/Isle_of_Man
+     * @return joda's DateTimeZone instance
+     * @throws IllegalDataException if unknown timezone id is passed
+     */
+    public static DateTimeZone getInstance(String timezoneId) {
+        return getInstance(ByteBuffer.wrap(Bytes.toBytes(timezoneId)));
+    }
+
+    private static LoadingCache<ByteBuffer, DateTimeZone> createTimezoneCache() {
+        return CacheBuilder.newBuilder().expireAfterAccess(CACHE_EXPRIRE_TIME_MINUTES, TimeUnit.MINUTES).build(new CacheLoader<ByteBuffer, DateTimeZone>() {
+
+            @Override
+            public DateTimeZone load(ByteBuffer timezone) throws Exception {
+                return DateTimeZone.forID(Bytes.toString(timezone.array()));
+            }
+        });
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b9002b7c/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java
index dcde31f..3ea47a6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java
@@ -15,21 +15,17 @@
  */
 package org.apache.phoenix.expression.function;
 
-import java.sql.Date;
 import java.sql.SQLException;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
-import java.util.TimeZone;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.cache.JodaTimezoneCache;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.parse.FunctionParseNode;
-import org.apache.phoenix.schema.IllegalDataException;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDate;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.schema.tuple.Tuple;
+import org.joda.time.DateTimeZone;
 
 /**
  * Build in function CONVERT_TZ(date, 'timezone_from', 'timezone_to). Convert date from one timezone to
@@ -43,7 +39,6 @@ import org.apache.phoenix.schema.tuple.Tuple;
 public class ConvertTimezoneFunction extends ScalarFunction {
 
     public static final String NAME = "CONVERT_TZ";
-    private final Map<String, TimeZone> cachedTimeZones = new HashMap<String, TimeZone>();
 
     public ConvertTimezoneFunction() {
     }
@@ -62,40 +57,25 @@ public class ConvertTimezoneFunction extends ScalarFunction {
         if (!children.get(0).evaluate(tuple, ptr)) {
             return false;
         }
-
-        Date dateo = (Date) PDate.INSTANCE.toObject(ptr, children.get(0).getSortOrder());
-        Long date = dateo.getTime();
+        long date = PDate.INSTANCE.getCodec().decodeLong(ptr, children.get(0).getSortOrder());
 
         if (!children.get(1).evaluate(tuple, ptr)) {
             return false;
         }
-        TimeZone timezoneFrom = getTimezoneFromCache(Bytes.toString(ptr.get(), ptr.getOffset(), ptr.getLength()));
+        DateTimeZone timezoneFrom = JodaTimezoneCache.getInstance(ptr);
 
         if (!children.get(2).evaluate(tuple, ptr)) {
             return false;
         }
-        TimeZone timezoneTo = TimeZone.getTimeZone(Bytes.toString(ptr.get(), ptr.getOffset(), ptr.getLength()));
-
-        long dateInUtc = date - timezoneFrom.getOffset(date);
-        long dateInTo = dateInUtc + timezoneTo.getOffset(dateInUtc);
-
-        ptr.set(PDate.INSTANCE.toBytes(new Date(dateInTo)));
+        DateTimeZone timezoneTo = JodaTimezoneCache.getInstance(ptr);
 
+        long convertedDate = date - timezoneFrom.getOffset(date) + timezoneTo.getOffset(date);
+        byte[] outBytes = new byte[8];
+        PDate.INSTANCE.getCodec().encodeLong(convertedDate, outBytes, 0);
+        ptr.set(outBytes);
         return true;
     }
 
-    private TimeZone getTimezoneFromCache(String timezone) throws IllegalDataException {
-        if (!cachedTimeZones.containsKey(timezone)) {
-            TimeZone tz = TimeZone.getTimeZone(timezone);
-            if (!tz.getID().equals(timezone)) {
-                throw new IllegalDataException("Invalid timezone " + timezone);
-            }
-            cachedTimeZones.put(timezone, tz);
-            return tz;
-        }
-        return cachedTimeZones.get(timezone);
-    }
-
     @Override
     public PDataType getDataType() {
         return PDate.INSTANCE;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b9002b7c/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TimezoneOffsetFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TimezoneOffsetFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TimezoneOffsetFunction.java
index 2cfbc25..8c70346 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TimezoneOffsetFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TimezoneOffsetFunction.java
@@ -18,22 +18,18 @@
 
 package org.apache.phoenix.expression.function;
 
-import java.sql.Date;
 import java.sql.SQLException;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
-import java.util.TimeZone;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.cache.JodaTimezoneCache;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.parse.FunctionParseNode;
-import org.apache.phoenix.schema.IllegalDataException;
 import org.apache.phoenix.schema.types.PDate;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.schema.tuple.Tuple;
+import org.joda.time.DateTimeZone;
 
 /**
  * Returns offset (shift in minutes) of timezone at particular datetime in minutes.
@@ -45,7 +41,6 @@ public class TimezoneOffsetFunction extends ScalarFunction {
 
     public static final String NAME = "TIMEZONE_OFFSET";
     private static final int MILLIS_TO_MINUTES = 60 * 1000;
-    private final Map<String, TimeZone> cachedTimeZones = new HashMap<String, TimeZone>();
 
     public TimezoneOffsetFunction() {
     }
@@ -64,24 +59,14 @@ public class TimezoneOffsetFunction extends ScalarFunction {
         if (!children.get(0).evaluate(tuple, ptr)) {
             return false;
         }
-
-        String timezone = Bytes.toString(ptr.get(), ptr.getOffset(), ptr.getLength());
+        DateTimeZone timezoneInstance = JodaTimezoneCache.getInstance(ptr);
 
         if (!children.get(1).evaluate(tuple, ptr)) {
             return false;
         }
+        long date = PDate.INSTANCE.getCodec().decodeLong(ptr, children.get(1).getSortOrder());
 
-        if (!cachedTimeZones.containsKey(timezone)) {
-            TimeZone tz = TimeZone.getTimeZone(timezone);
-            if (!tz.getID().equals(timezone)) {
-                throw new IllegalDataException("Invalid timezone " + timezone);
-            }
-            cachedTimeZones.put(timezone, tz);
-        }
-
-		Date date = (Date) PDate.INSTANCE.toObject(ptr, children.get(1).getSortOrder());
-		int offset = cachedTimeZones.get(timezone).getOffset(date.getTime());
-
+        int offset = timezoneInstance.getOffset(date);
         ptr.set(PInteger.INSTANCE.toBytes(offset / MILLIS_TO_MINUTES));
         return true;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b9002b7c/phoenix-core/src/test/java/org/apache/phoenix/cache/JodaTimezoneCacheTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/cache/JodaTimezoneCacheTest.java b/phoenix-core/src/test/java/org/apache/phoenix/cache/JodaTimezoneCacheTest.java
new file mode 100644
index 0000000..f388703
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/cache/JodaTimezoneCacheTest.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2015 Apache Software Foundation.
+ *
+ * Licensed 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.cache;
+
+import java.nio.ByteBuffer;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.IllegalDataException;
+import org.joda.time.DateTimeZone;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
+
+public class JodaTimezoneCacheTest {
+
+    @Test
+    public void testGetInstanceByteBufferUTC() {
+        DateTimeZone instance = JodaTimezoneCache.getInstance(ByteBuffer.wrap(Bytes.toBytes("UTC")));
+        assertTrue(instance instanceof DateTimeZone);
+    }
+
+    @Test
+    public void testGetInstanceString() {
+        DateTimeZone instance = JodaTimezoneCache.getInstance("America/St_Vincent");
+        assertTrue(instance instanceof DateTimeZone);
+    }
+
+    @Test(expected = IllegalDataException.class)
+    public void testGetInstanceStringUnknown() {
+        JodaTimezoneCache.getInstance("SOME_UNKNOWN_TIMEZONE");
+    }
+
+    @Test
+    public void testGetInstanceImmutableBytesWritable() {
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable(Bytes.toBytes("Europe/Isle_of_Man"));
+        DateTimeZone instance = JodaTimezoneCache.getInstance(ptr);
+        assertTrue(instance instanceof DateTimeZone);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b9002b7c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ecfd3ec..861c868 100644
--- a/pom.xml
+++ b/pom.xml
@@ -103,7 +103,7 @@
     <commons-codec.version>1.7</commons-codec.version>
     <htrace.version>3.1.0-incubating</htrace.version>
     <collections.version>3.2.1</collections.version>
-    <jodatime.version>2.3</jodatime.version>
+    <jodatime.version>2.7</jodatime.version>
 
     <!-- Test Dependencies -->
     <mockito-all.version>1.8.5</mockito-all.version>