Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-26536][BUILD][TEST] Upgrade Mockito to 2.23.4 #23452

Closed
wants to merge 3 commits into from
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 @@ -347,10 +347,10 @@ public void testRpcHandlerDelegate() throws Exception {
verify(handler).getStreamManager();

saslHandler.channelInactive(null);
verify(handler).channelInactive(any(TransportClient.class));
verify(handler).channelInactive(isNull());

saslHandler.exceptionCaught(null, null);
verify(handler).exceptionCaught(any(Throwable.class), any(TransportClient.class));
verify(handler).exceptionCaught(isNull(), isNull());
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
import org.mockito.ArgumentCaptor;

import static org.junit.Assert.*;
import static org.mockito.Matchers.any;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.*;

import org.apache.spark.network.buffer.ManagedBuffer;
Expand Down Expand Up @@ -79,6 +79,8 @@ public void testRegisterExecutor() {
@SuppressWarnings("unchecked")
@Test
public void testOpenShuffleBlocks() {
when(client.getClientId()).thenReturn("app0");
Copy link
Member Author

Choose a reason for hiding this comment

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

This UT uses app0 as String appId parameter.


RpcResponseCallback callback = mock(RpcResponseCallback.class);

ManagedBuffer block0Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[3]));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,10 +28,10 @@

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Matchers.anyLong;
import static org.mockito.Matchers.eq;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,8 +50,8 @@
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.mockito.Answers.RETURNS_SMART_NULLS;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyInt;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.Mockito.when;


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark

import scala.collection.mutable

import org.mockito.Matchers.{any, eq => meq}
import org.mockito.ArgumentMatchers.{any, eq => meq}
import org.mockito.Mockito.{mock, never, verify, when}
import org.scalatest.{BeforeAndAfter, PrivateMethodTester}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,7 @@ import scala.collection.mutable
import scala.concurrent.Future
import scala.concurrent.duration._

import org.mockito.Matchers
import org.mockito.Matchers._
import org.mockito.ArgumentMatchers.{any, eq => meq}
import org.mockito.Mockito.{mock, spy, verify, when}
import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester}

Expand Down Expand Up @@ -151,7 +150,7 @@ class HeartbeatReceiverSuite
heartbeatReceiverClock.advance(executorTimeout)
heartbeatReceiverRef.askSync[Boolean](ExpireDeadHosts)
// Only the second executor should be expired as a dead host
verify(scheduler).executorLost(Matchers.eq(executorId2), any())
verify(scheduler).executorLost(meq(executorId2), any())
val trackedExecutors = getTrackedExecutors
assert(trackedExecutors.size === 1)
assert(trackedExecutors.contains(executorId1))
Expand Down Expand Up @@ -223,10 +222,10 @@ class HeartbeatReceiverSuite
assert(!response.reregisterBlockManager)
// Additionally verify that the scheduler callback is called with the correct parameters
verify(scheduler).executorHeartbeatReceived(
Matchers.eq(executorId),
Matchers.eq(Array(1L -> metrics.accumulators())),
Matchers.eq(blockManagerId),
Matchers.eq(executorUpdates))
meq(executorId),
meq(Array(1L -> metrics.accumulators())),
meq(blockManagerId),
meq(executorUpdates))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark

import scala.collection.mutable.ArrayBuffer

import org.mockito.Matchers.any
import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito._

import org.apache.spark.LocalSparkContext._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.deploy
import scala.collection.mutable
import scala.concurrent.duration._

import org.mockito.Matchers.any
import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito.{mock, verify, when}
import org.scalatest.{BeforeAndAfterAll, PrivateMethodTester}
import org.scalatest.concurrent.Eventually._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import scala.collection.mutable

