Quellcode durchsuchen

HADOOP-16212. Update guava to 27.0-jre in hadoop-project branch-3.0. Contributed by Gabor Bota.

Sean Mackrory vor 6 Jahren
Ursprung
Commit
c352b0011e
20 geänderte Dateien mit 75 neuen und 43 gelöschten Zeilen
  1. 7 0
      hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
  2. 3 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  3. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
  4. 7 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SemaphoredDelegatingExecutor.java
  5. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ZKUtil.java
  6. 8 8
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java
  7. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestZKUtil.java
  8. 8 0
      hadoop-common-project/hadoop-kms/dev-support/findbugsExcludeFile.xml
  9. 1 1
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
  10. 1 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/LocalResolver.java
  11. 8 0
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  12. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
  13. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java
  14. 7 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java
  15. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/ThrottledAsyncChecker.java
  16. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java
  17. 7 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/checker/TestThrottledAsyncCheckerTimeout.java
  18. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
  19. 6 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java
  20. 1 1
      hadoop-project/pom.xml

+ 7 - 0
hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml

@@ -409,6 +409,13 @@
     <Bug pattern="NP_NULL_PARAM_DEREF"/>
   </Match>
 
+  <!-- propertyName is checked with isNullOrEmpty (fix after guava 27) -->
+  <Match>
+    <Class name="org.apache.hadoop.conf.Configuration"/>
+    <Method name="asXmlDocument"/>
+    <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/>
+  </Match>
+
   <Match>
     <Class name="org.apache.hadoop.ipc.ExternalCall"/>
     <Filed name="done"/>

+ 3 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -68,6 +68,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
+import javax.annotation.Nullable;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
 import javax.xml.stream.XMLInputFactory;
@@ -3361,7 +3362,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * <p>
    * @param out the writer to write to.
    */
-  public void writeXml(String propertyName, Writer out)
+  public void writeXml(@Nullable String propertyName, Writer out)
       throws IOException, IllegalArgumentException {
     Document doc = asXmlDocument(propertyName);
 
@@ -3383,7 +3384,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   /**
    * Return the XML DOM corresponding to this Configuration.
    */
-  private synchronized Document asXmlDocument(String propertyName)
+  private synchronized Document asXmlDocument(@Nullable String propertyName)
       throws IOException, IllegalArgumentException {
     Document doc;
     try {

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java

@@ -375,7 +375,7 @@ public class Groups {
           backgroundRefreshException.incrementAndGet();
           backgroundRefreshRunning.decrementAndGet();
         }
-      });
+      }, MoreExecutors.directExecutor());
       return listenableFuture;
     }
 

+ 7 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SemaphoredDelegatingExecutor.java

@@ -107,7 +107,7 @@ public class SemaphoredDelegatingExecutor extends
       queueingPermits.acquire();
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      return Futures.immediateFailedCheckedFuture(e);
+      return Futures.immediateFailedFuture(e);
     }
     return super.submit(new CallableWithPermitRelease<>(task));
   }
@@ -118,7 +118,7 @@ public class SemaphoredDelegatingExecutor extends
       queueingPermits.acquire();
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      return Futures.immediateFailedCheckedFuture(e);
+      return Futures.immediateFailedFuture(e);
     }
     return super.submit(new RunnableWithPermitRelease(task), result);
   }
@@ -129,7 +129,7 @@ public class SemaphoredDelegatingExecutor extends
       queueingPermits.acquire();
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      return Futures.immediateFailedCheckedFuture(e);
+      return Futures.immediateFailedFuture(e);
     }
     return super.submit(new RunnableWithPermitRelease(task));
   }
@@ -173,10 +173,10 @@ public class SemaphoredDelegatingExecutor extends
   public String toString() {
     final StringBuilder sb = new StringBuilder(
         "SemaphoredDelegatingExecutor{");
-    sb.append("permitCount=").append(getPermitCount());
-    sb.append(", available=").append(getAvailablePermits());
-    sb.append(", waiting=").append(getWaitingCount());
-    sb.append('}');
+    sb.append("permitCount=").append(getPermitCount())
+        .append(", available=").append(getAvailablePermits())
+        .append(", waiting=").append(getWaitingCount())
+        .append('}');
     return sb.toString();
   }
 

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ZKUtil.java

