Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,7 @@
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;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@
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")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@
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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@
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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
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;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@

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;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@

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;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,7 @@
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.
Expand All @@ -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 {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,6 @@
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 {
Expand All @@ -46,24 +44,32 @@ public ApplicationIdProto getProto() {

@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));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,8 +58,6 @@
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.
Expand Down Expand Up @@ -336,21 +334,29 @@ protected Result getResult(Configuration hbaseConf, Connection conn,

@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");
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -115,11 +114,15 @@ public Set<String> readEntityTypes(Configuration hbaseConf,

@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");
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,8 +46,6 @@
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.
Expand Down Expand Up @@ -82,8 +80,10 @@ protected BaseTableRW<?> getTable() {

@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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,8 +57,6 @@
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.
Expand Down Expand Up @@ -86,18 +84,25 @@ protected BaseTableRW<?> getTable() {

@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) {
Expand Down
Loading