forked from apache/storm
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
STORM-2258: Implementation of CoGroupByKey
- Loading branch information
1 parent
12e3f91
commit 58baf80
Showing
4 changed files
with
239 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
78 changes: 78 additions & 0 deletions
78
storm-client/src/jvm/org/apache/storm/streams/processors/CoGroupByKeyProcessor.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,78 @@ | ||
/** | ||
* 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 | ||
* | ||
* http://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.storm.streams.processors; | ||
|
||
import com.google.common.collect.ArrayListMultimap; | ||
import com.google.common.collect.ImmutableMultimap; | ||
import com.google.common.collect.Multimap; | ||
import org.apache.storm.streams.Pair; | ||
|
||
import java.util.ArrayList; | ||
|
||
/** | ||
* co-group by key implementation | ||
*/ | ||
public class CoGroupByKeyProcessor<K,V1, V2> extends BaseProcessor<Pair<K, ?>> implements BatchProcessor { | ||
private final String firstStream; | ||
private final String secondStream; | ||
private final Multimap<K, V1> firstMap = ArrayListMultimap.create(); | ||
private final Multimap<K, V2> secondMap = ArrayListMultimap.create(); | ||
|
||
|
||
public CoGroupByKeyProcessor(String firstStream, String secondStream) { | ||
this.firstStream = firstStream; | ||
this.secondStream = secondStream; | ||
} | ||
|
||
@Override | ||
public void execute(Pair<K, ?> input, String sourceStream) { | ||
K key = input.getFirst(); | ||
if (sourceStream.equals(firstStream)) { | ||
V1 val = (V1) input.getSecond(); | ||
firstMap.put(key, val); | ||
} else if (sourceStream.equals(secondStream)) { | ||
V2 val = (V2) input.getSecond(); | ||
secondMap.put(key, val); | ||
} | ||
if (!context.isWindowed()) { | ||
forwardValues(); | ||
} | ||
|
||
} | ||
|
||
@Override | ||
public void finish() { | ||
forwardValues(); | ||
firstMap.clear(); | ||
secondMap.clear(); | ||
} | ||
|
||
private void forwardValues() { | ||
firstMap.asMap().forEach((key, values) -> { | ||
context.forward(Pair.of(key, Pair.of(new ArrayList<>(values), secondMap.removeAll(key)))); | ||
}); | ||
|
||
secondMap.asMap().forEach((key, values) -> { | ||
context.forward(Pair.of(key, Pair.of(firstMap.removeAll(key), new ArrayList<>(values)))); | ||
}); | ||
|
||
} | ||
|
||
|
||
} |
112 changes: 112 additions & 0 deletions
112
storm-client/test/jvm/org/apache/storm/streams/processors/CoGroupByKeyProcessorTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,112 @@ | ||
/** | ||
* 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 | ||
* | ||
* http://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.storm.streams.processors; | ||
|
||
import org.apache.curator.shaded.com.google.common.collect.ImmutableBiMap; | ||
import org.apache.curator.shaded.com.google.common.collect.ImmutableMultimap; | ||
import org.apache.storm.streams.Pair; | ||
import org.apache.storm.streams.operations.PairValueJoiner; | ||
import org.junit.Test; | ||
|
||
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.Set; | ||
import java.util.Collection; | ||
import java.util.Arrays; | ||
|
||
|
||
import static org.junit.Assert.assertEquals; | ||
|
||
public class CoGroupByKeyProcessorTest { | ||
private CoGroupByKeyProcessor<Integer, Integer, Integer> coGroupByKeyProcessor; | ||
private String firstStream = "first"; | ||
private String secondStream = "second"; | ||
private List<Pair<Integer, Pair<List<Integer>, List<Integer>>>> res = new ArrayList<>(); | ||
|
||
private ProcessorContext context = new ProcessorContext() { | ||
@Override | ||
public <T> void forward(T input) { | ||
res.add((Pair<Integer, Pair<List<Integer>, List<Integer>>>)input); | ||
} | ||
|
||
@Override | ||
public <T> void forward(T input, String stream) { | ||
} | ||
|
||
@Override | ||
public boolean isWindowed() { | ||
return true; | ||
} | ||
|
||
@Override | ||
public Set<String> getWindowedParentStreams() { | ||
return null; | ||
} | ||
}; | ||
|
||
private List<Pair<Integer, Integer>> firstKeyValues = Arrays.asList( | ||
Pair.of(2, 4), | ||
Pair.of(5, 25), | ||
Pair.of(7, 49), | ||
Pair.of(7, 87) | ||
); | ||
|
||
private List<Pair<Integer, Integer>> secondKeyValues = Arrays.asList( | ||
Pair.of(1, 1), | ||
Pair.of(2, 8), | ||
Pair.of(5, 125), | ||
Pair.of(5,50), | ||
Pair.of(6, 216) | ||
|
||
); | ||
|
||
@Test | ||
public void testCoGroupByKey() throws Exception { | ||
coGroupByKeyProcessor = new CoGroupByKeyProcessor<>(firstStream, secondStream); | ||
processValues(); | ||
List<Pair<Integer, Pair<Collection<Integer>, Collection<Integer>>>> expected = new ArrayList<>(); | ||
Collection<Integer> list1 = new ArrayList<>(); | ||
list1.add(25); | ||
Collection<Integer> list2 = new ArrayList<>(); | ||
list2.add(125); | ||
list2.add(50); | ||
expected.add(Pair.of(5, Pair.of(list1, list2))); | ||
assertEquals(expected.get(0), res.get(1)); | ||
list1.clear(); | ||
list2.clear(); | ||
list1.add(49); | ||
list1.add(87); | ||
expected.clear(); | ||
expected.add(Pair.of(7, Pair.of(list1, list2))); | ||
assertEquals(expected.get(0), res.get(2)); | ||
} | ||
|
||
|
||
private void processValues() { | ||
res.clear(); | ||
coGroupByKeyProcessor.init(context); | ||
for (Pair<Integer, Integer> kv : firstKeyValues) { | ||
coGroupByKeyProcessor.execute(kv, firstStream); | ||
} | ||
for (Pair<Integer, Integer> kv : secondKeyValues) { | ||
coGroupByKeyProcessor.execute(kv, secondStream); | ||
} | ||
coGroupByKeyProcessor.finish(); | ||
} | ||
|
||
} |