Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
15 commits
Select commit Hold shift + click to select a range
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
302 changes: 135 additions & 167 deletions R/pkg/R/functions.R

Large diffs are not rendered by default.

42 changes: 28 additions & 14 deletions R/pkg/R/generics.R
Original file line number Diff line number Diff line change
Expand Up @@ -913,8 +913,9 @@ setGeneric("add_months", function(y, x) { standardGeneric("add_months") })
#' @name NULL
setGeneric("approxCountDistinct", function(x, ...) { standardGeneric("approxCountDistinct") })

#' @rdname array_contains
#' @rdname column_collection_functions
#' @export
#' @name NULL
setGeneric("array_contains", function(x, value) { standardGeneric("array_contains") })

#' @rdname column_string_functions
Expand Down Expand Up @@ -992,8 +993,9 @@ setGeneric("conv", function(x, fromBase, toBase) { standardGeneric("conv") })
#' @name NULL
setGeneric("countDistinct", function(x, ...) { standardGeneric("countDistinct") })

#' @rdname crc32
#' @rdname column_misc_functions
#' @export
#' @name NULL
setGeneric("crc32", function(x) { standardGeneric("crc32") })

#' @rdname column_nonaggregate_functions
Expand All @@ -1006,8 +1008,9 @@ setGeneric("create_array", function(x, ...) { standardGeneric("create_array") })
#' @name NULL
setGeneric("create_map", function(x, ...) { standardGeneric("create_map") })

#' @rdname hash
#' @rdname column_misc_functions
#' @export
#' @name NULL
setGeneric("hash", function(x, ...) { standardGeneric("hash") })

