@Test public void testGetInputSummaryPoolAndFailure() throws ExecutionException, InterruptedException, IOException { ExecutorService pool = mock(ExecutorService.class); when(pool.submit(any(Runnable.class))).thenReturn(mock(Future.class)); Set<Path> pathNeedProcess = new HashSet<>(); pathNeedProcess.add(new Path("dummy-path1")); pathNeedProcess.add(new Path("dummy-path2")); pathNeedProcess.add(new Path("dummy-path3")); SessionState.start(new HiveConf()); JobConf jobConf = new JobConf(); Context context = new Context(jobConf); Utilities.getInputSummaryWithPool(context, pathNeedProcess, mock(MapWork.class), new long[3], pool); verify(pool, times(3)).submit(any(Runnable.class)); verify(pool).shutdown(); verify(pool).shutdownNow(); }
public void clear() throws IOException { // First clear the other contexts created by this query for (Context subContext : rewrittenStatementContexts) { subContext.clear(); } // Then clear this context if (resDir != null) { try { FileSystem fs = resDir.getFileSystem(conf); LOG.debug("Deleting result dir: {}", resDir); fs.delete(resDir, true); } catch (IOException e) { LOG.info("Context clear error: " + StringUtils.stringifyException(e)); } } if (resFile != null) { try { FileSystem fs = resFile.getFileSystem(conf); LOG.debug("Deleting result file: {}", resFile); fs.delete(resFile, false); } catch (IOException e) { LOG.info("Context clear error: " + StringUtils.stringifyException(e)); } } removeMaterializedCTEs(); removeScratchDir(); originalTracker = null; setNeedLockMgr(false); }
private void releaseContext() { try { if (ctx != null) { ctx.clear(); if (ctx.getHiveLocks() != null) { hiveLocks.addAll(ctx.getHiveLocks()); ctx.setHiveLocks(null); } ctx = null; } } catch (Exception e) { LOG.debug("Exception while clearing the context ", e); } }
/** * Get a path to store tmp data destined for external Path. * * @param path external Path to which the tmp data has to be eventually moved * @return next available tmp path on the file system corresponding extURI */ public Path getExternalTmpPath(Path path) { URI extURI = path.toUri(); if ("viewfs".equals(extURI.getScheme())) { // if we are on viewfs we don't want to use /tmp as tmp dir since rename from /tmp/.. // to final /user/hive/warehouse/ will fail later, so instead pick tmp dir // on same namespace as tbl dir. return getExtTmpPathRelTo(path.getParent()); } return new Path(getExternalScratchDir(extURI), EXT_PREFIX + nextPathId()); }
try { if (ctx == null) { ctx = new Context(job); ctxCreated = true; Utilities.setMapRedWork(job, mrWork, ctx.getMRTmpPath()); try { if (ctxCreated) { ctx.clear();
ctx = new Context(conf); ctxCreated = true; if (!ctx.isLocalOnlyExecutionMode() && conf.getBoolVar(HiveConf.ConfVars.LOCALMODEAUTO)) { ctx.setOriginalTracker(ShimLoader.getHadoopShims().getJobLauncherRpcAddress(job)); ctx.restoreOriginalTracker(); Path planPath = new Path(ctx.getLocalTmpPath(), "plan.xml"); MapredWork plan = getWork(); LOG.info("Generating plan file " + planPath.toString()); cmdLine = cmdLine + " -files " + files; workDir = ctx.getLocalTmpPath().toUri().getPath(); ctx.clear();
ctx = new Context(conf); cleanContext = true; ctx.setWmContext(wmContext); ss.getHiveVariables().get("wmpool"), ss.getHiveVariables().get("wmapp")); WmContext wmContext = ctx.getWmContext(); Path scratchDir = utils.createTezDir(ctx.getMRScratchDir(), conf); CallerContext callerContext = CallerContext.create( "HIVE", queryPlan.getQueryId(), "HIVE_QUERY_ID", queryPlan.getQueryStr()); wmContext = ctx.getWmContext(); try { if (sessionRef.value != null) { ctx.clear(); } catch (Exception e) {
ContentSummary cs = ctx.getCS(path); if (cs == null) { if (path == null) { final ExecutorService executor; int numExecutors = getMaxExecutorsForInputListing(ctx.getConf(), pathNeedProcess.size()); if (numExecutors > 1) { LOG.info("Using " + numExecutors + " threads for getContentSummary"); Configuration conf = ctx.getConf(); JobConf jobConf = new JobConf(conf); for (Path path : pathNeedProcess) { summary[2] += cs.getDirectoryCount(); ctx.addCS(entry.getKey(), cs); LOG.info("Cache Content Summary for " + entry.getKey() + " length: " + cs.getLength() + " file count: "
ctx = new Context(conf); cleanContext = true; Path scratchDir = ctx.getMRScratchDir(); ctx.clear(); } catch (Exception e) {
Table materializedTab = ctx.getMaterializedTable(cteName); if (materializedTab == null) { ctx.setResDir(null); ctx.setResFile(null); fname = FileUtils.makeQualified(location, conf).toString(); } else { fname = ctx.getExtTmpPathRelTo( FileUtils.makeQualified(location, conf)).toString(); Path stagingPath = getStagingDirectoryPathname(qb); fname = stagingPath.toString(); ctx.setResDir(stagingPath);
Context ctx = new Context(job); MapWork mapWork = (MapWork) work.getAllWork().get(0); Path scratchDir = utils.createTezDir(ctx.getMRScratchDir(), job); FileSystem fs = scratchDir.getFileSystem(job); try { LocalResource appJarLr = createJarLocalResource(utils.getExecJarPathLocal(ctx.getConf()), utils, job);
/** * This allows us to take an arbitrary ASTNode and turn it back into SQL that produced it without * needing to understand what it is (except for QuotedIdentifiers). */ private String getMatchedText(ASTNode n) { quotedIdenfierHelper.visit(n); return ctx.getTokenRewriteStream().toString(n.getTokenStartIndex(), n.getTokenStopIndex() + 1).trim(); }
queryTmpdir = dest_path; } else { queryTmpdir = ctx.getExtTmpPathRelTo(dest_path); ctx.getLoadTableOutputMap().put(ltd, output); break; .getAuthority(), partPath.toUri().getPath()); queryTmpdir = ctx.getExternalTmpPath(dest_path); table_desc = Utilities.getTableDesc(dest_tab); queryTmpdir = ctx.getMRTmpPath(); } else { queryTmpdir = ctx.getExtTmpPathRelTo(qPath); } catch (Exception e) { throw new SemanticException("Error creating temporary folder on: " if (!ctx.isMRTmpFileURI(dest_path.toUri().toString())) { idToTableNameMap.put(String.valueOf(destTableId), dest_path.toUri().toString()); currentTableId = destTableId; String statsTmpLoc = ctx.getExtTmpPathRelTo(queryTmpdir).toString(); LOG.info("Set stats collection dir : " + statsTmpLoc); conf.set(StatsSetupConst.STATS_TMP_LOC, statsTmpLoc);
@Test public void testLockAcquisitionAndRelease() throws Exception { addTableInput(); QueryPlan qp = new MockQueryPlan(this, HiveOperation.QUERY); txnMgr.openTxn(ctx, "fred"); txnMgr.acquireLocks(qp, ctx, "fred"); List<HiveLock> locks = ctx.getHiveLocks(); Assert.assertEquals(1, locks.size()); txnMgr.commitTxn(); locks = txnMgr.getLockManager().getLocks(false, false); Assert.assertEquals(0, locks.size()); }
private ASTNode genRewrittenTree(String rewrittenQuery) throws SemanticException { // Parse the rewritten query string try { ctx = new Context(conf); } catch (IOException e) { throw new SemanticException(ErrorMsg.COLUMNSTATSCOLLECTOR_IO_ERROR.getMsg()); } ctx.setCmd(rewrittenQuery); try { return ParseUtils.parse(rewrittenQuery, ctx); } catch (ParseException e) { throw new SemanticException(ErrorMsg.COLUMNSTATSCOLLECTOR_PARSE_ERROR.getMsg()); } }
private SparkJobRef submit(final DriverContext driverContext, final SparkWork sparkWork) throws Exception { final Context ctx = driverContext.getCtx(); final HiveConf hiveConf = (HiveConf) ctx.getConf(); refreshLocalResources(sparkWork, hiveConf); final JobConf jobConf = new JobConf(hiveConf); //update the credential provider location in the jobConf HiveConfUtil.updateJobCredentialProviders(jobConf); // Create temporary scratch dir final Path emptyScratchDir = ctx.getMRTmpPath(); FileSystem fs = emptyScratchDir.getFileSystem(jobConf); fs.mkdirs(emptyScratchDir); // make sure NullScanFileSystem can be loaded - HIVE-18442 jobConf.set("fs." + NullScanFileSystem.getBaseScheme() + ".impl", NullScanFileSystem.class.getCanonicalName()); byte[] jobConfBytes = KryoSerializer.serializeJobConf(jobConf); byte[] scratchDirBytes = KryoSerializer.serialize(emptyScratchDir); byte[] sparkWorkBytes = KryoSerializer.serialize(sparkWork); JobStatusJob job = new JobStatusJob(jobConfBytes, scratchDirBytes, sparkWorkBytes); if (driverContext.isShutdown()) { throw new HiveException("Operation is cancelled."); } JobHandle<Serializable> jobHandle = remoteClient.submit(job); RemoteSparkJobStatus sparkJobStatus = new RemoteSparkJobStatus(remoteClient, jobHandle, sparkClientTimtout); return new RemoteSparkJobRef(hiveConf, jobHandle, sparkJobStatus); }
public static RelNode parseQuery(HiveConf conf, String viewQuery) throws SemanticException, IOException, ParseException { final Context ctx = new Context(conf); ctx.setIsLoadingMaterializedView(true); final ASTNode ast = parse(viewQuery, ctx); final CalcitePlanner analyzer = getAnalyzer(conf, ctx); return analyzer.genLogicalPlan(ast); }
/** * Create a temporary directory depending of the path specified. * - If path is an Object store filesystem, then use the default MR scratch directory (HDFS), unless isFinalJob and * {@link BlobStorageUtils#areOptimizationsEnabled(Configuration)} are both true, then return a path on * the blobstore. * - If path is on HDFS, then create a staging directory inside the path * * @param path Path used to verify the Filesystem to use for temporary directory * * @return A path to the new temporary directory */ public Path getTempDirForInterimJobPath(Path path) { // For better write performance, we use HDFS for temporary data when object store is used. // Note that the scratch directory configuration variable must use HDFS or any other // non-blobstorage system to take advantage of this performance. boolean isBlobstorageOptimized = BlobStorageUtils.isBlobStoragePath(conf, path) && !BlobStorageUtils.isBlobStorageAsScratchDir(conf) && BlobStorageUtils.areOptimizationsEnabled(conf); if (isPathLocal(path) || isBlobstorageOptimized) { return getMRTmpPath(); } return getExtTmpPathRelTo(path); }
if (viewFullyQualifiedName == null) { ctx.setTokenRewriteStream(tokens); } else { ctx.addViewTokenRewriteStream(viewFullyQualifiedName, tokens); lexer.setHiveConf(ctx.getConf()); parser.setHiveConf(ctx.getConf());
Configuration conf = context.getConf(); try { long aliasTotalKnownInputSize = getTotalKnownInputSize(context, currJoinWork.getMapWork(), resolverCtx.setTaskToAliases(taskToAliases); resolverCtx.setCommonJoinTask(currTask); resolverCtx.setLocalTmpDir(context.getLocalScratchDir(false)); resolverCtx.setHdfsTmpDir(context.getMRScratchDir()); cndTsk.setResolverCtx(resolverCtx);