Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[CYB-189] Update integration test for Enrichment Jobs. #63

Closed
wants to merge 4 commits into from
Closed
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 @@ -17,26 +17,41 @@
import com.cloudera.cyber.commands.EnrichmentCommand;
import com.cloudera.cyber.commands.EnrichmentCommandResponse;
import com.cloudera.cyber.enrichment.hbase.config.EnrichmentsConfig;
import com.cloudera.cyber.hbase.LookupKey;
import com.google.common.collect.ImmutableMap;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.Utils;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.ProcessFunction;
import org.apache.flink.test.util.CollectingSink;
import org.apache.flink.test.util.JobTester;
import org.apache.flink.test.util.ManualSource;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.platform.commons.support.ReflectionSupport;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;

import java.util.Collections;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;

import static com.cloudera.cyber.enrichment.ConfigUtils.PARAMS_CONFIG_FILE;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.entry;

@Ignore
public class HbaseJobTest extends HbaseJob {
public class HbaseJobTest extends HbaseJob implements Serializable {
private transient ManualSource<Message> source;
private final CollectingSink<EnrichmentCommandResponse> enrichmentResponseSink = new CollectingSink<>();
private final CollectingSink<Message> sink = new CollectingSink<>();
private transient DataStream<Message> dataStreamSource;
private transient HbaseEnrichmentMapFunction hbaseMapSpy;
private final transient CollectingSink<EnrichmentCommandResponse> enrichmentResponseSink = new CollectingSink<>();
private final transient CollectingSink<Message> sink = new CollectingSink<>();

@Override
protected DataStream<EnrichmentCommand> createEnrichmentSource(StreamExecutionEnvironment env, ParameterTool params) {
Expand All @@ -48,7 +63,7 @@ public DataStream<EnrichmentCommandResponse> writeEnrichments(StreamExecutionEnv
DataStream<EnrichmentCommand> enrichmentSource,
EnrichmentsConfig enrichmentsConfig) {
// usually this would send to hbase
return null;
return JobTester.createManualSource(env, TypeInformation.of(EnrichmentCommandResponse.class)).getDataStream();
}

@Override
Expand All @@ -64,19 +79,80 @@ protected void writeResults(StreamExecutionEnvironment env, ParameterTool params
@Override
public DataStream<Message> createSource(StreamExecutionEnvironment env, ParameterTool params) {
source = JobTester.createManualSource(env, TypeInformation.of(Message.class));
return source.getDataStream();
this.dataStreamSource = Mockito.spy(source.getDataStream());
Mockito.doAnswer((AnswerSerializable<DataStream<Message>>) invocation -> {
HbaseEnrichmentMapFunction argument = invocation.getArgument(0);
TypeInformation<Message> unaryOperatorReturnType = TypeExtractor.getUnaryOperatorReturnType(argument, ProcessFunction.class, 0, 1, TypeExtractor.NO_INDEX, dataStreamSource.getType(), Utils.getCallLocationName(), true);
hbaseMapSpy = Mockito.mock(argument.getClass(),
Mockito.withSettings().serializable()
.spiedInstance(argument)
.defaultAnswer(Mockito.CALLS_REAL_METHODS));
return dataStreamSource.process(hbaseMapSpy, unaryOperatorReturnType);
}).when(dataStreamSource).process(Mockito.any());
return this.dataStreamSource;
}

@Test
public void test() throws Exception {
public void hbaseEnrichment() throws Exception {
JobTester.startTest(createPipeline(ParameterTool.fromMap(ImmutableMap.of(
PARAMS_CONFIG_FILE, "config.json"
PARAMS_CONFIG_FILE, "src/test/resources/configs.json",
PARAMS_ENRICHMENT_CONFIG, "src/test/resources/enrichments_config.json"
))));
source.sendRecord(TestUtils.createMessage(Collections.singletonMap("hostname", "test")), 0);
List<Message> resultMessage = new ArrayList<>();


ReflectionSupport.invokeMethod(
hbaseMapSpy.getClass()
.getDeclaredMethod("connectHbase"),
Mockito.doNothing().when(hbaseMapSpy)
);
ReflectionSupport.invokeMethod(
hbaseMapSpy.getClass()
.getDeclaredMethod("fetch", LookupKey.class),
Mockito.doReturn(ImmutableMap.of("hbaseQualifier", "qualifierValue")).when(hbaseMapSpy),
Mockito.any(SimpleLookupKey.class)
);
ReflectionSupport.invokeMethod(
hbaseMapSpy.getClass()
.getDeclaredMethod("fetch", LookupKey.class),
Mockito.doReturn(ImmutableMap.of("hbaseQualifier2", "qualifierValue2")).when(hbaseMapSpy),
Mockito.any(MetronLookupKey.class)
);

sendMessage("2", ImmutableMap.of("hostname", "test", "hostname2", "test2"), 200);
source.sendWatermark(200L);

JobTester.stopTest();

while (!sink.isEmpty()) {
resultMessage.add(sink.poll());
}
assertThat(resultMessage).hasSize(1);
Message message = resultMessage.get(0);
assertThat(message.getExtensions()).contains(
entry("hostname", "test"),
entry("hostname2", "test2"),
entry("hostname.domain_rep.hbaseQualifier", "qualifierValue"),
entry("hostname2.domain_rep_2.hbaseQualifier2", "qualifierValue2")
);
}

private Message message(String message, Map<String, String> extensions, long ts) {
return Message.builder()
.originalSource(TestUtils.createOriginal())
.ts(ts)
.message(message)
.source("test")
.extensions(extensions)
.build();
}

private void sendMessage(String strMsg, Map<String, String> extensions, long ts) {
source.sendRecord(message(strMsg, extensions, ts), ts);
}

interface AnswerSerializable<T> extends Answer<T>, Serializable {
@Override
T answer(InvocationOnMock invocation) throws Throwable;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,10 @@
{
"name": "hostname",
"enrichmentType": "domain_rep"
},
{
"name": "hostname2",
"enrichmentType": "domain_rep_2"
}
]
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,9 +8,34 @@
"simple": {
"format": "HBASE_SIMPLE",
"hbaseTableName": "enrichments_metron"
},
"domain_rep_storage": {
"format": "HBASE_SIMPLE",
"hbaseTableName": "simple"
},
"domain_rep_storage_2": {
"format": "HBASE_METRON",
"hbaseTableName": "test_metron",
"columnFamily": "cf555"
}
},
"enrichmentConfigs": {
"domain_rep": {
"storage": "domain_rep_storage",
"fieldMapping": {
"keyFields": [
"k1"
]
}
},
"domain_rep_2": {
"storage": "domain_rep_storage_2",
"fieldMapping": {
"keyFields": [
"b2"
]
}
},
"metron_enrich": {
"storage": null,
"fieldMapping": {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,9 +36,9 @@

import static com.cloudera.cyber.enrichment.ConfigUtils.PARAMS_CONFIG_FILE;
import static com.cloudera.cyber.flink.FlinkUtils.PARAMS_PARALLELISM;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.*;
import static org.hamcrest.collection.IsMapWithSize.aMapWithSize;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.entry;


public class LookupJobTest extends LookupJob {

Expand All @@ -50,9 +50,10 @@ public class LookupJobTest extends LookupJob {
@Test
public void testEnrichments() throws Exception {
JobTester.startTest(createPipeline(ParameterTool.fromMap(ImmutableMap.of(
PARAMS_CONFIG_FILE, "config.json",
PARAMS_CONFIG_FILE, "src/test/resources/config.json",
PARAMS_PARALLELISM, "1"
))));
List<Message> resultMessage = new ArrayList<>();

// make up some enrichments (three types, multiple fields and multiple entries in some)
sendEnrichment("ip_whitelist", "10.0.0.1", Collections.singletonMap("whitelist", "true"), 100L);
Expand Down Expand Up @@ -81,17 +82,35 @@ public void testEnrichments() throws Exception {

JobTester.stopTest();

// assert that the sink contains fully enriched entities

List<Message> results = new ArrayList<>();
for (int i = 0; i < 3; i++) {
Message message = sink.poll();
assertThat("message is not null", message, notNullValue());
assertThat("Message had the correct field count", message.getExtensions(), aMapWithSize(Integer.valueOf(message.getMessage())));
results.add(message);
while (!sink.isEmpty()) {
resultMessage.add(sink.poll());
}
assertThat("All message received", results, hasSize(3));
assertThat("Message has been enriched", results, hasSize(3));
// assert that the sink contains fully enriched entities
assertThat(resultMessage).hasSize(3).extracting(Message::getMessage).containsExactly("2", "4", "1");
Message message1 = resultMessage.stream().filter(m -> m.getMessage().equals("1")).findFirst().get();
Message message2 = resultMessage.stream().filter(m -> m.getMessage().equals("2")).findFirst().get();
Message message4 = resultMessage.stream().filter(m -> m.getMessage().equals("4")).findFirst().get();
assertThat(message1.getExtensions()).contains(
entry("ip_src_addr", "10.0.0.2")
);
assertThat(message2.getExtensions()).contains(
entry("ip_src_addr", "10.0.0.1"),
entry("ip_src_addr.internal_ip.field1", "1"),
entry("ip_src_addr.internal_ip.field2", "2"),
entry("ip_src_addr.ip_whitelist.whitelist", "true"),
entry("ip_src_addr.asset.location", "office"),
entry("ip_src_addr.asset.owner", "mew")
);
assertThat(message4.getExtensions()).contains(
entry("ip_src_addr", "10.0.0.1"),
entry("ip_src_addr.internal_ip.field1", "1"),
entry("ip_src_addr.internal_ip.field2", "2"),
entry("ip_src_addr.ip_whitelist.whitelist", "true"),
entry("ip_src_addr.asset.location", "office"),
entry("ip_src_addr.asset.owner", "mew"),
entry("ip_dst_addr", "192.168.0.1"),
entry("ip_dst_addr.ip_whitelist.whitelist", "false")
);
}

@Override
Expand All @@ -107,7 +126,7 @@ protected void writeResults(StreamExecutionEnvironment env, ParameterTool params
@Override
public SingleOutputStreamOperator<Message> createSource(StreamExecutionEnvironment env, ParameterTool params) {
source = JobTester.createManualSource(env, TypeInformation.of(Message.class));
return source.getDataStream().map(s->s);
return source.getDataStream().map(s -> s);
}

@Override
Expand All @@ -126,20 +145,20 @@ public long extractTimestamp(EnrichmentCommand scoringRuleCommand) {

private EnrichmentCommand enrichment(String type, String key, Map<String, String> entries) {
return EnrichmentCommand.builder()
.type(CommandType.ADD)
.type(CommandType.ADD)
.headers(Collections.emptyMap())
.payload(EnrichmentEntry.builder()
.type(type)
.key(key)
.entries(entries)
.ts(0)
.build()).build();
.payload(EnrichmentEntry.builder()
.type(type)
.key(key)
.entries(entries)
.ts(0)
.build()).build();
}

private void sendEnrichment(String type, String key, Map<String, String> entries, long ts) throws TimeoutException {
enrichmentSource.sendRecord(enrichment(type, key, entries), ts);
// EnrichmentCommandResponse response = queryResults.poll();
// assertThat("Command succeed", response.isSuccess());
// EnrichmentCommandResponse response = queryResults.poll();
// assertThat("Command succeed", response.isSuccess());
}

private Message message(String message, Map<String, String> extensions, long ts) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,14 @@
{
"name": "ip_dst_addr",
"enrichmentType": "ip_whitelist"
},
{
"name": "ip_src_addr",
"enrichmentType": "asset"
},
{
"name": "ip_src_addr",
"enrichmentType": "internal_ip"
}
]
}, {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,14 @@
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Result;

import java.io.Serializable;
import java.util.Map;

@Getter
@SuperBuilder
@EqualsAndHashCode
@ToString
public abstract class LookupKey {
public abstract class LookupKey implements Serializable {
private String cf;
private String tableName;
private String key;
Expand Down
Loading