steveloughran commented on code in PR #5519:
URL: https://github.com/apache/hadoop/pull/5519#discussion_r1168511215


##########
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java:
##########
@@ -756,66 +756,75 @@ private void testConcurrentCommitTaskWithSubDir(int 
version)
     conf.setInt(FileOutputCommitter.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION,
         version);
 
-    conf.setClass("fs.file.impl", RLFS.class, FileSystem.class);
+    final String fileImpl = "fs.file.impl";
+    final String fileImplClassname = "org.apache.hadoop.fs.LocalFileSystem";
+    conf.setClass(fileImpl, RLFS.class, FileSystem.class);
     FileSystem.closeAll();
 
-    final JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
-    final FileOutputCommitter amCommitter =
-        new FileOutputCommitter(outDir, jContext);
-    amCommitter.setupJob(jContext);
-
-    final TaskAttemptContext[] taCtx = new TaskAttemptContextImpl[2];
-    taCtx[0] = new TaskAttemptContextImpl(conf, taskID);
-    taCtx[1] = new TaskAttemptContextImpl(conf, taskID1);
-
-    final TextOutputFormat[] tof = new TextOutputFormat[2];
-    for (int i = 0; i < tof.length; i++) {
-      tof[i] = new TextOutputFormat() {
-        @Override
-        public Path getDefaultWorkFile(TaskAttemptContext context,
-            String extension) throws IOException {
-          final FileOutputCommitter foc = (FileOutputCommitter)
-              getOutputCommitter(context);
-          return new Path(new Path(foc.getWorkPath(), SUB_DIR),
-              getUniqueFile(context, getOutputName(context), extension));
-        }
-      };
-    }
-
-    final ExecutorService executor = HadoopExecutors.newFixedThreadPool(2);
     try {
-      for (int i = 0; i < taCtx.length; i++) {
-        final int taskIdx = i;
-        executor.submit(new Callable<Void>() {
+      final JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
+      final FileOutputCommitter amCommitter =
+          new FileOutputCommitter(outDir, jContext);
+      amCommitter.setupJob(jContext);
+
+      final TaskAttemptContext[] taCtx = new TaskAttemptContextImpl[2];
+      taCtx[0] = new TaskAttemptContextImpl(conf, taskID);
+      taCtx[1] = new TaskAttemptContextImpl(conf, taskID1);
+
+      final TextOutputFormat[] tof = new TextOutputFormat[2];
+      for (int i = 0; i < tof.length; i++) {
+        tof[i] = new TextOutputFormat() {
           @Override
-          public Void call() throws IOException, InterruptedException {
-            final OutputCommitter outputCommitter =
-                tof[taskIdx].getOutputCommitter(taCtx[taskIdx]);
-            outputCommitter.setupTask(taCtx[taskIdx]);
-            final RecordWriter rw =
-                tof[taskIdx].getRecordWriter(taCtx[taskIdx]);
-            writeOutput(rw, taCtx[taskIdx]);
-            outputCommitter.commitTask(taCtx[taskIdx]);
-            return null;
+          public Path getDefaultWorkFile(TaskAttemptContext context,
+              String extension) throws IOException {
+            final FileOutputCommitter foc = (FileOutputCommitter)
+                getOutputCommitter(context);
+            return new Path(new Path(foc.getWorkPath(), SUB_DIR),
+                getUniqueFile(context, getOutputName(context), extension));
           }
-        });
+        };
       }
-    } finally {
-      executor.shutdown();
-      while (!executor.awaitTermination(1, TimeUnit.SECONDS)) {
-        LOG.info("Awaiting thread termination!");
+
+      final ExecutorService executor = HadoopExecutors.newFixedThreadPool(2);
+      try {
+        for (int i = 0; i < taCtx.length; i++) {
+          final int taskIdx = i;
+          executor.submit(new Callable<Void>() {
+            @Override
+            public Void call() throws IOException, InterruptedException {
+              final OutputCommitter outputCommitter =
+                  tof[taskIdx].getOutputCommitter(taCtx[taskIdx]);
+              outputCommitter.setupTask(taCtx[taskIdx]);
+              final RecordWriter rw =
+                  tof[taskIdx].getRecordWriter(taCtx[taskIdx]);
+              writeOutput(rw, taCtx[taskIdx]);
+              outputCommitter.commitTask(taCtx[taskIdx]);
+              return null;
+            }
+          });
+        }
+      } finally {
+        executor.shutdown();
+        while (!executor.awaitTermination(1, TimeUnit.SECONDS)) {
+          LOG.info("Awaiting thread termination!");
+        }
       }
-    }
 
-    amCommitter.commitJob(jContext);
-    final RawLocalFileSystem lfs = new RawLocalFileSystem();
-    lfs.setConf(conf);
-    assertFalse("Must not end up with sub_dir/sub_dir",
-        lfs.exists(new Path(OUT_SUB_DIR, SUB_DIR)));
+      amCommitter.commitJob(jContext);
+      final RawLocalFileSystem lfs = new RawLocalFileSystem();
+      lfs.setConf(conf);
+      assertFalse("Must not end up with sub_dir/sub_dir",
+          lfs.exists(new Path(OUT_SUB_DIR, SUB_DIR)));
 
-    // validate output
-    validateContent(OUT_SUB_DIR);
-    FileUtil.fullyDelete(new File(outDir.toString()));
+      // validate output
+      validateContent(OUT_SUB_DIR);
+      FileUtil.fullyDelete(new File(outDir.toString()));
+    } finally {
+      // needed to avoid this test contaminating others in the same JVM
+      FileSystem.closeAll();
+      conf.set(fileImpl, fileImplClassname);
+      conf.set(fileImpl, fileImplClassname);

Review Comment:
   duplicate. cut
   the reason it is in is not so much because of any change in the pr, as it 
surfaced a condition which is already there -this test changes the default 
"file" fs, and in some test runs that wasn't being reset, so other tests were 
failing later for no obvious reason



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

To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to