Prechádzať zdrojové kódy

HADOOP-18022. Add restrict-imports-enforcer-rule for Guava Preconditions and remove remaining usages (#3712)

Reviewed-by: Akira Ajisaka <aajisaka@apache.org>
Signed-off-by: Takanobu Asanuma <tasanuma@apache.org>
Viraj Jasani 3 rokov pred
rodič
commit
215388beea
26 zmenil súbory, kde vykonal 124 pridanie a 88 odobranie
  1. 1 1
      hadoop-cloud-storage-project/hadoop-cos/src/main/java/org/apache/hadoop/fs/cosn/auth/COSCredentialsProviderList.java
  2. 1 1
      hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSBlockOutputStream.java
  3. 1 1
      hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCommonUtils.java
  4. 1 1
      hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java
  5. 1 1
      hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.java
  6. 1 1
      hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSInputStream.java
  7. 1 1
      hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java
  8. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
  9. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java
  10. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/FileSystemCounterGroup.java
  11. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/FrameworkCounterGroup.java
  12. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java
  13. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java
  14. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/NamedCommitterFactory.java
  15. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.java
  16. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/buffer/ByteBufferDataWriter.java
  17. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/testutil/BytesFactory.java
  18. 2 3
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLocatedFileStatus.java
  19. 12 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationIdPBImpl.java
  20. 21 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
  21. 9 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java
  22. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java
  23. 18 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
  24. 18 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java
  25. 15 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/SubApplicationEntityReader.java
  26. 8 0
      pom.xml

+ 1 - 1
hadoop-cloud-storage-project/hadoop-cos/src/main/java/org/apache/hadoop/fs/cosn/auth/COSCredentialsProviderList.java

@@ -24,7 +24,7 @@ import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 import com.qcloud.cos.auth.AnonymousCOSCredentials;
 import com.qcloud.cos.auth.COSCredentials;
 import com.qcloud.cos.auth.COSCredentialsProvider;

+ 1 - 1
hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSBlockOutputStream.java

@@ -19,7 +19,7 @@
 package org.apache.hadoop.fs.obs;
 
 import org.apache.hadoop.classification.VisibleForTesting;
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures;
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture;
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService;

+ 1 - 1
hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCommonUtils.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.fs.obs;
 
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 import com.obs.services.ObsClient;
 import com.obs.services.exception.ObsException;
 import com.obs.services.model.AbortMultipartUploadRequest;

+ 1 - 1
hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java

@@ -19,7 +19,7 @@
 package org.apache.hadoop.fs.obs;
 
 import org.apache.hadoop.classification.VisibleForTesting;
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSExceptionMessages;

+ 1 - 1
hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.fs.obs;
 
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 import com.obs.services.exception.ObsException;
 
 import java.io.IOException;

+ 1 - 1
hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSInputStream.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.fs.obs;
 
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 import com.obs.services.ObsClient;
 import com.obs.services.exception.ObsException;
 import com.obs.services.model.GetObjectRequest;

+ 1 - 1
hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.fs.obs;
 
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 import com.obs.services.ObsClient;
 import com.obs.services.exception.ObsException;
 import com.obs.services.model.AbortMultipartUploadRequest;

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

@@ -146,7 +146,7 @@ import org.apache.hadoop.yarn.util.UnitsConversionUtil;
 import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 
 import org.apache.hadoop.classification.VisibleForTesting;
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java

@@ -78,7 +78,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 import com.google.inject.Inject;
 
 @Path("/ws/v1/mapreduce")

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/FileSystemCounterGroup.java

@@ -28,7 +28,7 @@ import java.util.Iterator;
 import java.util.Map;
 
 import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
-import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.*;
+import static org.apache.hadoop.util.Preconditions.*;
 import org.apache.hadoop.thirdparty.com.google.common.collect.AbstractIterator;
 import org.apache.hadoop.thirdparty.com.google.common.collect.Iterators;
 import org.apache.hadoop.thirdparty.com.google.common.collect.Maps;

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/FrameworkCounterGroup.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.mapreduce.counters;
 
-import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.util.Preconditions.checkNotNull;
 
 import java.io.DataInput;
 import java.io.DataOutput;

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java

@@ -37,7 +37,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 
 import org.apache.hadoop.classification.VisibleForTesting;
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.util.DurationInfo;
 import org.apache.hadoop.util.Progressable;
 import org.slf4j.Logger;

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java

@@ -21,7 +21,7 @@ package org.apache.hadoop.mapreduce.lib.output;
 import java.io.IOException;
 import java.text.NumberFormat;
 
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/NamedCommitterFactory.java

@@ -22,7 +22,7 @@ import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.mapreduce.lib.output;
 
 import java.io.IOException;
 
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/buffer/ByteBufferDataWriter.java

@@ -23,7 +23,7 @@ import java.nio.ByteBuffer;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.mapred.nativetask.NativeDataTarget;
 
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 
 /**
  * DataOutputStream implementation which buffers data in a fixed-size

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/testutil/BytesFactory.java

@@ -19,7 +19,7 @@ package org.apache.hadoop.mapred.nativetask.testutil;
 
 import java.util.Random;
 
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.thirdparty.com.google.common.primitives.Ints;
 import org.apache.hadoop.thirdparty.com.google.common.primitives.Longs;
 

+ 2 - 3
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLocatedFileStatus.java

@@ -22,8 +22,7 @@ import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.EtagSource;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.LocatedFileStatus;
-
-import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
+import org.apache.hadoop.util.Preconditions;
 
 /**
  * {@link LocatedFileStatus} extended to also carry an ETag.
@@ -38,7 +37,7 @@ public class AbfsLocatedFileStatus extends LocatedFileStatus implements EtagSour
   private final String etag;
 
   public AbfsLocatedFileStatus(FileStatus status, BlockLocation[] locations) {
-    super(checkNotNull(status), locations);
+    super(Preconditions.checkNotNull(status), locations);
     if (status instanceof EtagSource) {
       this.etag = ((EtagSource) status).getEtag();
     } else {

+ 12 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationIdPBImpl.java

@@ -24,8 +24,6 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 
-import org.apache.hadoop.util.Preconditions;
-
 @Private
 @Unstable
 public class ApplicationIdPBImpl extends ApplicationId {
@@ -46,24 +44,32 @@ public class ApplicationIdPBImpl extends ApplicationId {
 
   @Override
   public int getId() {
-    Preconditions.checkNotNull(proto);
+    if (proto == null) {
+      throw new NullPointerException("The argument object is NULL");
+    }
     return proto.getId();
   }
 
   @Override
   protected void setId(int id) {
-    Preconditions.checkNotNull(builder);
+    if (builder == null) {
+      throw new NullPointerException("The argument object is NULL");
+    }
     builder.setId(id);
   }
   @Override
   public long getClusterTimestamp() {
-    Preconditions.checkNotNull(proto);
+    if (proto == null) {
+      throw new NullPointerException("The argument object is NULL");
+    }
     return proto.getClusterTimestamp();
   }
 
   @Override
   protected void setClusterTimestamp(long clusterTimestamp) {
-    Preconditions.checkNotNull(builder);
+    if (builder == null) {
+      throw new NullPointerException("The argument object is NULL");
+    }
     builder.setClusterTimestamp((clusterTimestamp));
   }
 

+ 21 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java

@@ -58,8 +58,6 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 
-import org.apache.hadoop.util.Preconditions;
-
 /**
  * Timeline entity reader for application entities that are stored in the
  * application table.
@@ -336,21 +334,29 @@ class ApplicationEntityReader extends GenericEntityReader {
 
   @Override
   protected void validateParams() {
-    Preconditions.checkNotNull(getContext(), "context shouldn't be null");
-    Preconditions.checkNotNull(
-        getDataToRetrieve(), "data to retrieve shouldn't be null");
-    Preconditions.checkNotNull(getContext().getClusterId(),
-        "clusterId shouldn't be null");
-    Preconditions.checkNotNull(getContext().getEntityType(),
-        "entityType shouldn't be null");
+    if (getContext() == null) {
+      throw new NullPointerException("context shouldn't be null");
+    }
+    if (getDataToRetrieve() == null) {
+      throw new NullPointerException("data to retrieve shouldn't be null");
+    }
+    if (getContext().getClusterId() == null) {
+      throw new NullPointerException("clusterId shouldn't be null");
+    }
+    if (getContext().getEntityType() == null) {
+      throw new NullPointerException("entityType shouldn't be null");
+    }
     if (isSingleEntityRead()) {
-      Preconditions.checkNotNull(getContext().getAppId(),
-          "appId shouldn't be null");
+      if (getContext().getAppId() == null) {
+        throw new NullPointerException("appId shouldn't be null");
+      }
     } else {
-      Preconditions.checkNotNull(getContext().getUserId(),
-          "userId shouldn't be null");
-      Preconditions.checkNotNull(getContext().getFlowName(),
-          "flowName shouldn't be null");
+      if (getContext().getUserId() == null) {
+        throw new NullPointerException("userId shouldn't be null");
+      }
+      if (getContext().getFlowName() == null) {
+        throw new NullPointerException("flowName shouldn't be null");
+      }
     }
   }
 

+ 9 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
 
-import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
@@ -115,11 +114,15 @@ public final class EntityTypeReader extends AbstractTimelineStorageReader {
 
   @Override
   protected void validateParams() {
-    Preconditions.checkNotNull(getContext(), "context shouldn't be null");
-    Preconditions.checkNotNull(getContext().getClusterId(),
-        "clusterId shouldn't be null");
-    Preconditions.checkNotNull(getContext().getAppId(),
-        "appId shouldn't be null");
+    if (getContext() == null) {
+      throw new NullPointerException("context shouldn't be null");
+    }
+    if (getContext().getClusterId() == null) {
+      throw new NullPointerException("clusterId shouldn't be null");
+    }
+    if (getContext().getAppId() == null) {
+      throw new NullPointerException("appId shouldn't be null");
+    }
   }
 
   /**

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java

@@ -46,8 +46,6 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRo
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTableRW;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 
-import org.apache.hadoop.util.Preconditions;
-
 /**
  * Timeline entity reader for flow activity entities that are stored in the
  * flow activity table.
@@ -82,8 +80,10 @@ class FlowActivityEntityReader extends TimelineEntityReader {
 
   @Override
   protected void validateParams() {
-    Preconditions.checkNotNull(getContext().getClusterId(),
-        "clusterId shouldn't be null");
+    String clusterId = getContext().getClusterId();
+    if (clusterId == null) {
+      throw new NullPointerException("clusterId shouldn't be null");
+    }
   }
 
   @Override

+ 18 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java

@@ -57,8 +57,6 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKeyP
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTableRW;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 
-import org.apache.hadoop.util.Preconditions;
-
 /**
  * Timeline entity reader for flow run entities that are stored in the flow run
  * table.
@@ -86,18 +84,25 @@ class FlowRunEntityReader extends TimelineEntityReader {
 
   @Override
   protected void validateParams() {
-    Preconditions.checkNotNull(getContext(), "context shouldn't be null");
-    Preconditions.checkNotNull(getDataToRetrieve(),
-        "data to retrieve shouldn't be null");
-    Preconditions.checkNotNull(getContext().getClusterId(),
-        "clusterId shouldn't be null");
-    Preconditions.checkNotNull(getContext().getUserId(),
-        "userId shouldn't be null");
-    Preconditions.checkNotNull(getContext().getFlowName(),
-        "flowName shouldn't be null");
+    if (getContext() == null) {
+      throw new NullPointerException("context shouldn't be null");
+    }
+    if (getDataToRetrieve() == null) {
+      throw new NullPointerException("data to retrieve shouldn't be null");
+    }
+    if (getContext().getClusterId() == null) {
+      throw new NullPointerException("clusterId shouldn't be null");
+    }
+    if (getContext().getUserId() == null) {
+      throw new NullPointerException("userId shouldn't be null");
+    }
+    if (getContext().getFlowName() == null) {
+      throw new NullPointerException("flowName shouldn't be null");
+    }
     if (isSingleEntityRead()) {
-      Preconditions.checkNotNull(getContext().getFlowRunId(),
-          "flowRunId shouldn't be null");
+      if (getContext().getFlowRunId() == null) {
+        throw new NullPointerException("flowRunId shouldn't be null");
+      }
     }
     EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
     if (!isSingleEntityRead() && fieldsToRetrieve != null) {

+ 18 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java

@@ -63,8 +63,6 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTableRW;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 
-import org.apache.hadoop.util.Preconditions;
-
 /**
  * Timeline entity reader for generic entities that are stored in the entity
  * table.
@@ -406,18 +404,25 @@ class GenericEntityReader extends TimelineEntityReader {
 
   @Override
   protected void validateParams() {
-    Preconditions.checkNotNull(getContext(), "context shouldn't be null");
-    Preconditions.checkNotNull(getDataToRetrieve(),
-        "data to retrieve shouldn't be null");
-    Preconditions.checkNotNull(getContext().getClusterId(),
-        "clusterId shouldn't be null");
-    Preconditions.checkNotNull(getContext().getAppId(),
-        "appId shouldn't be null");
-    Preconditions.checkNotNull(getContext().getEntityType(),
-        "entityType shouldn't be null");
+    if (getContext() == null) {
+      throw new NullPointerException("context shouldn't be null");
+    }
+    if (getDataToRetrieve() == null) {
+      throw new NullPointerException("data to retrieve shouldn't be null");
+    }
+    if (getContext().getClusterId() == null) {
+      throw new NullPointerException("clusterId shouldn't be null");
+    }
+    if (getContext().getAppId() == null) {
+      throw new NullPointerException("appId shouldn't be null");
+    }
+    if (getContext().getEntityType() == null) {
+      throw new NullPointerException("entityType shouldn't be null");
+    }
     if (isSingleEntityRead()) {
-      Preconditions.checkNotNull(getContext().getEntityId(),
-          "entityId shouldn't be null");
+      if (getContext().getEntityId() == null) {
+        throw new NullPointerException("entityId shouldn't be null");
+      }
     }
   }
 

+ 15 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/SubApplicationEntityReader.java

@@ -56,8 +56,6 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubA
 import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTableRW;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 
-import org.apache.hadoop.util.Preconditions;
-
 class SubApplicationEntityReader extends GenericEntityReader {
   private static final SubApplicationTableRW SUB_APPLICATION_TABLE =
       new SubApplicationTableRW();
@@ -308,15 +306,21 @@ class SubApplicationEntityReader extends GenericEntityReader {
 
   @Override
   protected void validateParams() {
-    Preconditions.checkNotNull(getContext(), "context shouldn't be null");
-    Preconditions.checkNotNull(getDataToRetrieve(),
-        "data to retrieve shouldn't be null");
-    Preconditions.checkNotNull(getContext().getClusterId(),
-        "clusterId shouldn't be null");
-    Preconditions.checkNotNull(getContext().getDoAsUser(),
-        "DoAsUser shouldn't be null");
-    Preconditions.checkNotNull(getContext().getEntityType(),
-        "entityType shouldn't be null");
+    if (getContext() == null) {
+      throw new NullPointerException("context shouldn't be null");
+    }
+    if (getDataToRetrieve() == null) {
+      throw new NullPointerException("data to retrieve shouldn't be null");
+    }
+    if (getContext().getClusterId() == null) {
+      throw new NullPointerException("clusterId shouldn't be null");
+    }
+    if (getContext().getDoAsUser() == null) {
+      throw new NullPointerException("DoAsUser shouldn't be null");
+    }
+    if (getContext().getEntityType() == null) {
+      throw new NullPointerException("entityType shouldn't be null");
+    }
   }
 
   @Override

+ 8 - 0
pom.xml

@@ -254,6 +254,14 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x
                       <bannedImport>org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableListMultimap</bannedImport>
                     </bannedImports>
                   </restrictImports>
+                  <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
+                    <includeTestCode>true</includeTestCode>
+                    <reason>Use hadoop-common provided Preconditions rather than Guava provided</reason>
+                    <bannedImports>
+                      <bannedImport>org.apache.hadoop.thirdparty.com.google.common.base.Preconditions</bannedImport>
+                      <bannedImport>static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.**</bannedImport>
+                    </bannedImports>
+                  </restrictImports>
                 </rules>
               </configuration>
             </execution>