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 2019/11/19 03:25:09 UTC

[GitHub] [hive] jcamachor commented on a change in pull request #845: HIVE-22369 Handle HiveTableFunctionScan at return path

jcamachor commented on a change in pull request #845: HIVE-22369 Handle HiveTableFunctionScan at return path
URL: https://github.com/apache/hive/pull/845#discussion_r347714603
 
 

 ##########
 File path: ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
 ##########
 @@ -186,12 +193,67 @@ OpAttr dispatch(RelNode rn) throws SemanticException {
       return visit((HiveSortExchange) rn);
     } else if (rn instanceof HiveAggregate) {
       return visit((HiveAggregate) rn);
+    } else if (rn instanceof HiveTableFunctionScan) {
+      return visit((HiveTableFunctionScan) rn);
     }
     LOG.error(rn.getClass().getCanonicalName() + "operator translation not supported"
         + " yet in return path.");
     return null;
   }
 
+  private OpAttr visit(HiveTableFunctionScan scanRel) throws SemanticException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Translating operator rel#" + scanRel.getId() + ":"
+          + scanRel.getRelTypeName() + " with row type: [" + scanRel.getRowType() + "]");
+    }
+
+    RexCall call = (RexCall)scanRel.getCall();
+
+    String functionName = call.getOperator().getName();
+    FunctionInfo fi = FunctionRegistry.getFunctionInfo(functionName);
+    GenericUDTF genericUDTF = fi.getGenericUDTF();
+
+    RowResolver rowResolver = new RowResolver();
+    List<String> fieldNames = new ArrayList<>(scanRel.getRowType().getFieldNames());
+    List<String> exprNames = new ArrayList<>(fieldNames);
+    List<ExprNodeDesc> exprCols = new ArrayList<>();
+    Map<String, ExprNodeDesc> colExprMap = new HashMap<>();
+    for (int pos = 0; pos < call.getOperands().size(); pos++) {
+      ExprNodeConverter converter = new ExprNodeConverter(SemanticAnalyzer.DUMMY_TABLE, fieldNames.get(pos),
+          scanRel.getRowType(), scanRel.getRowType(), ((HiveTableScan)scanRel.getInput(0)).getPartOrVirtualCols(),
+          scanRel.getCluster().getTypeFactory(), true);
+      ExprNodeDesc exprCol = call.getOperands().get(pos).accept(converter);
+      colExprMap.put(exprNames.get(pos), exprCol);
+      exprCols.add(exprCol);
+
+      ColumnInfo columnInfo = new ColumnInfo(fieldNames.get(pos), exprCol.getWritableObjectInspector(), null, false);
+      rowResolver.put(columnInfo.getTabAlias(), columnInfo.getAlias(), columnInfo);
+    }
+
+    QB qb = new QB(semanticAnalyzer.getQB().getId(), nextAlias(), true);
+    qb.getMetaData().setSrcForAlias(SemanticAnalyzer.DUMMY_TABLE, semanticAnalyzer.getDummyTable());
+    TableScanOperator op = (TableScanOperator) semanticAnalyzer.genTablePlan(SemanticAnalyzer.DUMMY_TABLE, qb);
+    op.getConf().setRowLimit(1);
+    qb.addAlias(SemanticAnalyzer.DUMMY_TABLE);
+    qb.setTabAlias(SemanticAnalyzer.DUMMY_TABLE, SemanticAnalyzer.DUMMY_TABLE);
+
+    Operator<?> output = OperatorFactory.getAndMakeChild(new SelectDesc(exprCols, fieldNames, false),
+        new RowSchema(rowResolver.getRowSchema()), op);
+    output.setColumnExprMap(colExprMap);
+    semanticAnalyzer.putOpInsertMap(output, rowResolver);
+
+    Operator<?> funcOp = semanticAnalyzer.genUDTFPlan(genericUDTF, null, fieldNames, qb, output, false);
 
 Review comment:
   Should we move this method to `HiveOpConverter` and clean any context information that is not needed? Similar to `genReduceSink`. Idea is that when we enable this path, we can easily get rid of most dependencies from `SemanticAnalyzer`.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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