import com.codahale.metrics.Counter
import org.eclipse.jetty.servlet.ServletContextHandler
import org.mockito.Matchers._
import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito._
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ import org.apache.hadoop.hdfs.{DFSInputStream, DistributedFileSystem}
import org.apache.hadoop.security.AccessControlException
import org.json4s.jackson.JsonMethods._
import org.mockito.ArgumentMatcher
import org.mockito.Matchers.{any, argThat}
import org.mockito.ArgumentMatchers.{any, argThat}
import org.mockito.Mockito.{doThrow, mock, spy, verify, when}
import org.scalatest.BeforeAndAfter
import org.scalatest.Matchers
Expand Down Expand Up @@ -933,7 +933,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
val mockedFs = spy(provider.fs)
doThrow(new AccessControlException("Cannot read accessDenied file")).when(mockedFs).open(
argThat(new ArgumentMatcher[Path]() {
override def matches(path: Any): Boolean = {
override def matches(path: Path): Boolean = {
path.asInstanceOf[Path].getName.toLowerCase(Locale.ROOT) == "accessdenied"
}
}))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,8 @@ package org.apache.spark.deploy.history
import java.io.File

import org.mockito.AdditionalAnswers
import org.mockito.Matchers.{any, anyBoolean, anyLong, eq => meq}
import org.mockito.Mockito._
import org.mockito.ArgumentMatchers.{anyBoolean, anyLong, eq => meq}
import org.mockito.Mockito.{doAnswer, spy}
import org.scalatest.BeforeAndAfter

import org.apache.spark.{SparkConf, SparkFunSuite}
Expand All @@ -32,6 +32,8 @@ import org.apache.spark.util.kvstore.KVStore

class HistoryServerDiskManagerSuite extends SparkFunSuite with BeforeAndAfter {

private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*)

private val MAX_USAGE = 3L

private var testDir: File = _
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import java.io.File

import scala.concurrent.duration._

import org.mockito.Matchers._
import org.mockito.ArgumentMatchers.{any, anyInt}
import org.mockito.Mockito._
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import java.util.function.Supplier

import org.mockito.{Mock, MockitoAnnotations}
import org.mockito.Answers.RETURNS_SMART_NULLS
import org.mockito.Matchers._
import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito._
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import scala.concurrent.duration._
import scala.language.postfixOps

import org.mockito.ArgumentCaptor
import org.mockito.Matchers.{any, eq => meq}
import org.mockito.ArgumentMatchers.{any, eq => meq}
import org.mockito.Mockito.{inOrder, verify, when}
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import scala.collection.mutable
import scala.concurrent.{ExecutionContext, Future}
import scala.concurrent.duration.Duration

import org.mockito.Matchers.{any, anyLong}
import org.mockito.ArgumentMatchers.{any, anyLong}
import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS}
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
Expand Down
2 changes: 1 addition & 1 deletion core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import scala.concurrent.duration._
import scala.language.postfixOps

import com.google.common.io.Files
import org.mockito.Matchers.any
import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito.{mock, never, verify, when}
import org.scalatest.BeforeAndAfterAll
import org.scalatest.concurrent.Eventually._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import java.net.InetSocketAddress
import java.nio.ByteBuffer

import io.netty.channel.Channel
import org.mockito.Matchers._
import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito._

import org.apache.spark.SparkFunSuite
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.scheduler

import org.mockito.Matchers.any
import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito.{never, verify, when}
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream,

import scala.util.Random

import org.mockito.Mockito._
import org.mockito.Mockito.mock
import org.roaringbitmap.RoaringBitmap

import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkFunSuite}
Expand All @@ -31,6 +31,7 @@ import org.apache.spark.serializer.{JavaSerializer, KryoSerializer}
import org.apache.spark.storage.BlockManagerId

class MapStatusSuite extends SparkFunSuite {
private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*)

test("compressSize") {
assert(MapStatus.compressSize(0L) === 0)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,8 @@ import scala.language.postfixOps

import org.apache.hadoop.mapred._
import org.apache.hadoop.mapreduce.TaskType
import org.mockito.Matchers
import org.mockito.Mockito._
import org.mockito.ArgumentMatchers.{any, eq => meq}
import org.mockito.Mockito.{doAnswer, spy, times, verify}
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
import org.scalatest.BeforeAndAfter
Expand Down Expand Up @@ -71,6 +71,8 @@ import org.apache.spark.util.{ThreadUtils, Utils}
*/
class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter {

private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*)

var outputCommitCoordinator: OutputCommitCoordinator = null
var tempDir: File = null
var sc: SparkContext = null
Expand Down Expand Up @@ -103,7 +105,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter {
invoke.callRealMethod()
mockTaskScheduler.backend.reviveOffers()
}
}).when(mockTaskScheduler).submitTasks(Matchers.any())
}).when(mockTaskScheduler).submitTasks(any())

doAnswer(new Answer[TaskSetManager]() {
override def answer(invoke: InvocationOnMock): TaskSetManager = {
Expand All @@ -123,7 +125,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter {
}
}
}
}).when(mockTaskScheduler).createTaskSetManager(Matchers.any(), Matchers.any())
}).when(mockTaskScheduler).createTaskSetManager(any(), any())

