Return-Path: X-Original-To: apmail-phoenix-commits-archive@minotaur.apache.org Delivered-To: apmail-phoenix-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id C4B4B10D17 for ; Wed, 3 Dec 2014 16:36:09 +0000 (UTC) Received: (qmail 68338 invoked by uid 500); 3 Dec 2014 16:36:09 -0000 Delivered-To: apmail-phoenix-commits-archive@phoenix.apache.org Received: (qmail 68292 invoked by uid 500); 3 Dec 2014 16:36:09 -0000 Mailing-List: contact commits-help@phoenix.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@phoenix.apache.org Delivered-To: mailing list commits@phoenix.apache.org Received: (qmail 68279 invoked by uid 99); 3 Dec 2014 16:36:09 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 03 Dec 2014 16:36:09 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 6BFD9A08F5E; Wed, 3 Dec 2014 16:36:09 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: greid@apache.org To: commits@phoenix.apache.org Date: Wed, 03 Dec 2014 16:36:09 -0000 Message-Id: <7f10b37ba04e49c48ab708e08bf39afc@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [1/2] phoenix git commit: PHOENIX-1485 Add timezone awareness Repository: phoenix Updated Branches: refs/heads/3.2 44bd4bec1 -> cdcbcf218 PHOENIX-1485 Add timezone awareness * Add tz-aware methods to DateUtil * Add timezone parameter to TO_DATE * Add a configuration parameter to allow specifying the time zone to be used internally on a connection when parsing dates. Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/5911b4af Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/5911b4af Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/5911b4af Branch: refs/heads/3.2 Commit: 5911b4afe871d6161ccf06eab651763c3f784712 Parents: 44bd4be Author: Gabriel Reid Authored: Tue Dec 2 19:25:41 2014 +0100 Committer: Gabriel Reid Committed: Wed Dec 3 17:32:33 2014 +0100 ---------------------------------------------------------------------- .../phoenix/end2end/ToDateFunctionIT.java | 117 +++++++++++++++++++ .../phoenix/compile/StatementContext.java | 42 +++---- .../expression/function/ToDateFunction.java | 12 +- .../apache/phoenix/parse/ToDateParseNode.java | 15 ++- .../org/apache/phoenix/query/QueryServices.java | 1 + .../phoenix/query/QueryServicesOptions.java | 113 +++++++++--------- .../java/org/apache/phoenix/util/DateUtil.java | 37 ++++-- .../expression/SortOrderExpressionTest.java | 3 +- .../org/apache/phoenix/util/DateUtilTest.java | 78 ++++++++++++- 9 files changed, 320 insertions(+), 98 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/5911b4af/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToDateFunctionIT.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToDateFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToDateFunctionIT.java new file mode 100644 index 0000000..19257c1 --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToDateFunctionIT.java @@ -0,0 +1,117 @@ +/* + * 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 java.sql.Connection; +import java.sql.Date; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Properties; + +import org.apache.phoenix.query.QueryServices; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + + +public class ToDateFunctionIT extends BaseHBaseManagedTimeIT { + + private static final long ONE_HOUR_IN_MILLIS = 1000L * 60L * 60L; + + private Connection conn; + + @Before + public void setUp() throws SQLException { + conn = DriverManager.getConnection(getUrl()); + } + + @After + public void tearDown() throws SQLException { + conn.close(); + } + + private static Date callToDateFunction(Connection conn, String invocation) throws SQLException { + Statement stmt = conn.createStatement(); + ResultSet rs = stmt.executeQuery(String.format("SELECT %s FROM SYSTEM.CATALOG", invocation)); + assertTrue(rs.next()); + Date returnValue = rs.getDate(1); + rs.close(); + stmt.close(); + return returnValue; + } + + private Date callToDateFunction(String invocation) throws SQLException { + return callToDateFunction(conn, invocation); + } + + @Test + public void testToDate_Default() throws SQLException { + // Default time zone is GMT, so this is timestamp 0 + assertEquals(0L, callToDateFunction("TO_DATE('1970-01-01 00:00:00')").getTime()); + } + + @Test + public void testToDate_CustomDateFormat() throws SQLException { + // A date without time component is at midnight + assertEquals(0L, callToDateFunction("TO_DATE('1970-01-01', 'yyyy-MM-dd')").getTime()); + } + + @Test + public void testToDate_CustomTimeZone() throws SQLException { + // We're using GMT+1, so that's an hour before the Java epoch + assertEquals( + -ONE_HOUR_IN_MILLIS, + callToDateFunction("TO_DATE('1970-01-01', 'yyyy-MM-dd', 'GMT+1')").getTime()); + } + + @Test + public void testToDate_LocalTimeZone() throws SQLException { + assertEquals( + Date.valueOf("1970-01-01"), + callToDateFunction("TO_DATE('1970-01-01', 'yyyy-MM-dd', 'local')")); + } + + @Test + public void testToDate_CustomTimeZoneViaQueryServices() throws SQLException { + Properties props = new Properties(); + props.setProperty(QueryServices.DATE_FORMAT_TIMEZONE_ATTRIB, "GMT+1"); + Connection customTimeZoneConn = DriverManager.getConnection(getUrl(), props); + + assertEquals( + -ONE_HOUR_IN_MILLIS, + callToDateFunction(customTimeZoneConn, "TO_DATE('1970-01-01 00:00:00')").getTime()); + } + + @Test + public void testToDate_CustomTimeZoneViaQueryServicesAndCustomFormat() throws SQLException { + Properties props = new Properties(); + props.setProperty(QueryServices.DATE_FORMAT_TIMEZONE_ATTRIB, "GMT+1"); + Connection customTimeZoneConn = DriverManager.getConnection(getUrl(), props); + + assertEquals( + -ONE_HOUR_IN_MILLIS, + callToDateFunction( + customTimeZoneConn, "TO_DATE('1970-01-01', 'yyyy-MM-dd')").getTime()); + } +} http://git-wip-us.apache.org/repos/asf/phoenix/blob/5911b4af/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java index bfceff7..953ac10 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/StatementContext.java @@ -22,6 +22,7 @@ import java.text.Format; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.TimeZone; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; @@ -46,7 +47,7 @@ import com.google.common.collect.Maps; * Class that keeps common state used across processing the various clauses in a * top level JDBC statement such as SELECT, UPSERT, DELETE, etc. * - * + * * @since 0.1 */ public class StatementContext { @@ -56,26 +57,26 @@ public class StatementContext { private final ExpressionManager expressions; private final AggregationManager aggregates; private final String dateFormat; + private final TimeZone dateFormatTimeZone; private final Format dateFormatter; - private final Format dateParser; private final String numberFormat; private final ImmutableBytesWritable tempPtr; private final PhoenixStatement statement; - + private long currentTime = QueryConstants.UNSET_TIMESTAMP; private ScanRanges scanRanges = ScanRanges.EVERYTHING; - private final SequenceManager sequences; + private final SequenceManager sequences; private TableRef currentTable; private List> whereConditionColumns; private TimeRange scanTimeRange = null; - + private Map subqueryResults; - + public StatementContext(PhoenixStatement statement) { this(statement, new Scan()); } - + public StatementContext(PhoenixStatement statement, Scan scan) { this(statement, FromCompiler.EMPTY_TABLE_RESOLVER, new Scan(), new SequenceManager(statement)); } @@ -94,8 +95,9 @@ public class StatementContext { this.expressions = new ExpressionManager(); PhoenixConnection connection = statement.getConnection(); this.dateFormat = connection.getQueryServices().getProps().get(QueryServices.DATE_FORMAT_ATTRIB, DateUtil.DEFAULT_DATE_FORMAT); + this.dateFormatTimeZone = TimeZone.getTimeZone( + connection.getQueryServices().getProps().get(QueryServices.DATE_FORMAT_TIMEZONE_ATTRIB, DateUtil.DEFAULT_TIME_ZONE_ID)); this.dateFormatter = DateUtil.getDateFormatter(dateFormat); - this.dateParser = DateUtil.getDateParser(dateFormat); this.numberFormat = connection.getQueryServices().getProps().get(QueryServices.NUMBER_FORMAT_ATTRIB, NumberUtil.DEFAULT_NUMBER_FORMAT); this.tempPtr = new ImmutableBytesWritable(); this.currentTable = resolver != null && !resolver.getTables().isEmpty() ? resolver.getTables().get(0) : null; @@ -107,18 +109,18 @@ public class StatementContext { return dateFormat; } + public TimeZone getDateFormatTimeZone() { + return dateFormatTimeZone; + } + public Format getDateFormatter() { return dateFormatter; } - public Format getDateParser() { - return dateParser; - } - public String getNumberFormat() { return numberFormat; } - + public Scan getScan() { return scan; } @@ -126,11 +128,11 @@ public class StatementContext { public BindManager getBindManager() { return binds; } - + public TableRef getCurrentTable() { return currentTable; } - + public void setCurrentTable(TableRef table) { this.currentTable = table; } @@ -159,12 +161,12 @@ public class StatementContext { public ScanRanges getScanRanges() { return this.scanRanges; } - + public void setScanRanges(ScanRanges scanRanges) { this.scanRanges = scanRanges; scanRanges.initializeScan(scan); } - + public PhoenixConnection getConnection() { return statement.getConnection(); } @@ -209,11 +211,11 @@ public class StatementContext { public void setScanTimeRange(TimeRange value){ this.scanTimeRange = value; } - + public TimeRange getScanTimeRange() { return this.scanTimeRange; } - + public boolean isSubqueryResultAvailable(SelectStatement select) { return subqueryResults.containsKey(select); } @@ -221,7 +223,7 @@ public class StatementContext { public Object getSubqueryResult(SelectStatement select) { return subqueryResults.get(select); } - + public void setSubqueryResult(SelectStatement select, Object result) { subqueryResults.put(select, result); } http://git-wip-us.apache.org/repos/asf/phoenix/blob/5911b4af/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java index e14cb1d..2828e49 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java @@ -37,14 +37,18 @@ import org.apache.phoenix.util.DateUtil; /** * - * Implementation of the TO_DATE(,[]) built-in function. + * Implementation of the {@code TO_DATE(,[,[]])} built-in function. * The second argument is optional and defaults to the phoenix.query.dateFormat value - * from the HBase config. If present it must be a constant string. - * + * from the HBase config. If present it must be a constant string. The third argument is either a + * valid (constant) timezone id, or the string "local". The third argument is also optional, and + * it defaults to GMT. * * @since 0.1 */ -@BuiltInFunction(name=ToDateFunction.NAME, nodeClass=ToDateParseNode.class, args= {@Argument(allowedTypes={PDataType.VARCHAR}),@Argument(allowedTypes={PDataType.VARCHAR},isConstant=true,defaultValue="null")} ) +@BuiltInFunction(name=ToDateFunction.NAME, nodeClass=ToDateParseNode.class, + args={@Argument(allowedTypes={PDataType.VARCHAR}), + @Argument(allowedTypes={PDataType.VARCHAR},isConstant=true,defaultValue="null"), + @Argument(allowedTypes={PDataType.VARCHAR}, isConstant=true, defaultValue = "null") } ) public class ToDateFunction extends ScalarFunction { public static final String NAME = "TO_DATE"; private Format dateParser; http://git-wip-us.apache.org/repos/asf/phoenix/blob/5911b4af/phoenix-core/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java index 41f8b43..46bca63 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ToDateParseNode.java @@ -20,6 +20,7 @@ package org.apache.phoenix.parse; import java.sql.SQLException; import java.text.Format; import java.util.List; +import java.util.TimeZone; import org.apache.phoenix.compile.StatementContext; import org.apache.phoenix.expression.Expression; @@ -30,6 +31,7 @@ import org.apache.phoenix.util.DateUtil; public class ToDateParseNode extends FunctionParseNode { + public ToDateParseNode(String name, List children, BuiltInFunctionInfo info) { super(name, children, info); } @@ -37,13 +39,20 @@ public class ToDateParseNode extends FunctionParseNode { @Override public FunctionExpression create(List children, StatementContext context) throws SQLException { Format dateParser; - String dateFormat = (String)((LiteralExpression)children.get(1)).getValue(); + String dateFormat = (String) ((LiteralExpression) children.get(1)).getValue(); + String timeZoneId = (String) ((LiteralExpression) children.get(2)).getValue(); + TimeZone parserTimeZone = context.getDateFormatTimeZone(); if (dateFormat == null) { dateFormat = context.getDateFormat(); - dateParser = context.getDateParser(); + } + if (timeZoneId == null) { + parserTimeZone = context.getDateFormatTimeZone(); + } else if ("LOCAL".equalsIgnoreCase(timeZoneId)) { + parserTimeZone = TimeZone.getDefault(); } else { - dateParser = DateUtil.getDateParser(dateFormat); + parserTimeZone = TimeZone.getTimeZone(timeZoneId); } + dateParser = DateUtil.getDateParser(dateFormat, parserTimeZone); return new ToDateFunction(children, dateFormat, dateParser); } } http://git-wip-us.apache.org/repos/asf/phoenix/blob/5911b4af/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java index 31661a0..f444df9 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java @@ -68,6 +68,7 @@ public interface QueryServices extends SQLCloseable { public static final String MAX_MEMORY_WAIT_MS_ATTRIB = "phoenix.query.maxGlobalMemoryWaitMs"; public static final String MAX_TENANT_MEMORY_PERC_ATTRIB = "phoenix.query.maxTenantMemoryPercentage"; public static final String MAX_SERVER_CACHE_SIZE_ATTRIB = "phoenix.query.maxServerCacheBytes"; + public static final String DATE_FORMAT_TIMEZONE_ATTRIB = "phoenix.query.dateFormatTimeZone"; public static final String DATE_FORMAT_ATTRIB = "phoenix.query.dateFormat"; public static final String NUMBER_FORMAT_ATTRIB = "phoenix.query.numberFormat"; public static final String CALL_QUEUE_ROUND_ROBIN_ATTRIB = "ipc.server.callqueue.roundrobin"; http://git-wip-us.apache.org/repos/asf/phoenix/blob/5911b4af/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java index f4386b0..ff7d1dd 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java @@ -20,6 +20,7 @@ package org.apache.phoenix.query; import static org.apache.phoenix.query.QueryServices.CALL_QUEUE_PRODUCER_ATTRIB_NAME; import static org.apache.phoenix.query.QueryServices.CALL_QUEUE_ROUND_ROBIN_ATTRIB; import static org.apache.phoenix.query.QueryServices.DATE_FORMAT_ATTRIB; +import static org.apache.phoenix.query.QueryServices.DATE_FORMAT_TIMEZONE_ATTRIB; import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB; import static org.apache.phoenix.query.QueryServices.EXPLAIN_CHUNK_COUNT_ATTRIB; import static org.apache.phoenix.query.QueryServices.GROUPBY_MAX_CACHE_SIZE_ATTRIB; @@ -70,8 +71,8 @@ import org.apache.phoenix.util.ReadOnlyProps; /** * Options for {@link QueryServices}. - * - * + * + * * @since 0.1 */ public class QueryServicesOptions { @@ -88,13 +89,14 @@ public class QueryServicesOptions { public static final int DEFAULT_TARGET_QUERY_CONCURRENCY = 32; public static final int DEFAULT_MAX_QUERY_CONCURRENCY = 64; public static final String DEFAULT_DATE_FORMAT = DateUtil.DEFAULT_DATE_FORMAT; - public static final boolean DEFAULT_CALL_QUEUE_ROUND_ROBIN = true; + public static final String DEFAULT_DATE_FORMAT_TIMEZONE = DateUtil.DEFAULT_TIME_ZONE_ID; + public static final boolean DEFAULT_CALL_QUEUE_ROUND_ROBIN = true; public static final int DEFAULT_MAX_MUTATION_SIZE = 500000; public static final boolean DEFAULT_ROW_KEY_ORDER_SALTED_TABLE = true; // Merge sort on client to ensure salted tables are row key ordered public static final boolean DEFAULT_USE_INDEXES = true; // Use indexes public static final boolean DEFAULT_IMMUTABLE_ROWS = false; // Tables rows may be updated public static final boolean DEFAULT_DROP_METADATA = true; // Drop meta data also. - + public final static int DEFAULT_MUTATE_BATCH_SIZE = 1000; // Batch size for UPSERT SELECT and DELETE // The only downside of it being out-of-sync is that the parallelization of the scan won't be as balanced as it could be. public static final int DEFAULT_MAX_SERVER_CACHE_TIME_TO_LIVE_MS = 30000; // 30 sec (with no activity) @@ -109,8 +111,8 @@ public class QueryServicesOptions { // latency and less parallelization. public static final long DEFAULT_SCAN_RESULT_CHUNK_SIZE = 2999; public static final long DEFAULT_HISTOGRAM_BYTE_DEPTH = 1024 * 1024; - - // + + // // Spillable GroupBy - SPGBY prefix // // Enable / disable spillable group by @@ -120,10 +122,10 @@ public class QueryServicesOptions { public static final int DEFAULT_GROUPBY_SPILL_FILES = 2; // Max size of 1st level main memory cache in bytes --> upper bound public static final long DEFAULT_GROUPBY_MAX_CACHE_MAX = 1024L*1024L*100L; // 100 Mb - + public static final long DEFAULT_SEQUENCE_CACHE_SIZE = 100; // reserve 100 sequences at a time public static final int DEFAULT_INDEX_MAX_FILESIZE_PERC = 50; // % of data table max file size for index table - public static final long DEFAULT_MAX_SERVER_METADATA_CACHE_TIME_TO_LIVE_MS = 60000 * 30; // 30 mins + public static final long DEFAULT_MAX_SERVER_METADATA_CACHE_TIME_TO_LIVE_MS = 60000 * 30; // 30 mins public static final long DEFAULT_MAX_SERVER_METADATA_CACHE_SIZE = 1024L*1024L*20L; // 20 Mb public static final long DEFAULT_MAX_CLIENT_METADATA_CACHE_SIZE = 1024L*1024L*10L; // 10 Mb public static final int DEFAULT_GROUPBY_ESTIMATED_DISTINCT_VALUES = 1000; @@ -131,7 +133,7 @@ public class QueryServicesOptions { public static final boolean DEFAULT_INDEX_FAILURE_HANDLING_REBUILD = true; // auto rebuild on public static final long DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_INTERVAL = 10000; // 10 secs public static final long DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME = 300000; // 5 mins - + public static final long DEFAULT_STATS_HISTOGRAM_DEPTH_BYTE = 1024 * 1024 * 30; public static final int DEFAULT_STATS_UPDATE_FREQ_MS = 15 * 60000; // 15min public static final int DEFAULT_STATS_GUIDEPOST_PER_REGION = 0; // Uses guidepost width by default @@ -142,7 +144,7 @@ public class QueryServicesOptions { public static final boolean DEFAULT_STATS_USE_CURRENT_TIME = true; public static final boolean DEFAULT_USE_REVERSE_SCAN = true; - + /** * Use only first time SYSTEM.SEQUENCE table is created. */ @@ -154,15 +156,15 @@ public class QueryServicesOptions { public static final boolean DEFAULT_EXPLAIN_CHUNK_COUNT = true; private final Configuration config; - + private QueryServicesOptions(Configuration config) { this.config = config; } - + public ReadOnlyProps getProps(ReadOnlyProps defaultProps) { return new ReadOnlyProps(defaultProps, config.iterator()); } - + public QueryServicesOptions setAll(ReadOnlyProps props) { for (Entry entry : props) { config.set(entry.getKey(), entry.getValue()); @@ -186,6 +188,7 @@ public class QueryServicesOptions { .setIfUnset(MAX_SERVER_CACHE_SIZE_ATTRIB, DEFAULT_MAX_SERVER_CACHE_SIZE) .setIfUnset(SCAN_CACHE_SIZE_ATTRIB, DEFAULT_SCAN_CACHE_SIZE) .setIfUnset(DATE_FORMAT_ATTRIB, DEFAULT_DATE_FORMAT) + .setIfUnset(DATE_FORMAT_TIMEZONE_ATTRIB, DEFAULT_DATE_FORMAT_TIMEZONE) .setIfUnset(STATS_UPDATE_FREQ_MS_ATTRIB, DEFAULT_STATS_UPDATE_FREQ_MS) .setIfUnset(CALL_QUEUE_ROUND_ROBIN_ATTRIB, DEFAULT_CALL_QUEUE_ROUND_ROBIN) .setIfUnset(MAX_MUTATION_SIZE_ATTRIB, DEFAULT_MAX_MUTATION_SIZE) @@ -212,7 +215,7 @@ public class QueryServicesOptions { } return options; } - + public Configuration getConfiguration() { return config; } @@ -221,12 +224,12 @@ public class QueryServicesOptions { config.setIfUnset(name, Integer.toString(value)); return this; } - + private QueryServicesOptions setIfUnset(String name, boolean value) { config.setIfUnset(name, Boolean.toString(value)); return this; } - + private QueryServicesOptions setIfUnset(String name, long value) { config.setIfUnset(name, Long.toString(value)); return this; @@ -236,24 +239,24 @@ public class QueryServicesOptions { config.setIfUnset(name, value); return this; } - + public QueryServicesOptions setKeepAliveMs(int keepAliveMs) { return set(KEEP_ALIVE_MS_ATTRIB, keepAliveMs); } - + public QueryServicesOptions setThreadPoolSize(int threadPoolSize) { return set(THREAD_POOL_SIZE_ATTRIB, threadPoolSize); } - + public QueryServicesOptions setQueueSize(int queueSize) { config.setInt(QUEUE_SIZE_ATTRIB, queueSize); return this; } - + public QueryServicesOptions setThreadTimeoutMs(int threadTimeoutMs) { return set(THREAD_TIMEOUT_MS_ATTRIB, threadTimeoutMs); } - + public QueryServicesOptions setSpoolThresholdBytes(int spoolThresholdBytes) { return set(SPOOL_THRESHOLD_BYTES_ATTRIB, spoolThresholdBytes); } @@ -265,15 +268,15 @@ public class QueryServicesOptions { public QueryServicesOptions setMaxMemoryPerc(int maxMemoryPerc) { return set(MAX_MEMORY_PERC_ATTRIB, maxMemoryPerc); } - + public QueryServicesOptions setMaxMemoryWaitMs(int maxMemoryWaitMs) { return set(MAX_MEMORY_WAIT_MS_ATTRIB, maxMemoryWaitMs); } - + public QueryServicesOptions setMaxTenantMemoryPerc(int maxTenantMemoryPerc) { return set(MAX_TENANT_MEMORY_PERC_ATTRIB, maxTenantMemoryPerc); } - + public QueryServicesOptions setMaxServerCacheSize(long maxServerCacheSize) { return set(MAX_SERVER_CACHE_SIZE_ATTRIB, maxServerCacheSize); } @@ -289,31 +292,31 @@ public class QueryServicesOptions { public QueryServicesOptions setScanFetchSize(int scanFetchSize) { return set(SCAN_CACHE_SIZE_ATTRIB, scanFetchSize); } - + public QueryServicesOptions setDateFormat(String dateFormat) { return set(DATE_FORMAT_ATTRIB, dateFormat); } - + public QueryServicesOptions setCallQueueRoundRobin(boolean isRoundRobin) { return set(CALL_QUEUE_PRODUCER_ATTRIB_NAME, isRoundRobin); } - + public QueryServicesOptions setMaxMutateSize(int maxMutateSize) { return set(MAX_MUTATION_SIZE_ATTRIB, maxMutateSize); } - + public QueryServicesOptions setMutateBatchSize(int mutateBatchSize) { return set(MUTATE_BATCH_SIZE_ATTRIB, mutateBatchSize); } - + public QueryServicesOptions setRowKeyOrderSaltedTable(boolean rowKeyOrderSaltedTable) { return set(ROW_KEY_ORDER_SALTED_TABLE_ATTRIB, rowKeyOrderSaltedTable); } - + public QueryServicesOptions setDropMetaData(boolean dropMetadata) { return set(DROP_METADATA_ATTRIB, dropMetadata); } - + public QueryServicesOptions setGroupBySpill(boolean enabled) { return set(GROUPBY_SPILLABLE_ATTRIB, enabled); } @@ -321,27 +324,27 @@ public class QueryServicesOptions { public QueryServicesOptions setGroupBySpillMaxCacheSize(long size) { return set(GROUPBY_MAX_CACHE_SIZE_ATTRIB, size); } - + public QueryServicesOptions setGroupBySpillNumSpillFiles(long num) { return set(GROUPBY_SPILL_FILES_ATTRIB, num); } - + private QueryServicesOptions set(String name, boolean value) { config.set(name, Boolean.toString(value)); return this; } - + private QueryServicesOptions set(String name, int value) { config.set(name, Integer.toString(value)); return this; } - + private QueryServicesOptions set(String name, String value) { config.set(name, value); return this; } - + private QueryServicesOptions set(String name, long value) { config.set(name, Long.toString(value)); return this; @@ -350,19 +353,19 @@ public class QueryServicesOptions { public int getKeepAliveMs() { return config.getInt(KEEP_ALIVE_MS_ATTRIB, DEFAULT_KEEP_ALIVE_MS); } - + public int getThreadPoolSize() { return config.getInt(THREAD_POOL_SIZE_ATTRIB, DEFAULT_THREAD_POOL_SIZE); } - + public int getQueueSize() { return config.getInt(QUEUE_SIZE_ATTRIB, DEFAULT_QUEUE_SIZE); } - + public int getMaxMemoryPerc() { return config.getInt(MAX_MEMORY_PERC_ATTRIB, DEFAULT_MAX_MEMORY_PERC); } - + public int getMaxMemoryWaitMs() { return config.getInt(MAX_MEMORY_WAIT_MS_ATTRIB, DEFAULT_MAX_MEMORY_WAIT_MS); } @@ -382,19 +385,19 @@ public class QueryServicesOptions { public boolean isImmutableRows() { return config.getBoolean(IMMUTABLE_ROWS_ATTRIB, DEFAULT_IMMUTABLE_ROWS); } - + public boolean isDropMetaData() { return config.getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA); } - + public boolean isSpillableGroupByEnabled() { return config.getBoolean(GROUPBY_SPILLABLE_ATTRIB, DEFAULT_GROUPBY_SPILLABLE); } - + public long getSpillableGroupByMaxCacheSize() { return config.getLong(GROUPBY_MAX_CACHE_SIZE_ATTRIB, DEFAULT_GROUPBY_MAX_CACHE_MAX); } - + public int getSpillableGroupByNumSpillFiles() { return config.getInt(GROUPBY_SPILL_FILES_ATTRIB, DEFAULT_GROUPBY_SPILL_FILES); } @@ -402,27 +405,27 @@ public class QueryServicesOptions { public QueryServicesOptions setMaxServerCacheTTLMs(int ttl) { return set(MAX_SERVER_CACHE_TIME_TO_LIVE_MS_ATTRIB, ttl); } - + public QueryServicesOptions setMasterInfoPort(int port) { return set(MASTER_INFO_PORT_ATTRIB, port); } - + public QueryServicesOptions setRegionServerInfoPort(int port) { return set(REGIONSERVER_INFO_PORT_ATTRIB, port); } - + public QueryServicesOptions setRegionServerLeasePeriodMs(int period) { return set(REGIONSERVER_LEASE_PERIOD_ATTRIB, period); } - + public QueryServicesOptions setRpcTimeoutMs(int timeout) { return set(RPC_TIMEOUT_ATTRIB, timeout); } - + public QueryServicesOptions setUseIndexes(boolean useIndexes) { return set(USE_INDEXES_ATTRIB, useIndexes); } - + public QueryServicesOptions setImmutableRows(boolean isImmutableRows) { return set(IMMUTABLE_ROWS_ATTRIB, isImmutableRows); } @@ -438,19 +441,19 @@ public class QueryServicesOptions { public QueryServicesOptions setStatsUpdateFrequencyMs(int frequencyMs) { return set(STATS_UPDATE_FREQ_MS_ATTRIB, frequencyMs); } - + public QueryServicesOptions setMinStatsUpdateFrequencyMs(int frequencyMs) { return set(MIN_STATS_UPDATE_FREQ_MS_ATTRIB, frequencyMs); - } - + } + public QueryServicesOptions setSequenceSaltBuckets(int saltBuckets) { config.setInt(SEQUENCE_SALT_BUCKETS_ATTRIB, saltBuckets); return this; } - + public QueryServicesOptions setExplainChunkCount(boolean showChunkCount) { config.setBoolean(EXPLAIN_CHUNK_COUNT_ATTRIB, showChunkCount); return this; } - + } http://git-wip-us.apache.org/repos/asf/phoenix/blob/5911b4af/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java index 62d892f..c940067 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/util/DateUtil.java @@ -35,17 +35,20 @@ import org.apache.phoenix.schema.IllegalDataException; @SuppressWarnings("serial") public class DateUtil { - public static final TimeZone DATE_TIME_ZONE = TimeZone.getTimeZone("GMT"); + public static final String DEFAULT_TIME_ZONE_ID = "GMT"; + private static final TimeZone DEFAULT_TIME_ZONE = TimeZone.getTimeZone(DEFAULT_TIME_ZONE_ID); public static final String DEFAULT_DATE_FORMAT = "yyyy-MM-dd HH:mm:ss"; // This is the format the app sets in NLS settings for every connection. - public static final Format DEFAULT_DATE_FORMATTER = FastDateFormat.getInstance(DEFAULT_DATE_FORMAT, DATE_TIME_ZONE); + public static final Format DEFAULT_DATE_FORMATTER = FastDateFormat.getInstance( + DEFAULT_DATE_FORMAT, TimeZone.getTimeZone(DEFAULT_TIME_ZONE_ID)); public static final String DEFAULT_MS_DATE_FORMAT = "yyyy-MM-dd HH:mm:ss.SSS"; - public static final Format DEFAULT_MS_DATE_FORMATTER = FastDateFormat.getInstance(DEFAULT_MS_DATE_FORMAT, DATE_TIME_ZONE); + public static final Format DEFAULT_MS_DATE_FORMATTER = FastDateFormat.getInstance( + DEFAULT_MS_DATE_FORMAT, TimeZone.getTimeZone(DEFAULT_TIME_ZONE_ID)); private DateUtil() { } - public static Format getDateParser(String pattern) { + public static Format getDateParser(String pattern, TimeZone timeZone) { SimpleDateFormat format = new SimpleDateFormat(pattern) { @Override public java.util.Date parseObject(String source) throws ParseException { @@ -53,11 +56,15 @@ public class DateUtil { return new java.sql.Date(date.getTime()); } }; - format.setTimeZone(DateUtil.DATE_TIME_ZONE); + format.setTimeZone(timeZone); return format; } - public static Format getTimeParser(String pattern) { + public static Format getDateParser(String pattern) { + return getDateParser(pattern, DEFAULT_TIME_ZONE); + } + + public static Format getTimeParser(String pattern, TimeZone timeZone) { SimpleDateFormat format = new SimpleDateFormat(pattern) { @Override public java.util.Date parseObject(String source) throws ParseException { @@ -65,11 +72,15 @@ public class DateUtil { return new java.sql.Time(date.getTime()); } }; - format.setTimeZone(DateUtil.DATE_TIME_ZONE); + format.setTimeZone(timeZone); return format; } - public static Format getTimestampParser(String pattern) { + public static Format getTimeParser(String pattern) { + return getTimeParser(pattern, DEFAULT_TIME_ZONE); + } + + public static Format getTimestampParser(String pattern, TimeZone timeZone) { SimpleDateFormat format = new SimpleDateFormat(pattern) { @Override public java.util.Date parseObject(String source) throws ParseException { @@ -77,12 +88,18 @@ public class DateUtil { return new java.sql.Timestamp(date.getTime()); } }; - format.setTimeZone(DateUtil.DATE_TIME_ZONE); + format.setTimeZone(timeZone); return format; } + public static Format getTimestampParser(String pattern) { + return getTimestampParser(pattern, DEFAULT_TIME_ZONE); + } + public static Format getDateFormatter(String pattern) { - return DateUtil.DEFAULT_DATE_FORMAT.equals(pattern) ? DateUtil.DEFAULT_DATE_FORMATTER : FastDateFormat.getInstance(pattern, DateUtil.DATE_TIME_ZONE); + return DateUtil.DEFAULT_DATE_FORMAT.equals(pattern) + ? DateUtil.DEFAULT_DATE_FORMATTER + : FastDateFormat.getInstance(pattern, DateUtil.DEFAULT_TIME_ZONE); } private static ThreadLocal dateFormat = http://git-wip-us.apache.org/repos/asf/phoenix/blob/5911b4af/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java index 2c532aa..d7f4172 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java @@ -26,6 +26,7 @@ import java.sql.Date; import java.util.Calendar; import java.util.GregorianCalendar; import java.util.List; +import java.util.TimeZone; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; @@ -322,7 +323,7 @@ public class SortOrderExpressionTest { cal.set(Calendar.MINUTE, 0); cal.set(Calendar.SECOND, 0); cal.set(Calendar.MILLISECOND, 0); - cal.setTimeZone(DateUtil.DATE_TIME_ZONE); + cal.setTimeZone(TimeZone.getTimeZone(DateUtil.DEFAULT_TIME_ZONE_ID)); Date d = new Date(cal.getTimeInMillis()); return d; } http://git-wip-us.apache.org/repos/asf/phoenix/blob/5911b4af/phoenix-core/src/test/java/org/apache/phoenix/util/DateUtilTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/DateUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/DateUtilTest.java index 7746515..36acf70 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/util/DateUtilTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/util/DateUtilTest.java @@ -17,22 +17,27 @@ */ package org.apache.phoenix.util; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - +import java.sql.Date; +import java.sql.Time; import java.sql.Timestamp; +import java.text.ParseException; +import java.util.TimeZone; import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + /** - * * Test class for {@link DateUtil} * - * * @since 2.1.3 */ public class DateUtilTest { + private static final long ONE_HOUR_IN_MILLIS = 1000L * 60L * 60L; + @Test public void testDemonstrateSetNanosOnTimestampLosingMillis() { Timestamp ts1 = new Timestamp(120055); @@ -59,4 +64,67 @@ public class DateUtilTest { assertFalse(ts1.equals(ts2)); assertTrue(ts2.after(ts1)); } + + @Test + public void testGetDateParser_DefaultTimeZone() throws ParseException { + Date date = (Date) DateUtil.getDateParser("yyyy-MM-dd").parseObject("1970-01-01"); + assertEquals(0, date.getTime()); + } + + @Test + public void testGetDateParser_CustomTimeZone() throws ParseException { + Date date = (Date) DateUtil.getDateParser( + "yyyy-MM-dd", TimeZone.getTimeZone("GMT+1")).parseObject("1970-01-01"); + assertEquals(-ONE_HOUR_IN_MILLIS, date.getTime()); + } + + @Test + public void testGetDateParser_LocalTimeZone() throws ParseException { + Date date = (Date) DateUtil.getDateParser( + "yyyy-MM-dd", TimeZone.getDefault()).parseObject("1970-01-01"); + assertEquals(Date.valueOf("1970-01-01"), date); + } + + @Test + public void testGetTimestampParser_DefaultTimeZone() throws ParseException { + Timestamp ts = (Timestamp) DateUtil.getTimestampParser("yyyy-MM-dd HH:mm:ss") + .parseObject("1970-01-01 00:00:00"); + assertEquals(0, ts.getTime()); + } + + @Test + public void testGetTimestampParser_CustomTimeZone() throws ParseException { + Timestamp ts = (Timestamp) DateUtil.getTimestampParser("yyyy-MM-dd HH:mm:ss", TimeZone.getTimeZone("GMT+1")) + .parseObject("1970-01-01 00:00:00"); + assertEquals(-ONE_HOUR_IN_MILLIS, ts.getTime()); + } + + @Test + public void testGetTimestampParser_LocalTimeZone() throws ParseException { + Timestamp ts = (Timestamp) DateUtil.getTimestampParser( + "yyyy-MM-dd HH:mm:ss", + TimeZone.getDefault()).parseObject("1970-01-01 00:00:00"); + assertEquals(Timestamp.valueOf("1970-01-01 00:00:00"), ts); + } + + @Test + public void testGetTimeParser_DefaultTimeZone() throws ParseException { + Time time = (Time) DateUtil.getTimeParser("HH:mm:ss").parseObject("00:00:00"); + assertEquals(0, time.getTime()); + } + + @Test + public void testGetTimeParser_CustomTimeZone() throws ParseException { + Time time = (Time) DateUtil.getTimeParser( + "HH:mm:ss", + TimeZone.getTimeZone("GMT+1")).parseObject("00:00:00"); + assertEquals(-ONE_HOUR_IN_MILLIS, time.getTime()); + } + + @Test + public void testGetTimeParser_LocalTimeZone() throws ParseException { + Time time = (Time) DateUtil.getTimeParser( + "HH:mm:ss", TimeZone.getDefault()).parseObject("00:00:00"); + assertEquals(Time.valueOf("00:00:00"), time); + } }