Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 40236200BFB for ; Tue, 27 Dec 2016 19:29:30 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 3F12C160B1D; Tue, 27 Dec 2016 18:29:30 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 002EB160B3D for ; Tue, 27 Dec 2016 19:29:28 +0100 (CET) Received: (qmail 84011 invoked by uid 500); 27 Dec 2016 18:29:27 -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 82242 invoked by uid 99); 27 Dec 2016 18:29:25 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 27 Dec 2016 18:29:25 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 6F0C3F15AB; Tue, 27 Dec 2016 18:29:25 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: maryannxue@apache.org To: commits@phoenix.apache.org Date: Tue, 27 Dec 2016 18:30:12 -0000 Message-Id: <1623afe872254137a0006f624ca46cce@git.apache.org> In-Reply-To: <0bb397d37805426ab854e837c59be7ea@git.apache.org> References: <0bb397d37805426ab854e837c59be7ea@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [49/50] [abbrv] phoenix git commit: Fix sync conflicts archived-at: Tue, 27 Dec 2016 18:29:30 -0000 http://git-wip-us.apache.org/repos/asf/phoenix/blob/c67c7ba0/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateTable.java ---------------------------------------------------------------------- diff --cc phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateTable.java index dc4c5a7,0000000..6b72d0a mode 100644,000000..100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateTable.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/parse/SqlCreateTable.java @@@ -1,133 -1,0 +1,138 @@@ +/* + * 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.calcite.parse; + +import com.google.common.collect.ImmutableList; + +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlLiteral; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; +import java.util.List; + +/** + * Parse tree node for SQL {@code CREATE TABLE} command. + */ +public class SqlCreateTable extends SqlCall { + public final SqlOperator operator; + + public final SqlIdentifier tableName; ++ public final SqlLiteral immutable; + public final SqlLiteral ifNotExists; + public final SqlNodeList columnDefs; + public final SqlIdentifier pkConstraint; + public final SqlNodeList pkConstraintColumnDefs; + public final SqlIdentifier baseTableName; + public final SqlNode whereNode; + public final SqlNodeList tableOptions; + public final SqlNodeList splitKeyList; + + + /** Creates a CREATE TABLE. */ + public SqlCreateTable( + SqlParserPos pos, + SqlIdentifier tableName, ++ SqlLiteral immutable, + SqlLiteral ifNotExists, + SqlNodeList columnDefs, + SqlIdentifier pkConstraint, + SqlNodeList pkConstraintColumnDefs, + SqlNodeList tableOptions, + SqlNodeList splitKeyList) { + super(pos); + this.operator = new SqlDdlOperator("CREATE TABLE", SqlKind.CREATE_TABLE); + this.tableName = tableName; ++ this.immutable = immutable; + this.ifNotExists = ifNotExists; + this.columnDefs = columnDefs; + this.pkConstraint = pkConstraint; + this.pkConstraintColumnDefs = pkConstraintColumnDefs; + this.baseTableName = null; + this.whereNode = null; + this.tableOptions = tableOptions; + this.splitKeyList = splitKeyList; + } + + /** Creates a CREATE VIEW. */ + public SqlCreateTable( + SqlParserPos pos, + SqlIdentifier tableName, ++ SqlLiteral immutable, + SqlLiteral ifNotExists, + SqlNodeList columnDefs, + SqlIdentifier baseTableName, + SqlNode whereNode, + SqlNodeList tableOptions) { + super(pos); + this.operator = new SqlDdlOperator("CREATE VIEW", SqlKind.CREATE_VIEW); + this.tableName = tableName; ++ this.immutable = immutable; + this.ifNotExists = ifNotExists; + this.columnDefs = columnDefs; + this.pkConstraint = null; + this.pkConstraintColumnDefs = SqlNodeList.EMPTY; + this.baseTableName = baseTableName; + this.whereNode = whereNode; + this.tableOptions = tableOptions; + this.splitKeyList = SqlNodeList.EMPTY; + } + + public SqlOperator getOperator() { + return operator; + } + + public List getOperandList() { + return ImmutableList.of(tableName, ifNotExists, columnDefs, pkConstraint, pkConstraintColumnDefs, baseTableName, whereNode, tableOptions, splitKeyList); + } + + @Override + public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { + writer.keyword(operator.getName()); + if(SqlLiteral.value(ifNotExists).equals(Boolean.TRUE)) { + writer.keyword("IF NOT EXISTS"); + } + tableName.unparse(writer, 0, 0); + ((SqlDdlOperator)getOperator()).unparseListClause(writer, columnDefs); + if(pkConstraint != null) { + writer.keyword("CONSTRANT"); + pkConstraint.unparse(writer, 0, 0); + writer.keyword("PRIMARY KEY"); + ((SqlDdlOperator)getOperator()).unparseListClause(writer, pkConstraintColumnDefs); + } + if (baseTableName != null) { + writer.keyword("AS SELECT * FROM"); + baseTableName.unparse(writer, 0, 0); + if (whereNode != null) { + writer.keyword("WHERE"); + whereNode.unparse(writer, 0, 0); + } + } + if (SqlNodeList.isEmptyList(tableOptions)) { + ((SqlDdlOperator)getOperator()).unparseListClause(writer, tableOptions); + } + if (SqlNodeList.isEmptyList(splitKeyList)) { + writer.keyword("SPLIT ON"); + ((SqlDdlOperator)getOperator()).unparseListClause(writer, splitKeyList); + } + } +} http://git-wip-us.apache.org/repos/asf/phoenix/blob/c67c7ba0/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/c67c7ba0/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/c67c7ba0/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/c67c7ba0/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java ---------------------------------------------------------------------- diff --cc phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java index f4afc19,8ef1f8d..7b43c11 --- a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java @@@ -69,21 -69,11 +70,21 @@@ import com.google.common.collect.Lists public class ClientAggregatePlan extends ClientProcessingPlan { private final GroupBy groupBy; private final Expression having; - private final Aggregators serverAggregators; - private final Aggregators clientAggregators; - + private final ServerAggregators serverAggregators; + private final ClientAggregators clientAggregators; + public ClientAggregatePlan(StatementContext context, FilterableStatement statement, TableRef table, RowProjector projector, Integer limit, Integer offset, Expression where, OrderBy orderBy, GroupBy groupBy, Expression having, QueryPlan delegate) { + this(context, statement, table, projector, limit, offset, where, orderBy, groupBy, having, delegate, + ServerAggregators.deserialize( + context.getScan().getAttribute(BaseScannerRegionObserver.AGGREGATORS), + QueryServicesOptions.withDefaults().getConfiguration()), + context.getAggregationManager().getAggregators()); + } + + private ClientAggregatePlan(StatementContext context, FilterableStatement statement, TableRef table, RowProjector projector, + Integer limit, Integer offset, Expression where, OrderBy orderBy, GroupBy groupBy, Expression having, QueryPlan delegate, + Aggregators serverAggregators, Aggregators clientAggregators) { super(context, statement, table, projector, limit, offset, where, orderBy, delegate); this.groupBy = groupBy; this.having = having; http://git-wip-us.apache.org/repos/asf/phoenix/blob/c67c7ba0/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/c67c7ba0/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/c67c7ba0/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/c67c7ba0/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/c67c7ba0/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java ---------------------------------------------------------------------- diff --cc phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java index 1afff27,d3886db..7441bec --- a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java @@@ -342,51 -339,18 +339,24 @@@ public final class QueryUtil public static String getConnectionUrl(Properties props, Configuration conf) throws ClassNotFoundException, SQLException { - return getConnectionUrl(props, conf, null); + return getConnectionUrl(props, conf, null, false); } + public static String getConnectionUrl(Properties props, Configuration conf, String principal) throws ClassNotFoundException, SQLException { + return getConnectionUrl(props, conf, principal, false); + } + + public static String getConnectionUrl(Properties props, Configuration conf, String principal, boolean isCalciteEnabled) + throws ClassNotFoundException, SQLException { - // TODO: props is ignored! // read the hbase properties from the configuration - String server = ZKConfig.getZKQuorumServersString(conf); - // could be a comma-separated list - String[] rawServers = server.split(","); - List servers = new ArrayList(rawServers.length); - int port = -1; - for (String serverPort : rawServers) { - try { - server = Addressing.parseHostname(serverPort); - int specifiedPort = Addressing.parsePort(serverPort); - // there was a previously specified port and it doesn't match this server - if (port > 0 && specifiedPort != port) { - throw new IllegalStateException("Phoenix/HBase only supports connecting to a " + - "single zookeeper client port. Specify servers only as host names in " + - "HBase configuration"); - } - // set the port to the specified port - port = specifiedPort; - servers.add(server); - } catch (IllegalArgumentException e) { - } - } - // port wasn't set, shouldn't ever happen from HBase, but just in case - if (port == -1) { - port = conf.getInt(QueryServices.ZOOKEEPER_PORT_ATTRIB, -1); - if (port == -1) { - // TODO: fall back to the default in HConstants#DEFAULT_ZOOKEPER_CLIENT_PORT - throw new RuntimeException("Client zk port was not set!"); - } - } - server = Joiner.on(',').join(servers); + int port = conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT); + // Build the ZK quorum server string with "server:clientport" list, separated by ',' + final String server = + conf.get(HConstants.ZOOKEEPER_QUORUM, HConstants.LOCALHOST); String znodeParent = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); - String url = getUrl(server, port, znodeParent, principal); + String url = getUrl(server, port, znodeParent, principal, isCalciteEnabled); // Mainly for testing to tack on the test=true part to ensure driver is found on server String extraArgs = props.getProperty(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, conf.get(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS)); if (extraArgs.length() > 0) { http://git-wip-us.apache.org/repos/asf/phoenix/blob/c67c7ba0/pom.xml ----------------------------------------------------------------------