@@ -172,7 +172,7 @@ public class ZKUtil {
       return valInConf;
     }
     String path = valInConf.substring(1).trim();
-    return Files.toString(new File(path), Charsets.UTF_8).trim();
+    return Files.asCharSource(new File(path), Charsets.UTF_8).read().trim();
   }
 
   /**

+ 8 - 8
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java

@@ -41,8 +41,8 @@ public class TestTableMapping {
   public void testResolve() throws IOException {
     File mapFile = File.createTempFile(getClass().getSimpleName() +
         ".testResolve", ".txt");
-    Files.write(hostName1 + " /rack1\n" +
-                hostName2 + "\t/rack2\n", mapFile, Charsets.UTF_8);
+    Files.asCharSink(mapFile, Charsets.UTF_8).write(
+        hostName1 + " /rack1\n" + hostName2 + "\t/rack2\n");
     mapFile.deleteOnExit();
     TableMapping mapping = new TableMapping();
 
@@ -64,8 +64,8 @@ public class TestTableMapping {
   public void testTableCaching() throws IOException {
     File mapFile = File.createTempFile(getClass().getSimpleName() +
         ".testTableCaching", ".txt");
-    Files.write(hostName1 + " /rack1\n" +
-        hostName2 + "\t/rack2\n", mapFile, Charsets.UTF_8);
+    Files.asCharSink(mapFile, Charsets.UTF_8).write(
+        hostName1 + " /rack1\n" + hostName2 + "\t/rack2\n");
     mapFile.deleteOnExit();
     TableMapping mapping = new TableMapping();
 
@@ -128,8 +128,8 @@ public class TestTableMapping {
   public void testClearingCachedMappings() throws IOException {
     File mapFile = File.createTempFile(getClass().getSimpleName() +
         ".testClearingCachedMappings", ".txt");
-    Files.write(hostName1 + " /rack1\n" +
-                hostName2 + "\t/rack2\n", mapFile, Charsets.UTF_8);
+    Files.asCharSink(mapFile, Charsets.UTF_8).write(
+        hostName1 + " /rack1\n" + hostName2 + "\t/rack2\n");
     mapFile.deleteOnExit();
 
     TableMapping mapping = new TableMapping();
@@ -147,7 +147,7 @@ public class TestTableMapping {
     assertEquals("/rack1", result.get(0));
     assertEquals("/rack2", result.get(1));
 
-    Files.write("", mapFile, Charsets.UTF_8);
+    Files.asCharSink(mapFile, Charsets.UTF_8).write("");
 
     mapping.reloadCachedMappings();
 
@@ -166,7 +166,7 @@ public class TestTableMapping {
   public void testBadFile() throws IOException {
     File mapFile = File.createTempFile(getClass().getSimpleName() +
         ".testBadFile", ".txt");
-    Files.write("bad contents", mapFile, Charsets.UTF_8);
+    Files.asCharSink(mapFile, Charsets.UTF_8).write("bad contents");
     mapFile.deleteOnExit();
     TableMapping mapping = new TableMapping();
 

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestZKUtil.java

@@ -131,7 +131,7 @@ public class TestZKUtil {
     assertEquals("x", ZKUtil.resolveConfIndirection("x"));
     
     TEST_FILE.getParentFile().mkdirs();
-    Files.write("hello world", TEST_FILE, Charsets.UTF_8);
+    Files.asCharSink(TEST_FILE, Charsets.UTF_8).write("hello world");
     assertEquals("hello world", ZKUtil.resolveConfIndirection(
         "@" + TEST_FILE.getAbsolutePath()));
     

+ 8 - 0
hadoop-common-project/hadoop-kms/dev-support/findbugsExcludeFile.xml

@@ -15,6 +15,14 @@
    limitations under the License.
 -->
 <FindBugsFilter>
+
+  <!-- The called method signature is isNullOrEmpty(@Nullable String string) in guava 27, so this should be ignored. -->
+  <Match>
+      <Class name="org.apache.hadoop.crypto.key.kms.server.KMSAudit"/>
+      <Method name="op" />
+      <Bug pattern="NP_NULL_PARAM_DEREF"/>
+  </Match>
+
   <!--
     Findbug is complaining about System.out being NULL
   -->

+ 1 - 1
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java

@@ -555,7 +555,7 @@ public class KMS {
       throws Exception {
     LOG.trace("Entering reencryptEncryptedKeys method.");
     try {
-      final Stopwatch sw = new Stopwatch().start();
+      final Stopwatch sw = Stopwatch.createStarted();
       checkNotEmpty(name, "name");
       checkNotNull(jsonPayload, "jsonPayload");
       final UserGroupInformation user = HttpUserGroupInformation.get();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/LocalResolver.java

@@ -196,7 +196,7 @@ public class LocalResolver extends RouterResolver<String, String> {
         try {
           String nsId = nn.getNameserviceId();
           String rpcAddress = nn.getRpcAddress();
-          String hostname = HostAndPort.fromString(rpcAddress).getHostText();
+          String hostname = HostAndPort.fromString(rpcAddress).getHost();
           ret.put(hostname, nsId);
           if (hostname.equals(localHostname)) {
             ret.put(localIp, nsId);

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml

@@ -193,6 +193,14 @@
      </Match>
 
 
+     <!-- guava 27.0 update: @Nullable is not detected, however it's there -->
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.qjournal.server.Journal" />
+       <Method name="getPersistedPaxosData" />
+       <Bug pattern="NP_NULL_PARAM_DEREF" />
+     </Match>
+
+
      <!-- guava 27.0 update: @Nullable is not detected, however it's there -->
      <Match>
        <Class name="org.apache.hadoop.hdfs.qjournal.server.JournalNode" />

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java

@@ -446,7 +446,7 @@ public class IPCLoggerChannel implements AsyncLogger {
           public void onSuccess(Void t) {
             unreserveQueueSpace(data.length);
           }
-        });
+        }, MoreExecutors.directExecutor());
       }
     }
     return ret;

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java

@@ -22,6 +22,7 @@ import java.util.Map.Entry;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.util.concurrent.MoreExecutors;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StopWatch;
 import org.apache.hadoop.util.Timer;
@@ -80,7 +81,7 @@ class QuorumCall<KEY, RESULT> {
         public void onSuccess(RESULT res) {
           qr.addResult(e.getKey(), res);
         }
-      });
+      }, MoreExecutors.directExecutor());
     }
     return qr;
   }

+ 7 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java

@@ -25,6 +25,7 @@ import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -224,12 +225,12 @@ public class DatasetVolumeChecker {
         Futures.addCallback(olf.get(),
             new ResultHandler(reference, healthyVolumes, failedVolumes,
                 numVolumes, new Callback() {
-              @Override
-              public void call(Set<FsVolumeSpi> ignored1,
-                               Set<FsVolumeSpi> ignored2) {
-                latch.countDown();
-              }
-            }));
+                  @Override
+                  public void call(Set<FsVolumeSpi> ignored1,
+                                   Set<FsVolumeSpi> ignored2) {
+                    latch.countDown();
+                  }
+                }), MoreExecutors.directExecutor());
       } else {
         IOUtils.cleanup(null, reference);
         if (numVolumes.decrementAndGet() == 0) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/ThrottledAsyncChecker.java

@@ -182,7 +182,7 @@ public class ThrottledAsyncChecker<K, V> implements AsyncChecker<K, V> {
               t, timer.monotonicNow()));
         }
       }
-    });
+    }, MoreExecutors.directExecutor());
   }
 
   /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java

@@ -541,7 +541,7 @@ public class ReencryptionHandler implements Runnable {
       if (batch.isEmpty()) {
         return new ReencryptionTask(zoneNodeId, 0, batch);
       }
-      final Stopwatch kmsSW = new Stopwatch().start();
+      final Stopwatch kmsSW = Stopwatch.createStarted();
 
       int numFailures = 0;
       String result = "Completed";

+ 7 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/checker/TestThrottledAsyncCheckerTimeout.java

@@ -29,6 +29,7 @@ import com.google.common.base.Optional;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.MoreExecutors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -112,7 +113,7 @@ public class TestThrottledAsyncCheckerTimeout {
         numCallbackInvocationsFailure.incrementAndGet();
         callbackResult.set(true);
       }
-    });
+    }, MoreExecutors.directExecutor());
 
     while (!callbackResult.get()) {
       // Wait for the callback
@@ -144,7 +145,8 @@ public class TestThrottledAsyncCheckerTimeout {
         .schedule(target, true);
 
     assertTrue(olf1.isPresent());
-    Futures.addCallback(olf1.get(), futureCallback);
+    Futures.addCallback(olf1.get(), futureCallback,
+        MoreExecutors.directExecutor());
 
     // Verify that timeout results in only 1 onFailure call and 0 onSuccess
     // calls.
@@ -160,7 +162,8 @@ public class TestThrottledAsyncCheckerTimeout {
         .schedule(target, true);
 
     assertTrue(olf2.isPresent());
-    Futures.addCallback(olf2.get(), futureCallback);
+    Futures.addCallback(olf2.get(), futureCallback,
+        MoreExecutors.directExecutor());
 
     // Verify that normal check (dummy) results in only 1 onSuccess call.
     // Number of times onFailure is invoked should remain the same i.e. 1.
@@ -198,7 +201,7 @@ public class TestThrottledAsyncCheckerTimeout {
         throwable[0] = t;
         callbackResult.set(true);
       }
-    });
+    }, MoreExecutors.directExecutor());
 
     while (!callbackResult.get()) {
       // Wait for the callback

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java

@@ -163,7 +163,7 @@ public class TestDFSHAAdminMiniCluster {
     assertEquals(0, runTool("-ns", "minidfs-ns", "-failover", "nn2", "nn1"));
 
     // Fencer has not run yet, since none of the above required fencing 
-    assertEquals("", Files.toString(tmpFile, Charsets.UTF_8));
+    assertEquals("", Files.asCharSource(tmpFile, Charsets.UTF_8).read());
 
     // Test failover with fencer and forcefence option
     assertEquals(0, runTool("-failover", "nn1", "nn2", "--forcefence"));
@@ -171,8 +171,8 @@ public class TestDFSHAAdminMiniCluster {
     // The fence script should run with the configuration from the target
     // node, rather than the configuration from the fencing node. Strip
     // out any trailing spaces and CR/LFs which may be present on Windows.
-    String fenceCommandOutput =Files.toString(tmpFile, Charsets.UTF_8).
-            replaceAll(" *[\r\n]+", "");
+    String fenceCommandOutput = Files.asCharSource(tmpFile, Charsets.UTF_8)
+        .read().replaceAll(" *[\r\n]+", "");
     assertEquals("minidfs-ns.nn1 " + nn1Port + " nn1", fenceCommandOutput);
     tmpFile.delete();
     

+ 6 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java

@@ -120,7 +120,8 @@ public class LocatedFileStatusFetcher {
       runningTasks.incrementAndGet();
       ListenableFuture<ProcessInitialInputPathCallable.Result> future = exec
           .submit(new ProcessInitialInputPathCallable(p, conf, inputFilter));
-      Futures.addCallback(future, processInitialInputPathCallback);
+      Futures.addCallback(future, processInitialInputPathCallback,
+          MoreExecutors.directExecutor());
     }
 
     runningTasks.decrementAndGet();
@@ -267,7 +268,8 @@ public class LocatedFileStatusFetcher {
             ListenableFuture<ProcessInputDirCallable.Result> future = exec
                 .submit(new ProcessInputDirCallable(result.fs, fileStatus,
                     recursive, inputFilter));
-            Futures.addCallback(future, processInputDirCallback);
+            Futures.addCallback(future, processInputDirCallback,
+                MoreExecutors.directExecutor());
           }
         }
         decrementRunningAndCheckCompletion();
@@ -353,7 +355,8 @@ public class LocatedFileStatusFetcher {
             ListenableFuture<ProcessInputDirCallable.Result> future = exec
                 .submit(new ProcessInputDirCallable(result.fs, matched,
                     recursive, inputFilter));
-            Futures.addCallback(future, processInputDirCallback);
+            Futures.addCallback(future, processInputDirCallback,
+                MoreExecutors.directExecutor());
           }
         }
         decrementRunningAndCheckCompletion();

+ 1 - 1
hadoop-project/pom.xml

@@ -90,7 +90,7 @@
     <findbugs.version>3.0.0</findbugs.version>
     <spotbugs.version>3.1.0-RC1</spotbugs.version>
 
-    <guava.version>11.0.2</guava.version>
+    <guava.version>27.0-jre</guava.version>
     <guice.version>4.0</guice.version>
     <joda-time.version>2.9.9</joda-time.version>