Skip to content

Commit

Permalink
[streaming] Added CoBatchedDataStream and CoWindowDataStream with red…
Browse files Browse the repository at this point in the history
…uce functionality + CoBatchReduce bugfix

[streaming] RAT & Checkstyle fix
  • Loading branch information
szape authored and mbalassi committed Oct 17, 2014
1 parent a813785 commit b8e8bd7
Show file tree
Hide file tree
Showing 4 changed files with 415 additions and 38 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.streaming.api.datastream;

import org.apache.flink.streaming.api.function.co.CoReduceFunction;
import org.apache.flink.streaming.api.invokable.operator.co.CoBatchReduceInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoGroupedBatchReduceInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;

/**
* A {@link CoBatchedDataStream} represents a two data stream whose elements are
* batched together in sliding batches. Operation
* {@link #reduce(ReduceFunction)} can be applied for each batch and the batch
* is slid afterwards.
*
* @param <IN1>
* The type of the first input data stream
* @param <IN2>
* The type of the second input data stream
*/
public class CoBatchedDataStream<IN1, IN2> extends ConnectedDataStream<IN1, IN2> {

protected long batchSize1;
protected long batchSize2;
protected long slideSize1;
protected long slideSize2;

protected CoBatchedDataStream(DataStream<IN1> dataStream1, DataStream<IN2> dataStream2,
long batchSize1, long batchSize2, long slideSize1, long slideSize2) {
super(dataStream1, dataStream2);
this.batchSize1 = batchSize1;
this.batchSize2 = batchSize2;
this.slideSize1 = slideSize1;
this.slideSize2 = slideSize2;
}

protected CoBatchedDataStream(ConnectedDataStream<IN1, IN2> coDataStream, long batchSize1,
long batchSize2, long slideSize1, long slideSize2) {
super(coDataStream);
this.batchSize1 = batchSize1;
this.batchSize2 = batchSize2;
this.slideSize1 = slideSize1;
this.slideSize2 = slideSize2;
}

protected CoBatchedDataStream(CoBatchedDataStream<IN1, IN2> coBatchedDataStream) {
super(coBatchedDataStream);
this.batchSize1 = coBatchedDataStream.batchSize1;
this.batchSize2 = coBatchedDataStream.batchSize2;
this.slideSize1 = coBatchedDataStream.slideSize1;
this.slideSize2 = coBatchedDataStream.slideSize2;
}

/**
* Groups the elements of the {@link CoBatchedDataStream} by the given key
* positions to be used with grouped operators.
*
* @param keyPosition1
* The position of the field on which the first input data stream
* will be grouped.
* @param keyPosition2
* The position of the field on which the second input data
* stream will be grouped.
* @return The transformed {@link CoBatchedDataStream}
*/
public CoBatchedDataStream<IN1, IN2> groupBy(int keyPosition1, int keyPosition2) {
return new CoBatchedDataStream<IN1, IN2>(dataStream1.groupBy(keyPosition1),
dataStream2.groupBy(keyPosition2), batchSize1, batchSize2, slideSize1, slideSize2);
}

@Override
protected <OUT> CoInvokable<IN1, IN2, OUT> getReduceInvokable(
CoReduceFunction<IN1, IN2, OUT> coReducer) {
CoBatchReduceInvokable<IN1, IN2, OUT> invokable;
if (isGrouped) {
invokable = new CoGroupedBatchReduceInvokable<IN1, IN2, OUT>(coReducer, batchSize1,
batchSize2, slideSize1, slideSize2, keyPosition1, keyPosition2);
} else {
invokable = new CoBatchReduceInvokable<IN1, IN2, OUT>(coReducer, batchSize1,
batchSize2, slideSize1, slideSize2);
}
return invokable;
}

protected CoBatchedDataStream<IN1, IN2> copy() {
return new CoBatchedDataStream<IN1, IN2>(this);
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.streaming.api.datastream;

import org.apache.flink.streaming.api.function.co.CoReduceFunction;
import org.apache.flink.streaming.api.invokable.operator.co.CoGroupedWindowReduceInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoWindowReduceInvokable;
import org.apache.flink.streaming.api.invokable.util.TimeStamp;

/**
* A {@link CoWindowDataStream} represents two data streams whose elements are
* batched together into sliding windows. Operation
* {@link #reduce(CoReduceFunction)} can be applied for each window.
*
* @param <IN1>
* The type of the first input data stream
* @param <IN2>
* The type of the second input data stream
*/
public class CoWindowDataStream<IN1, IN2> extends CoBatchedDataStream<IN1, IN2> {
TimeStamp<IN1> timeStamp1;
TimeStamp<IN2> timeStamp2;

protected CoWindowDataStream(DataStream<IN1> dataStream1, DataStream<IN2> dataStream2,
long windowSize1, long windowSize2, long slideInterval1, long slideInterval2,
TimeStamp<IN1> timeStamp1, TimeStamp<IN2> timeStamp2) {
super(dataStream1, dataStream2, windowSize1, windowSize2, slideInterval1, slideInterval2);
this.timeStamp1 = timeStamp1;
this.timeStamp2 = timeStamp2;
}

protected CoWindowDataStream(ConnectedDataStream<IN1, IN2> coDataStream, long windowSize1,
long windowSize2, long slideInterval1, long slideInterval2, TimeStamp<IN1> timeStamp1,
TimeStamp<IN2> timeStamp2) {
super(coDataStream, windowSize1, windowSize2, slideInterval1, slideInterval2);
this.timeStamp1 = timeStamp1;
this.timeStamp2 = timeStamp2;
}

protected CoWindowDataStream(CoWindowDataStream<IN1, IN2> coWindowDataStream) {
super(coWindowDataStream);
this.timeStamp1 = coWindowDataStream.timeStamp1;
this.timeStamp2 = coWindowDataStream.timeStamp2;
}

public CoWindowDataStream<IN1, IN2> groupBy(int keyPosition1, int keyPosition2) {
return new CoWindowDataStream<IN1, IN2>(dataStream1.groupBy(keyPosition1),
dataStream2.groupBy(keyPosition2), batchSize1, batchSize2, slideSize1, slideSize2,
timeStamp1, timeStamp2);
}

@Override
protected <OUT> CoInvokable<IN1, IN2, OUT> getReduceInvokable(
CoReduceFunction<IN1, IN2, OUT> coReducer) {
CoWindowReduceInvokable<IN1, IN2, OUT> invokable;
if (isGrouped) {
invokable = new CoGroupedWindowReduceInvokable<IN1, IN2, OUT>(coReducer, batchSize1,
batchSize2, slideSize1, slideSize2, keyPosition1, keyPosition2, timeStamp1,
timeStamp2);
} else {
invokable = new CoWindowReduceInvokable<IN1, IN2, OUT>(coReducer, batchSize1,
batchSize2, slideSize1, slideSize2, timeStamp1, timeStamp2);
}
return invokable;
}

protected CoWindowDataStream<IN1, IN2> copy() {
return new CoWindowDataStream<IN1, IN2>(this);
}
}
Loading

0 comments on commit b8e8bd7

Please sign in to comment.