private FetchWork convertToWork() throws HiveException { inputs.clear(); Utilities.addSchemaEvolutionToTableScanOperator(table, scanOp); TableDesc tableDesc = Utilities.getTableDesc(table); if (!table.isPartitioned()) { inputs.add(new ReadEntity(table, parent, !table.isView() && parent == null)); FetchWork work = new FetchWork(table.getPath(), tableDesc); PlanUtils.configureInputJobPropertiesForStorageHandler(work.getTblDesc()); work.setSplitSample(splitSample); return work; } List<Path> listP = new ArrayList<Path>(); List<PartitionDesc> partP = new ArrayList<PartitionDesc>(); for (Partition partition : partsList.getNotDeniedPartns()) { inputs.add(new ReadEntity(partition, parent, parent == null)); listP.add(partition.getDataLocation()); partP.add(Utilities.getPartitionDescFromTableDesc(tableDesc, partition, true)); } Table sourceTable = partsList.getSourceTable(); inputs.add(new ReadEntity(sourceTable, parent, parent == null)); TableDesc table = Utilities.getTableDesc(sourceTable); FetchWork work = new FetchWork(listP, partP, table); if (!work.getPartDesc().isEmpty()) { PartitionDesc part0 = work.getPartDesc().get(0); PlanUtils.configureInputJobPropertiesForStorageHandler(part0.getTableDesc()); work.setSplitSample(splitSample); } return work; }
public FetchWork getFetchWork() { // FetchWork's sink is used to hold results, so each query needs a separate copy of FetchWork FetchWork fetch = new FetchWork(cachedResultsPath, fetchWork.getTblDesc(), fetchWork.getLimit()); fetch.setCachedResult(true); return fetch; }
private void initialize() throws HiveException { if (isStatReader) { outputOI = work.getStatRowOI(); return; iterPath = work.getPartDir().iterator(); iterPartDesc = work.getPartDesc().iterator(); } else { iterPath = Arrays.asList(work.getTblDir()).iterator(); iterPartDesc = Iterators.cycle(new PartitionDesc(work.getTblDesc(), null)); context = setupExecContext(operator, work.getPathLists());
public FetchOperator(FetchWork work, JobConf job, Operator<?> operator, List<VirtualColumn> vcCols) throws HiveException { this.job = job; this.work = work; this.operator = operator; if (operator instanceof TableScanOperator) { Utilities.addTableSchemaToConf(job, (TableScanOperator) operator); } this.vcCols = vcCols; this.hasVC = vcCols != null && !vcCols.isEmpty(); this.isStatReader = work.getTblDesc() == null; this.isPartitioned = !isStatReader && work.isPartitioned(); this.isNonNativeTable = !isStatReader && work.getTblDesc().isNonNative(); initialize(); }
/** * used for bucket map join */ public void setupContext(List<Path> paths) { this.iterPath = paths.iterator(); List<PartitionDesc> partitionDescs; if (!isPartitioned) { this.iterPartDesc = Iterators.cycle(new PartitionDesc(work.getTblDesc(), null)); } else { this.iterPartDesc = work.getPartDescs(paths).iterator(); } this.context = setupExecContext(operator, paths); }
return; pCtx.getFetchTask().getWork().setHiveServerQuery(SessionState.get().isHiveServerQuery()); TableDesc resultTab = pCtx.getFetchTask().getTblDesc(); FetchWork fetch = new FetchWork(loadFileDesc.getSourcePath(), resultTab, outerQueryLimit); boolean isHiveServerQuery = SessionState.get().isHiveServerQuery(); fetch.setHiveServerQuery(isHiveServerQuery); fetch.setSource(pCtx.getFetchSource()); fetch.setSink(pCtx.getFetchSink()); if (isHiveServerQuery && null != resultTab && resultTab.getSerdeClassName().equalsIgnoreCase(ThriftJDBCBinarySerDe.class.getName()) && HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_SERVER2_THRIFT_RESULTSET_SERIALIZE_IN_TASKS)) { fetch.setIsUsingThriftJDBCBinarySerDe(true); } else { fetch.setIsUsingThriftJDBCBinarySerDe(false); pCtx.getFetchTask().getWork().setLeastNumRows(globalLimitCtx.getGlobalLimit());
private void getNextPath() throws Exception { if (work.getTblDir() != null) { if (!tblDataDone) { currPath = work.getTblDirPath(); currTbl = work.getTblDesc(); if (isNativeTable) { FileSystem fs = currPath.getFileSystem(job); iterPath = FetchWork.convertStringToPathArray(work.getPartDir()).iterator(); iterPartDesc = work.getPartDesc().iterator();
if (!fTask.getWork().isNotPartitioned() && fTask.getWork().getLimit() == -1 && scanLimit < fTask.getWork().getPartDir().size()) { throw new SemanticException(ErrorMsg.PARTITION_SCAN_LIMIT_EXCEEDED, "" + fTask.getWork().getPartDir().size(), "" + fTask.getWork().getTblDesc().getTableName(), "" + scanLimit);
/** * Create a FetchTask for a given schema. * * @param schema string */ protected FetchTask createFetchTask(String schema) { Properties prop = new Properties(); // Sets delimiter to tab (ascii 9) prop.setProperty(serdeConstants.SERIALIZATION_FORMAT, Integer.toString(Utilities.tabCode)); prop.setProperty(serdeConstants.SERIALIZATION_NULL_FORMAT, " "); String[] colTypes = schema.split("#"); prop.setProperty("columns", colTypes[0]); prop.setProperty("columns.types", colTypes[1]); prop.setProperty(serdeConstants.SERIALIZATION_LIB, LazySimpleSerDe.class.getName()); FetchWork fetch = new FetchWork(ctx.getResFile(), new TableDesc(TextInputFormat.class, IgnoreKeyTextOutputFormat.class, prop), -1); fetch.setSerializationNullFormat(" "); return (FetchTask) TaskFactory.get(fetch); }
@SuppressWarnings("unchecked") private FetchTask optimize(ParseContext pctx, String alias, TableScanOperator source) throws Exception { String mode = HiveConf.getVar( pctx.getConf(), HiveConf.ConfVars.HIVEFETCHTASKCONVERSION); boolean aggressive = "more".equals(mode); final int limit = pctx.getQueryProperties().getOuterQueryLimit(); // limit = 0 means that we do not need any task. if (limit == 0) { return null; } FetchData fetch = checkTree(aggressive, pctx, alias, source); if (fetch != null && checkThreshold(fetch, limit, pctx)) { FetchWork fetchWork = fetch.convertToWork(); FetchTask fetchTask = (FetchTask) TaskFactory.get(fetchWork); fetchWork.setSink(fetch.completed(pctx, fetchWork)); fetchWork.setSource(source); fetchWork.setLimit(limit); return fetchTask; } return null; }
if (fTask != null) { fWork = fTask.getWork(); fWork.getRowsComputedUsingStats().addAll(allRows); } else { StandardStructObjectInspector sOI = ObjectInspectorFactory. getStandardStructObjectInspector(colNames, ois); fWork = new FetchWork(allRows, sOI); fTask = (FetchTask) TaskFactory.get(fWork); pctx.setFetchTask(fTask); fWork.setLimit(fWork.getRowsComputedUsingStats().size()); isOptimized = true; return null;
/** * used for bucket map join. there is a hack for getting partitionDesc. bucket map join right now * only allow one partition present in bucket map join. */ public void setupContext(Iterator<Path> iterPath, Iterator<PartitionDesc> iterPartDesc) { this.iterPath = iterPath; this.iterPartDesc = iterPartDesc; if (iterPartDesc == null) { if (work.getTblDir() != null) { this.currTbl = work.getTblDesc(); } else { // hack, get the first. List<PartitionDesc> listParts = work.getPartDesc(); currPart = listParts.get(0); } } }
boolean partitionedTable = fetchWork.isPartitioned(); List<Path> directories; if (partitionedTable) { LOG.info("Printing orc file dump for files from partitioned directory.."); directories = fetchWork.getPartDir(); } else { LOG.info("Printing orc file dump for files from table directory.."); directories = Lists.newArrayList(); directories.add(fetchWork.getTblDir());
FetchWork fetch = new FetchWork(loadFileDesc.getSourcePath(), resultTab, outerQueryLimit); fetch.setSource(pCtx.getFetchSource()); fetch.setSink(pCtx.getFetchSink()); pCtx.getFetchTask().getWork().setLeastNumRows(globalLimitCtx.getGlobalLimit());
private StructObjectInspector setupOutputObjectInspector() throws HiveException { TableDesc tableDesc = work.getTblDesc(); try { tableSerDe = tableDesc.getDeserializer(job, true); tableOI = (StructObjectInspector) tableSerDe.getObjectInspector(); if (!isPartitioned) { return getTableRowOI(tableOI); } partKeyOI = getPartitionKeyOI(tableDesc); PartitionDesc partDesc = new PartitionDesc(tableDesc, null); List<PartitionDesc> listParts = work.getPartDesc(); // Chose the table descriptor if none of the partitions is present. // For eg: consider the query: // select /*+mapjoin(T1)*/ count(*) from T1 join T2 on T1.key=T2.key // Both T1 and T2 and partitioned tables, but T1 does not have any partitions // FetchOperator is invoked for T1, and listParts is empty. In that case, // use T1's schema to get the ObjectInspector. if (listParts == null || listParts.isEmpty() || !needConversion(tableDesc, listParts)) { return getPartitionedRowOI(tableOI); } convertedOI = (StructObjectInspector) ObjectInspectorConverters.getConvertedOI( tableOI, tableOI, null, false); return getPartitionedRowOI(convertedOI); } catch (Exception e) { throw new HiveException("Failed with exception " + e.getMessage() + StringUtils.stringifyException(e)); } }
if (fetchWork.getTblDir() != null) { currWork.mergeAliasedInput(alias, fetchWork.getTblDir(), partitionInfo); } else { for (Path pathDir : fetchWork.getPartDir()) { currWork.mergeAliasedInput(alias, pathDir, partitionInfo);
@Override protected void localizeMRTmpFilesImpl(Context ctx) { String s = work.getTblDir(); if ((s != null) && ctx.isMRTmpFileURI(s)) { work.setTblDir(ctx.localizeMRTmpFileURI(s)); } ArrayList<String> ls = work.getPartDir(); if (ls != null) { ctx.localizePaths(ls); } }