Skip to content
Closed
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,14 +24,14 @@
import java.util.HashSet;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.Objects;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.function.BiConsumer;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;

import com.google.common.base.Objects;
import com.google.common.base.Preconditions;

import org.apache.spark.annotation.Private;
Expand Down Expand Up @@ -70,7 +70,7 @@ public long count(Class<?> type, String index, Object indexedValue) throws Excep
Object comparable = asKey(indexedValue);
KVTypeInfo.Accessor accessor = list.getIndexAccessor(index);
for (Object o : view(type)) {
if (Objects.equal(comparable, asKey(accessor.get(o)))) {
if (Objects.equals(comparable, asKey(accessor.get(o)))) {
count++;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.network.protocol;

import com.google.common.base.Objects;
import java.util.Objects;

import org.apache.spark.network.buffer.ManagedBuffer;

Expand Down Expand Up @@ -48,7 +48,7 @@ public boolean isBodyInFrame() {
}

protected boolean equals(AbstractMessage other) {
return isBodyInFrame == other.isBodyInFrame && Objects.equal(body, other.body);
return isBodyInFrame == other.isBodyInFrame && Objects.equals(body, other.body);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -32,13 +32,13 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.Executors;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
Expand All @@ -47,6 +47,8 @@
import com.google.common.collect.Maps;
import com.google.common.primitives.Ints;
import com.google.common.primitives.Longs;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;
import org.roaringbitmap.RoaringBitmap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -760,19 +762,19 @@ public boolean equals(Object o) {
return false;
}
AppShuffleId that = (AppShuffleId) o;
return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
return shuffleId == that.shuffleId && Objects.equals(appId, that.appId);
}

@Override
public int hashCode() {
return Objects.hashCode(appId, shuffleId);
return Objects.hash(appId, shuffleId);
}

@Override
public String toString() {
return Objects.toStringHelper(this)
.add("appId", appId)
.add("shuffleId", shuffleId)
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It isn't super important here I think, but does this result in the same string?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The results are different. Using Objects to return is like "AppShuffleId{appId=appId, shuffleId=100}", using ToStringBuilder to return is like "RemoteBlockPushResolver.AppShuffleId[appId=appId,shuffleId=100]". Will it cause some problems?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Like with hash function changes, it shouldn't matter to programs. But if some program did rely on it, directly or accidentally, this might break. It's a tough call - how much is the change worth? overall it's an OK improvement but yeah I'm hesitant for just this reason. It's more the hash change than this one.

.append("appId", appId)
.append("shuffleId", shuffleId)
.toString();
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,11 @@

package org.apache.spark.network.shuffle.protocol;

import com.google.common.base.Objects;
import java.util.Objects;

import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

import org.apache.spark.network.protocol.Encoders;

Expand Down Expand Up @@ -46,22 +49,22 @@ protected BlockTransferMessage.Type type() {

@Override
public int hashCode() {
return Objects.hashCode(appId, shuffleId);
return Objects.hash(appId, shuffleId);
}

@Override
public String toString() {
return Objects.toStringHelper(this)
.add("appId", appId)
.add("shuffleId", shuffleId)
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("appId", appId)
.append("shuffleId", shuffleId)
.toString();
}

@Override
public boolean equals(Object other) {
if (other != null && other instanceof FinalizeShuffleMerge) {
FinalizeShuffleMerge o = (FinalizeShuffleMerge) other;
return Objects.equal(appId, o.appId)
return Objects.equals(appId, o.appId)
&& shuffleId == o.shuffleId;
}
return false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,11 @@
package org.apache.spark.network.shuffle.protocol;

import java.util.Arrays;
import java.util.Objects;

import com.google.common.base.Objects;
import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;
import org.roaringbitmap.RoaringBitmap;

import org.apache.spark.network.protocol.Encoders;
Expand Down Expand Up @@ -69,24 +71,24 @@ protected Type type() {

@Override
public int hashCode() {
int objectHashCode = Objects.hashCode(shuffleId);
int objectHashCode = Objects.hash(shuffleId);
return (objectHashCode * 41 + Arrays.hashCode(reduceIds) * 41
+ Arrays.hashCode(bitmaps) * 41 + Arrays.hashCode(sizes));
}

@Override
public String toString() {
return Objects.toStringHelper(this)
.add("shuffleId", shuffleId)
.add("reduceId size", reduceIds.length)
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("shuffleId", shuffleId)
.append("reduceId size", reduceIds.length)
.toString();
}

@Override
public boolean equals(Object other) {
if (other != null && other instanceof MergeStatuses) {
MergeStatuses o = (MergeStatuses) other;
return Objects.equal(shuffleId, o.shuffleId)
return Objects.equals(shuffleId, o.shuffleId)
&& Arrays.equals(bitmaps, o.bitmaps)
&& Arrays.equals(reduceIds, o.reduceIds)
&& Arrays.equals(sizes, o.sizes);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,11 @@

package org.apache.spark.network.shuffle.protocol;

import com.google.common.base.Objects;
import java.util.Objects;

import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

import org.apache.spark.network.protocol.Encoders;

Expand Down Expand Up @@ -54,25 +57,25 @@ protected Type type() {

@Override
public int hashCode() {
return Objects.hashCode(appId, shuffleId, mapIndex , reduceId, index);
return Objects.hash(appId, shuffleId, mapIndex , reduceId, index);
}

@Override
public String toString() {
return Objects.toStringHelper(this)
.add("appId", appId)
.add("shuffleId", shuffleId)
.add("mapIndex", mapIndex)
.add("reduceId", reduceId)
.add("index", index)
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("appId", appId)
.append("shuffleId", shuffleId)
.append("mapIndex", mapIndex)
.append("reduceId", reduceId)
.append("index", index)
.toString();
}

@Override
public boolean equals(Object other) {
if (other != null && other instanceof PushBlockStream) {
PushBlockStream o = (PushBlockStream) other;
return Objects.equal(appId, o.appId)
return Objects.equals(appId, o.appId)
&& shuffleId == o.shuffleId
&& mapIndex == o.mapIndex
&& reduceId == o.reduceId
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.spark.network.shuffle.protocol.mesos;

import com.google.common.base.Objects;
import io.netty.buffer.ByteBuf;

import org.apache.spark.network.protocol.Encoders;
Expand All @@ -26,6 +25,8 @@
// Needed by ScalaDoc. See SPARK-7726
import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type;

import java.util.Objects;

/**
* A message sent from the driver to register with the MesosExternalShuffleService.
*/
Expand Down Expand Up @@ -58,15 +59,15 @@ public void encode(ByteBuf buf) {

@Override
public int hashCode() {
return Objects.hashCode(appId, heartbeatTimeoutMs);
return Objects.hash(appId, heartbeatTimeoutMs);
}

@Override
public boolean equals(Object o) {
if (!(o instanceof RegisterDriver)) {
return false;
}
return Objects.equal(appId, ((RegisterDriver) o).appId);
return Objects.equals(appId, ((RegisterDriver) o).appId);
}

public static RegisterDriver decode(ByteBuf buf) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,13 @@

package org.apache.spark.rdd

import java.util.Objects
import java.util.concurrent.atomic.AtomicInteger

import com.fasterxml.jackson.annotation.{JsonIgnore, JsonInclude, JsonPropertyOrder}
import com.fasterxml.jackson.annotation.JsonInclude.Include
import com.fasterxml.jackson.databind.ObjectMapper
import com.fasterxml.jackson.module.scala.DefaultScalaModule
import com.google.common.base.Objects

import org.apache.spark.SparkContext
import org.apache.spark.internal.Logging
Expand Down Expand Up @@ -69,7 +69,7 @@ private[spark] class RDDOperationScope(
}
}

override def hashCode(): Int = Objects.hashCode(id, name, parent)
override def hashCode(): Int = Objects.hash(id, name, parent)

override def toString: String = toJson
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.mllib.linalg

import java.util.{Arrays, Random}
import java.util.{Arrays, Objects, Random}

import scala.collection.mutable.{ArrayBuffer, ArrayBuilder => MArrayBuilder, HashSet => MHashSet}
import scala.language.implicitConversions
Expand Down Expand Up @@ -313,7 +313,7 @@ class DenseMatrix @Since("1.3.0") (
}

override def hashCode: Int = {
com.google.common.base.Objects.hashCode(numRows: Integer, numCols: Integer, toArray)
Objects.hash(numRows: Integer, numCols: Integer, toArray)
}

private[mllib] def asBreeze: BM[Double] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.spark.mllib.linalg.distributed

import java.util.Objects

import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, Matrix => BM}
import scala.collection.mutable.ArrayBuffer

Expand Down Expand Up @@ -90,7 +92,7 @@ private[mllib] class GridPartitioner(
}

override def hashCode: Int = {
com.google.common.base.Objects.hashCode(
Objects.hash(
rows: java.lang.Integer,
cols: java.lang.Integer,
rowsPerPart: java.lang.Integer,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.spark.mllib.tree.model

import java.util.Objects

import org.apache.spark.annotation.Since
import org.apache.spark.mllib.tree.impurity.ImpurityCalculator

Expand Down Expand Up @@ -56,7 +58,7 @@ class InformationGainStats(
}

override def hashCode: Int = {
com.google.common.base.Objects.hashCode(
Objects.hash(
gain: java.lang.Double,
impurity: java.lang.Double,
leftImpurity: java.lang.Double,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.spark.mllib.tree.model

import java.util.Objects

import org.apache.spark.annotation.Since

/**
Expand All @@ -39,6 +41,6 @@ class Predict @Since("1.2.0") (
}

override def hashCode: Int = {
com.google.common.base.Objects.hashCode(predict: java.lang.Double, prob: java.lang.Double)
Objects.hash(predict: java.lang.Double, prob: java.lang.Double)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,15 +22,14 @@ import java.net.{InetAddress, UnknownHostException, URI}
import java.nio.ByteBuffer
import java.nio.charset.StandardCharsets
import java.nio.file.Files
import java.util.{Locale, Properties, UUID}
import java.util.{Locale, Objects, Properties, UUID}
import java.util.zip.{ZipEntry, ZipOutputStream}

import scala.collection.JavaConverters._
import scala.collection.immutable.{Map => IMap}
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map}
import scala.util.control.NonFatal

import com.google.common.base.Objects
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs._
import org.apache.hadoop.fs.permission.FsPermission
Expand Down Expand Up @@ -1567,7 +1566,7 @@ private object Client extends Logging {
}
}

Objects.equal(srcHost, dstHost) && srcUri.getPort() == dstUri.getPort()
Objects.equals(srcHost, dstHost) && srcUri.getPort() == dstUri.getPort()

}

Expand Down
Loading