Skip to content
Open
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 @@ -3,6 +3,8 @@
import com.spbsu.flamestream.core.Batch;
import com.spbsu.flamestream.core.data.meta.GlobalTime;

import java.util.concurrent.CompletionStage;

/**
* User: Artem
* Date: 27.11.2017
Expand All @@ -15,9 +17,9 @@ class MinTime {
}

/**
* Sync call, return mean accept
* Sync call, completion means accept
*/
void accept(Batch batch);
CompletionStage<?> accept(Batch batch);

Batch last();
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,8 +42,8 @@ public static Props props(EdgeContext edgeContext, FlameRuntime.RearInstance<?>
public Receive createReceive() {
return ReceiveBuilder.create()
.match(Batch.class, b -> {
rear.accept(b);
sender().tell(new BatchAccepted(), self());
final ActorRef sender = sender(), self = self();
rear.accept(b).thenRun(() -> sender.tell(new BatchAccepted(), self));
})
.match(GimmeLastBatch.class, l -> sender().tell(rear.last(), ActorRef.noSender()))
.build();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
import com.spbsu.flamestream.runtime.utils.akka.AwaitResolver;
import com.spbsu.flamestream.runtime.utils.akka.LoggingActor;

import java.util.concurrent.CompletionStage;
import java.util.concurrent.ExecutionException;
import java.util.function.Consumer;

Expand All @@ -30,12 +31,8 @@ public AkkaRear(EdgeContext edgeContext, ActorRefFactory refFactory, String loca
}

@Override
public void accept(Batch batch) {
try {
PatternsCS.ask(innerActor, batch, FlameConfig.config.smallTimeout()).toCompletableFuture().get();
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
}
public CompletionStage<?> accept(Batch batch) {
return PatternsCS.ask(innerActor, batch, FlameConfig.config.smallTimeout());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,25 +5,30 @@
import com.esotericsoftware.kryonet.Connection;
import com.esotericsoftware.kryonet.Listener;
import com.spbsu.flamestream.core.Batch;
import com.spbsu.flamestream.runtime.edge.Rear;
import com.spbsu.flamestream.core.DataItem;
import com.spbsu.flamestream.core.data.PayloadDataItem;
import com.spbsu.flamestream.core.data.meta.EdgeId;
import com.spbsu.flamestream.core.data.meta.GlobalTime;
import com.spbsu.flamestream.core.data.meta.Meta;
import com.spbsu.flamestream.runtime.edge.EdgeContext;
import com.spbsu.flamestream.runtime.edge.Rear;
import org.objenesis.strategy.StdInstantiatorStrategy;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.util.Arrays;
import java.util.Iterator;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage;

/**
* User: Artem
* Date: 29.12.2017
*/
public class SocketRear implements Rear {
private static final Logger LOG = LoggerFactory.getLogger(SocketRear.class);
public static final int BUFFER_SIZE = 1_000_000;

private final EdgeId edgeId;
private final Client client;
Expand All @@ -32,7 +37,7 @@ public class SocketRear implements Rear {

public SocketRear(EdgeContext edgeContext, String host, int port, Class[] classes) {
edgeId = edgeContext.edgeId();
client = new Client(1_000_000, 1234);
client = new Client(BUFFER_SIZE, 1234);
Arrays.stream(classes).forEach(clazz -> client.getKryo().register(clazz));
{ //register inners of data item
client.getKryo().register(PayloadDataItem.class);
Expand Down Expand Up @@ -64,19 +69,43 @@ public void disconnected(Connection connection) {
}

@Override
public void accept(Batch batch) {
public CompletionStage<?> accept(Batch batch) {
final CompletableFuture<Void> future = new CompletableFuture<>();
if (client.isConnected()) {
batch.payload().forEach(client::sendTCP);
if (batch.payload().noneMatch(ignored -> true)) {
client.sendTCP(new MinTime(batch.time()));
final Iterator<DataItem> iterator = batch.payload().iterator();
if (writeAllUntilFull(iterator)) {
future.complete(null);
} else {
client.addListener(new Listener() {
@Override
public void idle(Connection connection) {
if (writeAllUntilFull(iterator)) {
future.complete(null);
client.removeListener(this);
}
}
});
}
} else {
LOG.warn("{}: writing to closed log", edgeId);
future.complete(null);
}
return future;
}

@Override
public Batch last() {
return Batch.Default.EMPTY;
}

private boolean writeAllUntilFull(Iterator<DataItem> iterator) {
while (iterator.hasNext()) {
if (client.getTcpWriteBufferSize() < BUFFER_SIZE * 0.9) {
client.sendTCP(iterator.next());
} else {
return false;
}
}
return true;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,11 +34,15 @@ public int expected() {
}

public void await(long timeout, TimeUnit unit) throws InterruptedException {
final long stop = System.currentTimeMillis() + unit.toMillis(timeout);
synchronized (counter) {
while (counter.longValue() < expectedSize && System.currentTimeMillis() < stop) {
while (counter.longValue() < expectedSize) {
counter.wait(unit.toMillis(timeout));
}
}
}

@Override
public String toString() {
return "AwaitCountConsumer{" + counter.get() + "/" + expectedSize + "}";
}
}