#' @param x empty. Should be used with no argument.
Expand Down Expand Up @@ -1060,12 +1063,14 @@ setGeneric("dense_rank", function(x = "missing") { standardGeneric("dense_rank")
#' @name NULL
setGeneric("encode", function(x, charset) { standardGeneric("encode") })

#' @rdname explode
#' @rdname column_collection_functions
#' @export
#' @name NULL
setGeneric("explode", function(x) { standardGeneric("explode") })

#' @rdname explode_outer
#' @rdname column_collection_functions
#' @export
#' @name NULL
setGeneric("explode_outer", function(x) { standardGeneric("explode_outer") })

#' @rdname column_nonaggregate_functions
Expand All @@ -1088,8 +1093,9 @@ setGeneric("format_number", function(y, x) { standardGeneric("format_number") })
#' @name NULL
setGeneric("format_string", function(format, x, ...) { standardGeneric("format_string") })

#' @rdname from_json
#' @rdname column_collection_functions
#' @export
#' @name NULL
setGeneric("from_json", function(x, schema, ...) { standardGeneric("from_json") })

#' @rdname column_datetime_functions
Expand Down Expand Up @@ -1205,8 +1211,9 @@ setGeneric("lpad", function(x, len, pad) { standardGeneric("lpad") })
#' @name NULL
setGeneric("ltrim", function(x) { standardGeneric("ltrim") })

#' @rdname md5
#' @rdname column_misc_functions
#' @export
#' @name NULL
setGeneric("md5", function(x) { standardGeneric("md5") })

#' @rdname column_datetime_functions
Expand Down Expand Up @@ -1272,12 +1279,14 @@ setGeneric("percent_rank", function(x = "missing") { standardGeneric("percent_ra
#' @name NULL
setGeneric("pmod", function(y, x) { standardGeneric("pmod") })

#' @rdname posexplode
#' @rdname column_collection_functions
#' @export
#' @name NULL
setGeneric("posexplode", function(x) { standardGeneric("posexplode") })

#' @rdname posexplode_outer
#' @rdname column_collection_functions
#' @export
#' @name NULL
setGeneric("posexplode_outer", function(x) { standardGeneric("posexplode_outer") })

#' @rdname column_datetime_functions
Expand Down Expand Up @@ -1350,12 +1359,14 @@ setGeneric("sd", function(x, na.rm = FALSE) { standardGeneric("sd") })
#' @name NULL
setGeneric("second", function(x) { standardGeneric("second") })

#' @rdname sha1
#' @rdname column_misc_functions
#' @export
#' @name NULL
setGeneric("sha1", function(x) { standardGeneric("sha1") })

#' @rdname sha2
#' @rdname column_misc_functions
#' @export
#' @name NULL
setGeneric("sha2", function(y, x) { standardGeneric("sha2") })

#' @rdname column_math_functions
Expand All @@ -1378,17 +1389,19 @@ setGeneric("shiftRightUnsigned", function(y, x) { standardGeneric("shiftRightUns
#' @name NULL
setGeneric("signum", function(x) { standardGeneric("signum") })

#' @rdname size
#' @rdname column_collection_functions
#' @export
#' @name NULL
setGeneric("size", function(x) { standardGeneric("size") })

#' @rdname column_aggregate_functions
#' @export
#' @name NULL
setGeneric("skewness", function(x) { standardGeneric("skewness") })

#' @rdname sort_array
#' @rdname column_collection_functions
#' @export
#' @name NULL
setGeneric("sort_array", function(x, asc = TRUE) { standardGeneric("sort_array") })

#' @rdname column_string_functions
Expand Down Expand Up @@ -1451,8 +1464,9 @@ setGeneric("toRadians", function(x) { standardGeneric("toRadians") })
#' @name NULL
setGeneric("to_date", function(x, format) { standardGeneric("to_date") })

#' @rdname to_json
#' @rdname column_collection_functions
#' @export
#' @name NULL
setGeneric("to_json", function(x, ...) { standardGeneric("to_json") })

#' @rdname column_datetime_functions
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -179,7 +179,7 @@ public void stream(String streamId, StreamCallback callback) {
// written to the socket atomically, so that callbacks are called in the right order
// when responses arrive.
synchronized (this) {
handler.addStreamCallback(callback);
handler.addStreamCallback(streamId, callback);
channel.writeAndFlush(new StreamRequest(streamId)).addListener(future -> {
if (future.isSuccess()) {
long timeTaken = System.currentTimeMillis() - startTime;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.atomic.AtomicLong;

import scala.Tuple2;

import com.google.common.annotations.VisibleForTesting;
import io.netty.channel.Channel;
import org.slf4j.Logger;
Expand Down Expand Up @@ -56,7 +58,7 @@ public class TransportResponseHandler extends MessageHandler<ResponseMessage> {

private final Map<Long, RpcResponseCallback> outstandingRpcs;

private final Queue<StreamCallback> streamCallbacks;
private final Queue<Tuple2<String, StreamCallback>> streamCallbacks;
private volatile boolean streamActive;

/** Records the time (in system nanoseconds) that the last fetch or RPC request was sent. */
Expand Down Expand Up @@ -88,9 +90,9 @@ public void removeRpcRequest(long requestId) {
outstandingRpcs.remove(requestId);
}

public void addStreamCallback(StreamCallback callback) {
public void addStreamCallback(String streamId, StreamCallback callback) {
timeOfLastRequestNs.set(System.nanoTime());
streamCallbacks.offer(callback);
streamCallbacks.offer(new Tuple2<>(streamId, callback));
}

@VisibleForTesting
Expand All @@ -104,15 +106,31 @@ public void deactivateStream() {
*/
private void failOutstandingRequests(Throwable cause) {
for (Map.Entry<StreamChunkId, ChunkReceivedCallback> entry : outstandingFetches.entrySet()) {
entry.getValue().onFailure(entry.getKey().chunkIndex, cause);
try {
entry.getValue().onFailure(entry.getKey().chunkIndex, cause);
} catch (Exception e) {
logger.warn("ChunkReceivedCallback.onFailure throws exception", e);
}
}
for (Map.Entry<Long, RpcResponseCallback> entry : outstandingRpcs.entrySet()) {
entry.getValue().onFailure(cause);
try {
entry.getValue().onFailure(cause);
} catch (Exception e) {
logger.warn("RpcResponseCallback.onFailure throws exception", e);
}
}
for (Tuple2<String, StreamCallback> entry : streamCallbacks) {
try {
entry._2().onFailure(entry._1(), cause);
} catch (Exception e) {
logger.warn("StreamCallback.onFailure throws exception", e);
}
}

// It's OK if new fetches appear, as they will fail immediately.
outstandingFetches.clear();
outstandingRpcs.clear();
streamCallbacks.clear();
}

@Override
Expand Down Expand Up @@ -190,8 +208,9 @@ public void handle(ResponseMessage message) throws Exception {
}
} else if (message instanceof StreamResponse) {
StreamResponse resp = (StreamResponse) message;
StreamCallback callback = streamCallbacks.poll();
if (callback != null) {
Tuple2<String, StreamCallback> entry = streamCallbacks.poll();
if (entry != null) {
StreamCallback callback = entry._2();
if (resp.byteCount > 0) {
StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
callback);
Expand All @@ -216,8 +235,9 @@ public void handle(ResponseMessage message) throws Exception {
}
} else if (message instanceof StreamFailure) {
StreamFailure resp = (StreamFailure) message;
StreamCallback callback = streamCallbacks.poll();
if (callback != null) {
Tuple2<String, StreamCallback> entry = streamCallbacks.poll();
if (entry != null) {
StreamCallback callback = entry._2();
try {
callback.onFailure(resp.streamId, new RuntimeException(resp.error));
} catch (IOException ioe) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.network;

import java.io.IOException;
import java.nio.ByteBuffer;

import io.netty.channel.Channel;
Expand Down Expand Up @@ -127,17 +128,43 @@ public void testActiveStreams() throws Exception {

StreamResponse response = new StreamResponse("stream", 1234L, null);
StreamCallback cb = mock(StreamCallback.class);
handler.addStreamCallback(cb);
handler.addStreamCallback("stream", cb);
assertEquals(1, handler.numOutstandingRequests());
handler.handle(response);
assertEquals(1, handler.numOutstandingRequests());
handler.deactivateStream();
assertEquals(0, handler.numOutstandingRequests());

StreamFailure failure = new StreamFailure("stream", "uh-oh");
handler.addStreamCallback(cb);
handler.addStreamCallback("stream", cb);
assertEquals(1, handler.numOutstandingRequests());
handler.handle(failure);
assertEquals(0, handler.numOutstandingRequests());
}

@Test
public void failOutstandingStreamCallbackOnClose() throws Exception {
Channel c = new LocalChannel();
c.pipeline().addLast(TransportFrameDecoder.HANDLER_NAME, new TransportFrameDecoder());
TransportResponseHandler handler = new TransportResponseHandler(c);

StreamCallback cb = mock(StreamCallback.class);
handler.addStreamCallback("stream-1", cb);
handler.channelInactive();

verify(cb).onFailure(eq("stream-1"), isA(IOException.class));
}

@Test
public void failOutstandingStreamCallbackOnException() throws Exception {
Channel c = new LocalChannel();
c.pipeline().addLast(TransportFrameDecoder.HANDLER_NAME, new TransportFrameDecoder());
TransportResponseHandler handler = new TransportResponseHandler(c);

StreamCallback cb = mock(StreamCallback.class);
handler.addStreamCallback("stream-1", cb);
handler.exceptionCaught(new IOException("Oops!"));

verify(cb).onFailure(eq("stream-1"), isA(IOException.class));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1088,6 +1088,12 @@ public UTF8String clone() {
return fromBytes(getBytes());
}

public UTF8String copy() {
byte[] bytes = new byte[numBytes];
copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes);
return fromBytes(bytes);
}

@Override
public int compareTo(@Nonnull final UTF8String other) {
int len = Math.min(numBytes, other.numBytes);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -160,14 +160,10 @@ public final class BytesToBytesMap extends MemoryConsumer {

private final boolean enablePerfMetrics;

private long timeSpentResizingNs = 0;

private long numProbes = 0;

private long numKeyLookups = 0;

private long numHashCollisions = 0;

private long peakMemoryUsedBytes = 0L;

private final int initialCapacity;
Expand Down Expand Up @@ -489,10 +485,6 @@ public void safeLookup(Object keyBase, long keyOffset, int keyLength, Location l
);
if (areEqual) {
return;
} else {
if (enablePerfMetrics) {
numHashCollisions++;
}
}
}
}
Expand Down Expand Up @@ -859,16 +851,6 @@ public long getPeakMemoryUsedBytes() {
return peakMemoryUsedBytes;
}

/**
* Returns the total amount of time spent resizing this map (in nanoseconds).
*/
public long getTimeSpentResizingNs() {
if (!enablePerfMetrics) {
throw new IllegalStateException();
}
return timeSpentResizingNs;
}

/**
* Returns the average number of probes per key lookup.
*/
Expand All @@ -879,13 +861,6 @@ public double getAverageProbesPerLookup() {
return (1.0 * numProbes) / numKeyLookups;
}

public long getNumHashCollisions() {
if (!enablePerfMetrics) {
throw new IllegalStateException();
}
return numHashCollisions;
}

@VisibleForTesting
public int getNumDataPages() {
return dataPages.size();
Expand Down Expand Up @@ -923,10 +898,6 @@ public void reset() {
void growAndRehash() {
assert(longArray != null);

long resizeStartTime = -1;
if (enablePerfMetrics) {
resizeStartTime = System.nanoTime();
}
// Store references to the old data structures to be used when we re-hash
final LongArray oldLongArray = longArray;
final int oldCapacity = (int) oldLongArray.size() / 2;
Expand All @@ -951,9 +922,5 @@ void growAndRehash() {
longArray.set(newPos * 2 + 1, hashcode);
}
freeArray(oldLongArray);

if (enablePerfMetrics) {
timeSpentResizingNs += System.nanoTime() - resizeStartTime;
}
}
}
Loading