sc.taskScheduler = mockTaskScheduler
val dagSchedulerWithMockTaskScheduler = new DAGScheduler(sc, mockTaskScheduler)
Expand Down Expand Up @@ -154,7 +156,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter {
test("Job should not complete if all commits are denied") {
// Create a mock OutputCommitCoordinator that denies all attempts to commit
doReturn(false).when(outputCommitCoordinator).handleAskPermissionToCommit(
Matchers.any(), Matchers.any(), Matchers.any(), Matchers.any())
any(), any(), any(), any())
val rdd: RDD[Int] = sc.parallelize(Seq(1), 1)
def resultHandler(x: Int, y: Unit): Unit = {}
val futureAction: SimpleFutureAction[Unit] = sc.submitJob[Int, Unit, Unit](rdd,
Expand Down Expand Up @@ -268,8 +270,8 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter {
assert(retriedStage.size === 1)
assert(sc.dagScheduler.outputCommitCoordinator.isEmpty)
verify(sc.env.outputCommitCoordinator, times(2))
.stageStart(Matchers.eq(retriedStage.head), Matchers.any())
verify(sc.env.outputCommitCoordinator).stageEnd(Matchers.eq(retriedStage.head))
.stageStart(meq(retriedStage.head), any())
verify(sc.env.outputCommitCoordinator).stageEnd(meq(retriedStage.head))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.scheduler

import java.util.Properties

import org.mockito.Matchers.any
import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito._
import org.scalatest.BeforeAndAfter

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import scala.util.control.NonFatal

import com.google.common.util.concurrent.MoreExecutors
import org.mockito.ArgumentCaptor
import org.mockito.Matchers.{any, anyLong}
import org.mockito.ArgumentMatchers.{any, anyLong}
import org.mockito.Mockito.{spy, times, verify}
import org.scalatest.BeforeAndAfter
import org.scalatest.concurrent.Eventually._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import java.nio.ByteBuffer
import scala.collection.mutable.HashMap
import scala.concurrent.duration._

import org.mockito.Matchers.{anyInt, anyObject, anyString, eq => meq}
import org.mockito.ArgumentMatchers.{any, anyInt, anyString, eq => meq}
import org.mockito.Mockito.{atLeast, atMost, never, spy, times, verify, when}
import org.scalatest.BeforeAndAfterEach
import org.scalatest.concurrent.Eventually
Expand Down Expand Up @@ -430,7 +430,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
verify(blacklist, never).updateBlacklistForSuccessfulTaskSet(
stageId = meq(2),
stageAttemptId = anyInt(),
failuresByExec = anyObject())
failuresByExec = any())
}

test("scheduled tasks obey node and executor blacklists") {
Expand Down Expand Up @@ -504,7 +504,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
WorkerOffer("executor3", "host1", 2)
)).flatten.size === 0)
assert(tsm.isZombie)
verify(tsm).abort(anyString(), anyObject())
verify(tsm).abort(anyString(), any())
}

test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
Expand Down Expand Up @@ -1184,7 +1184,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
assert(finalTsm.isZombie)

// no taskset has completed all of its tasks, so no updates to the blacklist tracker yet
verify(blacklist, never).updateBlacklistForSuccessfulTaskSet(anyInt(), anyInt(), anyObject())
verify(blacklist, never).updateBlacklistForSuccessfulTaskSet(anyInt(), anyInt(), any())

// finally, lets complete all the tasks. We simulate failures in attempt 1, but everything
// else succeeds, to make sure we get the right updates to the blacklist in all cases.
Expand All @@ -1202,7 +1202,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
// we update the blacklist for the stage attempts with all successful tasks. Even though
// some tasksets had failures, we still consider them all successful from a blacklisting
// perspective, as the failures weren't from a problem w/ the tasks themselves.
verify(blacklist).updateBlacklistForSuccessfulTaskSet(meq(0), meq(stageAttempt), anyObject())
verify(blacklist).updateBlacklistForSuccessfulTaskSet(meq(0), meq(stageAttempt), any())
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
*/
package org.apache.spark.scheduler

import org.mockito.Matchers.isA
import org.mockito.ArgumentMatchers.isA
import org.mockito.Mockito.{never, verify}
import org.scalatest.BeforeAndAfterEach
import org.scalatest.mockito.MockitoSugar
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import java.util.{Properties, Random}
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer

import org.mockito.Matchers.{any, anyInt, anyString}
import org.mockito.ArgumentMatchers.{any, anyInt, anyString}
import org.mockito.Mockito.{mock, never, spy, times, verify, when}
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
Expand Down Expand Up @@ -1319,7 +1319,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
when(taskSetManagerSpy.addPendingTask(anyInt())).thenAnswer(
new Answer[Unit] {
override def answer(invocationOnMock: InvocationOnMock): Unit = {
val task = invocationOnMock.getArgumentAt(0, classOf[Int])
val task: Int = invocationOnMock.getArgument(0)
assert(taskSetManager.taskSetBlacklistHelperOpt.get.
isExecutorBlacklistedForTask(exec, task))
}
Expand Down
Loading