You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2020/07/09 14:02:50 UTC

[GitHub] [hive] belugabehr commented on a change in pull request #1125: HIVE-23244 Extract Create View analyzer from SemanticAnalyzer (Miklos Gergely)

belugabehr commented on a change in pull request #1125:
URL: https://github.com/apache/hive/pull/1125#discussion_r452239123



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AbstractCreateViewAnalyzer.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.hadoop.hive.ql.ddl.view.create;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.PlanUtils;
+
+/**
+ * Abstract ancestor of analyzers that can create a view.
+ */
+abstract class AbstractCreateViewAnalyzer extends BaseSemanticAnalyzer {
+  AbstractCreateViewAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  protected List<FieldSchema> schema;
+
+  @Override
+  public List<FieldSchema> getResultSchema() {
+    return schema;
+  }
+
+  protected SemanticAnalyzer analyzeQuery(ASTNode select, String fqViewName) throws SemanticException {
+    QueryState innerQueryState = new QueryState.Builder().withHiveConf(conf).build();
+    innerQueryState.getConf().setBoolVar(HiveConf.ConfVars.HIVE_RESULTSET_USE_UNIQUE_COLUMN_NAMES, false);
+
+    SemanticAnalyzer analyzer = (SemanticAnalyzer) SemanticAnalyzerFactory.get(innerQueryState, select);
+    ctx.setEnableUnparse(true);
+    analyzer.forViewCreation(fqViewName);
+    analyzer.analyze(select, ctx);
+    analyzer.executeUnparseTranlations();
+
+    queryState.setLineageState(innerQueryState.getLineageState());
+    queryState.getLineageState().mapDirToOp(new Path(fqViewName), analyzer.getSinkOp());
+
+    addInputs(analyzer);
+
+    return analyzer;
+  }
+
+  private void addInputs(SemanticAnalyzer analyzer) {
+    inputs.addAll(analyzer.getInputs());
+    for (Map.Entry<String, TableScanOperator> entry : analyzer.getTopOps().entrySet()) {
+      String alias = entry.getKey();
+      TableScanOperator topOp = entry.getValue();
+      ReadEntity parentViewInfo = PlanUtils.getParentViewInfo(alias, analyzer.getViewAliasToInput());
+
+      // Adds tables only for create view (PPD filter can be appended by outer query)
+      Table table = topOp.getConf().getTableMetadata();
+      PlanUtils.addInput(inputs, new ReadEntity(table, parentViewInfo));
+    }
+  }
+
+  protected void validateTablesUsed(SemanticAnalyzer analyzer) throws SemanticException {
+    // Do not allow view to be defined on temp table or other materialized view
+    Set<String> tableAliases = analyzer.getQB().getTabAliases();
+    for (String alias : tableAliases) {
+      try {
+        if (SemanticAnalyzer.DUMMY_TABLE.equals(alias)) {
+          continue;
+        }
+        Table table = analyzer.getTableObjectByName(analyzer.getQB().getTabNameForAlias(alias));
+        if (table.isTemporary()) {
+          throw new SemanticException("View definition references temporary table " + alias);
+        }
+        if (table.isMaterializedView()) {
+          throw new SemanticException("View definition references materialized view " + alias);
+        }
+      } catch (HiveException ex) {
+        throw new SemanticException(ex);
+      }

Review comment:
       Try laying this out a bit differently.  It's not proper form to catch your own exception: `SemanticException` is subclass of `HiveException`

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AbstractCreateViewAnalyzer.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.hadoop.hive.ql.ddl.view.create;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzerFactory;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.PlanUtils;
+
+/**
+ * Abstract ancestor of analyzers that can create a view.
+ */
+abstract class AbstractCreateViewAnalyzer extends BaseSemanticAnalyzer {
+  AbstractCreateViewAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  protected List<FieldSchema> schema;
+
+  @Override
+  public List<FieldSchema> getResultSchema() {
+    return schema;
+  }
+
+  protected SemanticAnalyzer analyzeQuery(ASTNode select, String fqViewName) throws SemanticException {
+    QueryState innerQueryState = new QueryState.Builder().withHiveConf(conf).build();
+    innerQueryState.getConf().setBoolVar(HiveConf.ConfVars.HIVE_RESULTSET_USE_UNIQUE_COLUMN_NAMES, false);
+
+    SemanticAnalyzer analyzer = (SemanticAnalyzer) SemanticAnalyzerFactory.get(innerQueryState, select);
+    ctx.setEnableUnparse(true);
+    analyzer.forViewCreation(fqViewName);
+    analyzer.analyze(select, ctx);
+    analyzer.executeUnparseTranlations();
+
+    queryState.setLineageState(innerQueryState.getLineageState());
+    queryState.getLineageState().mapDirToOp(new Path(fqViewName), analyzer.getSinkOp());
+
+    addInputs(analyzer);
+
+    return analyzer;
+  }
+
+  private void addInputs(SemanticAnalyzer analyzer) {
+    inputs.addAll(analyzer.getInputs());
+    for (Map.Entry<String, TableScanOperator> entry : analyzer.getTopOps().entrySet()) {
+      String alias = entry.getKey();
+      TableScanOperator topOp = entry.getValue();
+      ReadEntity parentViewInfo = PlanUtils.getParentViewInfo(alias, analyzer.getViewAliasToInput());
+
+      // Adds tables only for create view (PPD filter can be appended by outer query)
+      Table table = topOp.getConf().getTableMetadata();
+      PlanUtils.addInput(inputs, new ReadEntity(table, parentViewInfo));
+    }
+  }
+
+  protected void validateTablesUsed(SemanticAnalyzer analyzer) throws SemanticException {
+    // Do not allow view to be defined on temp table or other materialized view
+    Set<String> tableAliases = analyzer.getQB().getTabAliases();
+    for (String alias : tableAliases) {
+      try {
+        if (SemanticAnalyzer.DUMMY_TABLE.equals(alias)) {
+          continue;
+        }
+        Table table = analyzer.getTableObjectByName(analyzer.getQB().getTabNameForAlias(alias));
+        if (table.isTemporary()) {
+          throw new SemanticException("View definition references temporary table " + alias);
+        }
+        if (table.isMaterializedView()) {
+          throw new SemanticException("View definition references materialized view " + alias);
+        }
+      } catch (HiveException ex) {
+        throw new SemanticException(ex);
+      }
+    }
+  }
+
+  protected void validateReplaceWithPartitions(String viewName, Table oldView, List<FieldSchema> partitionColumns)
+      throws SemanticException {
+    String partitionViewErrorMsg = "The following view has partition, it could not be replaced: " + viewName;
+    try {
+      if (!oldView.getPartCols().isEmpty() && !db.getPartitions(oldView).isEmpty() &&
+          !oldView.getPartCols().equals(partitionColumns)) {
+        throw new SemanticException(ErrorMsg.REPLACE_VIEW_WITH_PARTITION.getMsg(partitionViewErrorMsg));
+      }
+    } catch (HiveException e) {
+      throw new SemanticException(ErrorMsg.REPLACE_VIEW_WITH_PARTITION.getMsg(partitionViewErrorMsg));

Review comment:
       Again. this `SemanticException` will be double-wrapped

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/ddl/view/create/AlterViewAsAnalyzer.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.hadoop.hive.ql.ddl.view.create;
+
+import java.util.ArrayList;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.ddl.DDLWork;
+import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory.DDLType;
+import org.apache.hadoop.hive.ql.ddl.DDLUtils;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.ParseUtils;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+/**
+ * Analyzer for alter view ... as commands.
+ */
+@DDLType(types = HiveParser.TOK_ALTERVIEW_AS)
+public class AlterViewAsAnalyzer extends AbstractCreateViewAnalyzer {
+  public AlterViewAsAnalyzer(QueryState queryState) throws SemanticException {
+    super(queryState);
+  }
+
+  @Override
+  public void analyzeInternal(ASTNode root) throws SemanticException {
+    TableName viewName = getQualifiedTableName((ASTNode) root.getChild(0));
+    String fqViewName = viewName.getNotEmptyDbTable();
+    LOG.info("Altering the query of view " + fqViewName + " position=" + root.getCharPositionInLine());
+
+    ASTNode select = (ASTNode) root.getChild(1).getChild(0);
+
+    String originalText = ctx.getTokenRewriteStream().toString(select.getTokenStartIndex(), select.getTokenStopIndex());
+
+    SemanticAnalyzer analyzer = analyzeQuery(select, fqViewName);
+
+    schema = new ArrayList<FieldSchema>(analyzer.getResultSchema());
+    ParseUtils.validateColumnNameUniqueness(
+        analyzer.getOriginalResultSchema() == null ? schema : analyzer.getOriginalResultSchema());
+
+    String expandedText = ctx.getTokenRewriteStream().toString(select.getTokenStartIndex(), select.getTokenStopIndex());
+
+    AlterViewAsDesc desc = new AlterViewAsDesc(fqViewName, schema, originalText, expandedText);
+    validateCreateView(desc, analyzer);
+
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc)));
+    DDLUtils.addDbAndTableToOutputs(getDatabase(viewName.getDb()), viewName, TableType.VIRTUAL_VIEW, false,
+        null, outputs);
+  }
+
+  private void validateCreateView(AlterViewAsDesc desc, SemanticAnalyzer analyzer) throws SemanticException {
+    try {
+      validateTablesUsed(analyzer);
+
+      //replace view
+      Table oldView = getTable(desc.getViewName(), false);
+      if (oldView == null) {
+        String viewNotExistErrorMsg = "The following view does not exist: " + desc.getViewName();
+        throw new SemanticException(
+            ErrorMsg.ALTER_VIEW_AS_SELECT_NOT_EXIST.getMsg(viewNotExistErrorMsg));

Review comment:
       Catching your own exception




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org