Skip to content

Commit

Permalink
[hotfix][test] Drop mockito usage from TestSingleInputGate
Browse files Browse the repository at this point in the history
  • Loading branch information
pnowojski committed May 10, 2019
1 parent 261efa8 commit 6a9d8f7
Show file tree
Hide file tree
Showing 2 changed files with 83 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@

import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createSingleInputGate;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.mockito.Mockito.spy;

/**
* A test input gate to mock reading data.
Expand All @@ -36,7 +35,7 @@ public class TestSingleInputGate {
public TestSingleInputGate(int numberOfInputChannels, boolean initialize) {
checkArgument(numberOfInputChannels >= 1);

inputGate = spy(createSingleInputGate(numberOfInputChannels));
inputGate = createSingleInputGate(numberOfInputChannels);
inputChannels = new TestInputChannel[numberOfInputChannels];

if (initialize) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,16 @@
package org.apache.flink.runtime.io.network.partition.consumer;

import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.event.AbstractEvent;
import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer;
import org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
import org.apache.flink.runtime.io.network.buffer.BufferListener;
import org.apache.flink.runtime.io.network.buffer.BufferPool;
import org.apache.flink.runtime.io.network.partition.consumer.InputChannel.BufferAndAvailability;
import org.apache.flink.runtime.io.network.partition.consumer.TestInputChannel.BufferAndAvailabilityProvider;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
Expand All @@ -40,7 +44,6 @@

import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.buildSingleBuffer;
import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createBufferBuilder;
import static org.mockito.Mockito.doReturn;

/**
* Test {@link InputGate} that allows setting multiple channels. Use
Expand Down Expand Up @@ -76,7 +79,7 @@ public StreamTestSingleInputGate(
inputQueues = new ConcurrentLinkedQueue[numInputChannels];

setupInputChannels();
doReturn(bufferSize).when(inputGate).getPageSize();
inputGate.setBufferPool(new NoOpBufferPool(bufferSize));
}

@SuppressWarnings("unchecked")
Expand Down Expand Up @@ -219,4 +222,81 @@ public boolean isEvent() {
return isEvent;
}
}

private static class NoOpBufferPool implements BufferPool {
private int bufferSize;

public NoOpBufferPool(int bufferSize) {
this.bufferSize = bufferSize;
}

@Override
public void lazyDestroy() {
}

@Override
public int getMemorySegmentSize() {
return bufferSize;
}

@Override
public Buffer requestBuffer() throws IOException {
throw new UnsupportedOperationException();
}

@Override
public Buffer requestBufferBlocking() throws IOException, InterruptedException {
throw new UnsupportedOperationException();
}

@Override
public BufferBuilder requestBufferBuilderBlocking() throws IOException, InterruptedException {
throw new UnsupportedOperationException();
}

@Override
public boolean addBufferListener(BufferListener listener) {
throw new UnsupportedOperationException();
}

@Override
public boolean isDestroyed() {
throw new UnsupportedOperationException();
}

@Override
public int getNumberOfRequiredMemorySegments() {
throw new UnsupportedOperationException();
}

@Override
public int getMaxNumberOfMemorySegments() {
throw new UnsupportedOperationException();
}

@Override
public int getNumBuffers() {
throw new UnsupportedOperationException();
}

@Override
public void setNumBuffers(int numBuffers) throws IOException {
throw new UnsupportedOperationException();
}

@Override
public int getNumberOfAvailableMemorySegments() {
throw new UnsupportedOperationException();
}

@Override
public int bestEffortGetNumOfUsedBuffers() {
throw new UnsupportedOperationException();
}

@Override
public void recycle(MemorySegment memorySegment) {
throw new UnsupportedOperationException();
}
}
}

0 comments on commit 6a9d8f7

Please sign in to comment.