From 41ecf36cd7c2da0399d03a37fee5916a9bfa87e7 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Tue, 12 Jun 2018 19:48:41 -0600 Subject: [PATCH 01/36] Add metron-job project. Update pcap to be Statusable. --- metron-platform/metron-job/pom.xml | 39 ++++ .../java/org/apache/metron/job/JobStatus.java | 67 +++++++ .../org/apache/metron/job/Statusable.java | 37 ++++ .../org/apache/metron/pcap/PcapJobTest.java | 180 ++++++++++++++++++ metron-platform/metron-pcap/pom.xml | 7 +- .../org/apache/metron/pcap/mr/PcapJob.java | 164 ++++++++++++---- metron-platform/pom.xml | 1 + 7 files changed, 454 insertions(+), 41 deletions(-) create mode 100644 metron-platform/metron-job/pom.xml create mode 100644 metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java create mode 100644 metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java diff --git a/metron-platform/metron-job/pom.xml b/metron-platform/metron-job/pom.xml new file mode 100644 index 0000000000..fa36ff7125 --- /dev/null +++ b/metron-platform/metron-job/pom.xml @@ -0,0 +1,39 @@ + + + + + 4.0.0 + + + org.apache.metron + metron-platform + 0.5.0 + + + metron-job + metron-job + Metron Job + https://metron.apache.org/ + + + + org.apache.metron + metron-common + ${project.parent.version} + + + + diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java new file mode 100644 index 0000000000..656c43ba51 --- /dev/null +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java @@ -0,0 +1,67 @@ +/** + * 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.metron.job; + +import org.apache.hadoop.fs.Path; + +/** + * Capture metadata about a batch job + */ +public class JobStatus { + + public enum STATE { + NOT_RUNNING, + RUNNING, + SUCCEEDED, + FAILED, + KILLED + } + + private STATE state = STATE.NOT_RUNNING; + private double percentComplete = 0.0; + private Path resultPath; + + public JobStatus withState(STATE state) { + this.state = state; + return this; + } + + public JobStatus withPercentComplete(double percentComplete) { + this.percentComplete = percentComplete; + return this; + } + + public JobStatus withResultPath(Path resultPath) { + this.resultPath = resultPath; + return this; + } + + public STATE getState() { + return state; + } + + public double getPercentComplete() { + return percentComplete; + } + + public Path getResultPath() { + return resultPath; + } + +} diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java new file mode 100644 index 0000000000..dbe52be63c --- /dev/null +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java @@ -0,0 +1,37 @@ +/** + * 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.metron.job; + +public interface Statusable { + + /** + * Current job status. + * + * @return status + */ + JobStatus getStatus(); + + /** + * Completion flag. + * + * @return true if job is completed, whether KILLED, FAILED, SUCCEEDED. False otherwise. + */ + boolean isDone(); + +} diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java index 3536a7e113..9bb3d6758c 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java @@ -19,18 +19,64 @@ package org.apache.metron.pcap; import static java.lang.Long.toUnsignedString; +import static java.lang.String.format; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.startsWith; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobStatus.State; import org.apache.metron.common.utils.timestamp.TimestampConverters; +import org.apache.metron.job.JobStatus; +import org.apache.metron.job.JobStatus.STATE; +import org.apache.metron.job.Statusable; +import org.apache.metron.pcap.filter.PcapFilterConfigurator; +import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.mr.PcapJob; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; public class PcapJobTest { + @Mock + private Job job; + @Mock + private org.apache.hadoop.mapreduce.JobStatus mrStatus; + private Path basePath; + private Path baseOutPath; + private long startTime; + private long endTime; + private int numReducers; + private Map fixedFields; + private Configuration hadoopConfig; + + @Before + public void setup() { + MockitoAnnotations.initMocks(this); + basePath = new Path("basepath"); + baseOutPath = new Path("outpath"); + startTime = 100; + endTime = 200; + numReducers = 5; + fixedFields = new HashMap<>(); + fixedFields.put("ip_src_addr", "192.168.1.1"); + hadoopConfig = new Configuration(); + } + @Test public void partition_gives_value_in_range() throws Exception { long start = 1473897600000000000L; @@ -46,4 +92,138 @@ public void partition_gives_value_in_range() throws Exception { equalTo(8)); } + private class TestJob extends PcapJob { + + @Override + public Job createJob(Path basePath, Path outputPath, long beginNS, long endNS, + int numReducers, T fields, Configuration conf, FileSystem fs, + PcapFilterConfigurator filterImpl) throws IOException { + return job; + } + } + + @Test + public void job_succeeds_synchronously() throws Exception { + when(job.isComplete()).thenReturn(true); + when(mrStatus.getState()).thenReturn(State.SUCCEEDED); + when(job.getStatus()).thenReturn(mrStatus); + TestJob testJob = new TestJob(); + Statusable statusable = testJob.query(basePath, + baseOutPath, + startTime, + endTime, + numReducers, + fixedFields, + hadoopConfig, + FileSystem.get(hadoopConfig), + new FixedPcapFilter.Configurator(), + true); + verify(job, times(1)).waitForCompletion(true); + JobStatus status = statusable.getStatus(); + Assert.assertThat(status.getState(), equalTo(STATE.SUCCEEDED)); + Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); + String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); + Assert.assertThat(status.getResultPath(), notNullValue()); + Assert.assertThat(status.getResultPath().toString(), startsWith(expectedOutPath)); + } + + @Test + public void job_fails_synchronously() throws Exception { + when(job.isComplete()).thenReturn(true); + when(mrStatus.getState()).thenReturn(State.FAILED); + when(job.getStatus()).thenReturn(mrStatus); + TestJob testJob = new TestJob(); + Statusable statusable = testJob.query(basePath, + baseOutPath, + startTime, + endTime, + numReducers, + fixedFields, + hadoopConfig, + FileSystem.get(hadoopConfig), + new FixedPcapFilter.Configurator(), + true); + JobStatus status = statusable.getStatus(); + Assert.assertThat(status.getState(), equalTo(STATE.FAILED)); + Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); + String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); + Assert.assertThat(status.getResultPath(), notNullValue()); + Assert.assertThat(status.getResultPath().toString(), startsWith(expectedOutPath)); + } + + @Test + public void job_fails_with_killed_status_synchronously() throws Exception { + when(job.isComplete()).thenReturn(true); + when(mrStatus.getState()).thenReturn(State.KILLED); + when(job.getStatus()).thenReturn(mrStatus); + TestJob testJob = new TestJob(); + Statusable statusable = testJob.query(basePath, + baseOutPath, + startTime, + endTime, + numReducers, + fixedFields, + hadoopConfig, + FileSystem.get(hadoopConfig), + new FixedPcapFilter.Configurator(), + true); + JobStatus status = statusable.getStatus(); + Assert.assertThat(status.getState(), equalTo(STATE.KILLED)); + Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); + String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); + Assert.assertThat(status.getResultPath(), notNullValue()); + Assert.assertThat(status.getResultPath().toString(), startsWith(expectedOutPath)); + } + + @Test + public void job_succeeds_asynchronously() throws Exception { + when(job.isComplete()).thenReturn(true); + when(mrStatus.getState()).thenReturn(State.SUCCEEDED); + when(job.getStatus()).thenReturn(mrStatus); + TestJob testJob = new TestJob(); + Statusable statusable = testJob.query(basePath, + baseOutPath, + startTime, + endTime, + numReducers, + fixedFields, + hadoopConfig, + FileSystem.get(hadoopConfig), + new FixedPcapFilter.Configurator(), + false); + JobStatus status = statusable.getStatus(); + Assert.assertThat(status.getState(), equalTo(STATE.SUCCEEDED)); + Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); + String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); + Assert.assertThat(status.getResultPath(), notNullValue()); + Assert.assertThat(status.getResultPath().toString(), startsWith(expectedOutPath)); + } + + @Test + public void job_reports_percent_complete() throws Exception { + when(job.isComplete()).thenReturn(false); + when(mrStatus.getState()).thenReturn(State.RUNNING); + when(job.getStatus()).thenReturn(mrStatus); + TestJob testJob = new TestJob(); + Statusable statusable = testJob.query(basePath, + baseOutPath, + startTime, + endTime, + numReducers, + fixedFields, + hadoopConfig, + FileSystem.get(hadoopConfig), + new FixedPcapFilter.Configurator(), + false); + when(job.mapProgress()).thenReturn(0.5f); + when(job.reduceProgress()).thenReturn(0f); + JobStatus status = statusable.getStatus(); + Assert.assertThat(status.getState(), equalTo(STATE.RUNNING)); + Assert.assertThat(status.getPercentComplete(), equalTo(25.0)); + when(job.mapProgress()).thenReturn(1.0f); + when(job.reduceProgress()).thenReturn(0.5f); + status = statusable.getStatus(); + Assert.assertThat(status.getPercentComplete(), equalTo(75.0)); + } + } diff --git a/metron-platform/metron-pcap/pom.xml b/metron-platform/metron-pcap/pom.xml index c9f873ea20..640cdc6f9f 100644 --- a/metron-platform/metron-pcap/pom.xml +++ b/metron-platform/metron-pcap/pom.xml @@ -39,9 +39,14 @@ + + org.apache.metron + metron-hbase + ${project.parent.version} + org.apache.metron - metron-hbase + metron-job ${project.parent.version} diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index 14ea3cb753..99bcb9a303 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -48,6 +48,9 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; import org.apache.metron.common.hadoop.SequenceFileIterable; +import org.apache.metron.job.JobStatus; +import org.apache.metron.job.JobStatus.STATE; +import org.apache.metron.job.Statusable; import org.apache.metron.pcap.PacketInfo; import org.apache.metron.pcap.PcapHelper; import org.apache.metron.pcap.filter.PcapFilter; @@ -57,12 +60,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class PcapJob { +public class PcapJob implements Statusable { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); public static final String START_TS_CONF = "start_ts"; public static final String END_TS_CONF = "end_ts"; public static final String WIDTH_CONF = "width"; + private Job job; // store a running MR job reference for async status check + private Path outputPath; public static enum PCAP_COUNTER { MALFORMED_PACKET_COUNT @@ -156,23 +161,8 @@ protected void reduce(LongWritable key, Iterable values, Context } /** - * Returns a lazily-read Iterable over a set of sequence files + * Run query synchronously. */ - private SequenceFileIterable readResults(Path outputPath, Configuration config, FileSystem fs) throws IOException { - List files = new ArrayList<>(); - for (RemoteIterator it = fs.listFiles(outputPath, false); it.hasNext(); ) { - Path p = it.next().getPath(); - if (p.getName().equals("_SUCCESS")) { - fs.delete(p, false); - continue; - } - files.add(p); - } - LOG.debug("Output path={}", outputPath); - Collections.sort(files, (o1,o2) -> o1.getName().compareTo(o2.getName())); - return new SequenceFileIterable(files, config); - } - public SequenceFileIterable query(Path basePath , Path baseOutputPath , long beginNS @@ -183,42 +173,87 @@ public SequenceFileIterable query(Path basePath , FileSystem fs , PcapFilterConfigurator filterImpl ) throws IOException, ClassNotFoundException, InterruptedException { + Statusable statusable = query(basePath, baseOutputPath, beginNS, endNS, numReducers, fields, + conf, + fs, filterImpl, true); + JobStatus jobStatus = statusable.getStatus(); + if (jobStatus.getState() == STATE.SUCCEEDED) { + Path resultPath = jobStatus.getResultPath(); + return readResults(resultPath, conf, fs); + } else { + throw new RuntimeException( + "Unable to complete query due to errors. Please check logs for full errors."); + } + } + + /** + * Run query sync OR async based on flag. Async mode allows the client to check the returned + * statusable object for status details. + */ + public Statusable query(Path basePath, + Path baseOutputPath, + long beginNS, + long endNS, + int numReducers, + T fields, + Configuration conf, + FileSystem fs, + PcapFilterConfigurator filterImpl, + boolean sync) + throws IOException, ClassNotFoundException, InterruptedException { String fileName = Joiner.on("_").join(beginNS, endNS, filterImpl.queryToString(fields), UUID.randomUUID().toString()); if(LOG.isDebugEnabled()) { DateFormat format = SimpleDateFormat.getDateTimeInstance( SimpleDateFormat.LONG - , SimpleDateFormat.LONG - ); + , SimpleDateFormat.LONG + ); String from = format.format(new Date(Long.divideUnsigned(beginNS, 1000000))); String to = format.format(new Date(Long.divideUnsigned(endNS, 1000000))); LOG.debug("Executing query {} on timerange from {} to {}", filterImpl.queryToString(fields), from, to); } - Path outputPath = new Path(baseOutputPath, fileName); - Job job = createJob( basePath - , outputPath - , beginNS - , endNS - , numReducers - , fields - , conf - , fs - , filterImpl - ); - if (job == null) { - LOG.info("No files to process with specified date range."); - return new SequenceFileIterable(new ArrayList<>(), conf); - } - boolean completed = job.waitForCompletion(true); - if(completed) { - return readResults(outputPath, conf, fs); + outputPath = new Path(baseOutputPath, fileName); + job = createJob( basePath + , outputPath + , beginNS + , endNS + , numReducers + , fields + , conf + , fs + , filterImpl + ); + if (sync) { + job.waitForCompletion(true); } else { - throw new RuntimeException("Unable to complete query due to errors. Please check logs for full errors."); + job.submit(); } + return this; } - public static long findWidth(long start, long end, int numReducers) { - return Long.divideUnsigned(end - start, numReducers) + 1; + /** + * Returns a lazily-read Iterable over a set of sequence files + */ + private SequenceFileIterable readResults(Path outputPath, Configuration config, FileSystem fs) throws IOException { + List files = new ArrayList<>(); + for (RemoteIterator it = fs.listFiles(outputPath, false); it.hasNext(); ) { + Path p = it.next().getPath(); + if (p.getName().equals("_SUCCESS")) { + fs.delete(p, false); + continue; + } + files.add(p); + } + if (files.size() == 0) { + LOG.info("No files to process with specified date range."); + } else { + LOG.debug("Output path={}", outputPath); + Collections.sort(files, (o1, o2) -> o1.getName().compareTo(o2.getName())); + } + return new SequenceFileIterable(files, config); } + /** + * Creates, but does not submit the job. + */ public Job createJob( Path basePath , Path outputPath , long beginNS @@ -256,6 +291,10 @@ public Job createJob( Path basePath return job; } + public static long findWidth(long start, long end, int numReducers) { + return Long.divideUnsigned(end - start, numReducers) + 1; + } + protected Iterable listFiles(FileSystem fs, Path basePath) throws IOException { List ret = new ArrayList<>(); RemoteIterator filesIt = fs.listFiles(basePath, true); @@ -265,4 +304,49 @@ protected Iterable listFiles(FileSystem fs, Path basePath) throws IOExcept return ret; } + @Override + public JobStatus getStatus() { + // Note: this method is only reading state from the underlying job, so locking not needed + JobStatus status = new JobStatus().withResultPath(outputPath); + if (job == null) { + status.withPercentComplete(100).withState(STATE.SUCCEEDED); + } else { + try { + if (job.isComplete()) { + status.withPercentComplete(100); + switch (job.getStatus().getState()) { + case SUCCEEDED: + status.withState(STATE.SUCCEEDED); + break; + case FAILED: + status.withState(STATE.FAILED); + break; + case KILLED: + status.withState(STATE.KILLED); + break; + } + } else { + float mapProg = job.mapProgress(); + float reduceProg = job.reduceProgress(); + float totalProgress = ((mapProg / 2) + (reduceProg / 2)) * 100; + status.withPercentComplete(totalProgress).withState(STATE.RUNNING); + } + } catch (Exception e) { + throw new RuntimeException("Error occurred while attempting to retrieve job status.", e); + } + } + return status; + } + + @Override + public boolean isDone() { + // Note: this method is only reading state from the underlying job, so locking not needed + try { + return job.isComplete(); + } catch (Exception e) { + throw new RuntimeException("Error occurred while attempting to retrieve job status.", e); + } + } + + } diff --git a/metron-platform/pom.xml b/metron-platform/pom.xml index a99dbc7cc7..6866386330 100644 --- a/metron-platform/pom.xml +++ b/metron-platform/pom.xml @@ -46,6 +46,7 @@ metron-enrichment metron-solr metron-parsers + metron-job metron-pcap-backend metron-data-management metron-pcap From a83b472778e8e40cd81132f86402873ab4b8b2b1 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Sat, 16 Jun 2018 17:10:49 -0600 Subject: [PATCH 02/36] Save progress on Pageable implementation --- .../apache/metron/common/utils/HDFSUtils.java | 18 +++++++ .../metron/common/utils/HDFSUtilsTest.java | 48 +++++++++++++++++++ .../java/org/apache/metron/job/JobStatus.java | 12 ++++- .../java/org/apache/metron/job/Pageable.java | 38 +++++++++++++++ .../org/apache/metron/job/Statusable.java | 19 ++++++++ .../org/apache/metron/pcap/query/PcapCli.java | 27 +++-------- .../org/apache/metron/pcap/PcapJobTest.java | 3 ++ .../apache/metron/pcap/query/PcapCliTest.java | 1 + .../org/apache/metron/pcap/mr/PcapJob.java | 43 ++++++++++++++++- .../metron/pcap/writer}/ResultsWriter.java | 31 ++++++++---- 10 files changed, 207 insertions(+), 33 deletions(-) create mode 100644 metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/HDFSUtilsTest.java create mode 100644 metron-platform/metron-job/src/main/java/org/apache/metron/job/Pageable.java rename metron-platform/{metron-pcap-backend/src/main/java/org/apache/metron/pcap/query => metron-pcap/src/main/java/org/apache/metron/pcap/writer}/ResultsWriter.java (59%) diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java index ee00b7e6f8..1d03a6d5a0 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java @@ -23,6 +23,7 @@ import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -56,4 +57,21 @@ public static List readFile(Configuration config, String path) throws IO return IOUtils.readLines(inputStream, "UTF-8"); } + /** + * Write file to HDFS. Writes to local FS if file:// used as protocol. + * + * @param config filesystem configuration + * @param bytes bytes to write + * @param path output path + * @throws IOException This is the generic Hadoop "everything is an IOException." + */ + public static void write(Configuration config, byte[] bytes, String path) throws IOException { + FileSystem fs = FileSystem.newInstance(config); + Path outPath = new Path(path); + fs.mkdirs(outPath.getParent()); + try (FSDataOutputStream outputStream = fs.create(outPath)) { + outputStream.write(bytes); + outputStream.flush(); + } + } } diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/HDFSUtilsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/HDFSUtilsTest.java new file mode 100644 index 0000000000..53b6f299c2 --- /dev/null +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/HDFSUtilsTest.java @@ -0,0 +1,48 @@ +/** + * 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.metron.common.utils; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; + +import java.io.File; +import java.nio.charset.StandardCharsets; +import org.apache.hadoop.conf.Configuration; +import org.apache.metron.integration.utils.TestUtils; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class HDFSUtilsTest { + + @Rule + public TemporaryFolder tempDir = new TemporaryFolder(); + + @Test + public void writes_file_to_local_fs() throws Exception { + String outText = "small brown bike and casket lottery"; + String outFile = tempDir.getRoot().getAbsolutePath() + "outfile.txt"; + Configuration config = new Configuration(); + config.set("fs.default.name", "file:///"); + HDFSUtils.write(config, outText.getBytes(StandardCharsets.UTF_8), outFile); + String actual = TestUtils.read(new File(outFile)); + assertThat("Text should match", actual, equalTo(outText)); + } + +} diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java index 656c43ba51..d3cda65e54 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java @@ -23,7 +23,7 @@ /** * Capture metadata about a batch job */ -public class JobStatus { +public class JobStatus { public enum STATE { NOT_RUNNING, @@ -36,6 +36,7 @@ public enum STATE { private STATE state = STATE.NOT_RUNNING; private double percentComplete = 0.0; private Path resultPath; + private Pageable pagedResults; public JobStatus withState(STATE state) { this.state = state; @@ -52,6 +53,11 @@ public JobStatus withResultPath(Path resultPath) { return this; } + public JobStatus withPagedResults(Pageable pagedResults) { + this.pagedResults = pagedResults; + return this; + } + public STATE getState() { return state; } @@ -64,4 +70,8 @@ public Path getResultPath() { return resultPath; } + public Pageable getPagedResults() { + return pagedResults; + } + } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Pageable.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Pageable.java new file mode 100644 index 0000000000..1038ab8e3c --- /dev/null +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Pageable.java @@ -0,0 +1,38 @@ +/** + * 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.metron.job; + +public interface Pageable { + + /** + * Transform into an Iterable. + * + * @return Iterable version of this Pageable. + */ + Iterable asIterable(); + + /** + * Provides access to a specific page of results in the result set. + * + * @param num page number to access. + * @return value at the specified page. + */ + T getPage(int num); + +} diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java index dbe52be63c..7a8fc02cef 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java @@ -18,6 +18,12 @@ package org.apache.metron.job; +import java.io.IOException; +import java.util.Map; + +/** + * Abstraction for getting status on running jobs. Also provides options for killing and validating. + */ public interface Statusable { /** @@ -34,4 +40,17 @@ public interface Statusable { */ boolean isDone(); + /** + * Kill job. + */ + void kill() throws IOException; + + /** + * Validate job after submitted. + * + * @param configuration config for validating the job. + * @return true if job is valid based on passed configuration, false if invalid. + */ + boolean validate(Map configuration); + } diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java index b50d488b7c..06545b6894 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java @@ -17,11 +17,9 @@ */ package org.apache.metron.pcap.query; -import com.google.common.collect.Iterables; import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.Arrays; -import java.util.List; import java.util.UUID; import org.apache.commons.cli.ParseException; import org.apache.commons.lang3.tuple.Pair; @@ -35,6 +33,7 @@ import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.filter.query.QueryPcapFilter; import org.apache.metron.pcap.mr.PcapJob; +import org.apache.metron.pcap.writer.ResultsWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -156,30 +155,16 @@ public int run(String[] args) { printBasicHelp(); return -1; } - try { - Iterable> partitions = Iterables.partition(results, commonConfig.getNumRecordsPerFile()); - int part = 1; - if (partitions.iterator().hasNext()) { - for (List data : partitions) { - String outFileName = String.format("pcap-data-%s+%04d.pcap", commonConfig.getPrefix(), part++); - if(data.size() > 0) { - resultsWriter.write(data, outFileName); - } - } - } else { - System.out.println("No results returned."); - } + try { + jobRunner.writeResults(results, resultsWriter, new Path("file://."), + commonConfig.getNumRecordsPerFile(), + commonConfig.getPrefix()); } catch (IOException e) { LOGGER.error("Unable to write file", e); return -1; - } finally { - try { - results.cleanup(); - } catch(IOException e) { - LOGGER.warn("Unable to cleanup files in HDFS", e); - } } + return 0; } diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java index 9bb3d6758c..e24b43f316 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java @@ -20,6 +20,7 @@ import static java.lang.Long.toUnsignedString; import static java.lang.String.format; +import static org.hamcrest.CoreMatchers.endsWith; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.startsWith; @@ -125,6 +126,8 @@ public void job_succeeds_synchronously() throws Exception { String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); Assert.assertThat(status.getResultPath(), notNullValue()); Assert.assertThat(status.getResultPath().toString(), startsWith(expectedOutPath)); + Assert.assertThat(status.getPagedResults(), notNullValue()); + Assert.assertThat(((Path)status.getPagedResults().getPage(0)).getName(), endsWith("pcap-data-1234+0001.pcap")); } @Test diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java index 7202819f6c..0d16651adb 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java @@ -28,6 +28,7 @@ import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.filter.query.QueryPcapFilter; import org.apache.metron.pcap.mr.PcapJob; +import org.apache.metron.pcap.writer.ResultsWriter; import org.junit.Before; import org.junit.Test; import org.mockito.Mock; diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index 99bcb9a303..deee1834fb 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -22,6 +22,7 @@ import static org.apache.metron.pcap.PcapHelper.lessThanOrEqualTo; import com.google.common.base.Joiner; +import com.google.common.collect.Iterables; import java.io.IOException; import java.lang.invoke.MethodHandles; import java.text.DateFormat; @@ -30,6 +31,7 @@ import java.util.Collections; import java.util.Date; import java.util.List; +import java.util.Map; import java.util.UUID; import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; @@ -57,6 +59,7 @@ import org.apache.metron.pcap.filter.PcapFilterConfigurator; import org.apache.metron.pcap.filter.PcapFilters; import org.apache.metron.pcap.utils.FileFilterUtil; +import org.apache.metron.pcap.writer.ResultsWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -173,7 +176,7 @@ public SequenceFileIterable query(Path basePath , FileSystem fs , PcapFilterConfigurator filterImpl ) throws IOException, ClassNotFoundException, InterruptedException { - Statusable statusable = query(basePath, baseOutputPath, beginNS, endNS, numReducers, fields, + Statusable statusable = query(basePath, baseOutputPath, finalBaseOutputPath, beginNS, endNS, numReducers, fields, conf, fs, filterImpl, true); JobStatus jobStatus = statusable.getStatus(); @@ -192,6 +195,7 @@ public SequenceFileIterable query(Path basePath */ public Statusable query(Path basePath, Path baseOutputPath, + Path finalBaseOutputPath, long beginNS, long endNS, int numReducers, @@ -251,6 +255,33 @@ private SequenceFileIterable readResults(Path outputPath, Configuration config, return new SequenceFileIterable(files, config); } + public List writeResults(SequenceFileIterable results, ResultsWriter resultsWriter, + Path outPath, int recPerFile, String prefix) throws IOException { + List outFiles = new ArrayList<>(); + try { + Iterable> partitions = Iterables.partition(results, recPerFile); + int part = 1; + if (partitions.iterator().hasNext()) { + for (List data : partitions) { + String outFileName = String.format("%s/pcap-data-%s+%04d.pcap", outPath, prefix, part++); + if (data.size() > 0) { + resultsWriter.write(new Configuration(), data, outFileName); + outFiles.add(new Path(outFileName)); + } + } + } else { + LOG.info("No results returned."); + } + } finally { + try { + results.cleanup(); + } catch (IOException e) { + LOG.warn("Unable to cleanup files in HDFS", e); + } + } + return outFiles; + } + /** * Creates, but does not submit the job. */ @@ -348,5 +379,15 @@ public boolean isDone() { } } + @Override + public void kill() throws IOException { + job.killJob(); + } + + @Override + public boolean validate(Map configuration) { + // default implementation placeholder + return true; + } } diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/ResultsWriter.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/writer/ResultsWriter.java similarity index 59% rename from metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/ResultsWriter.java rename to metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/writer/ResultsWriter.java index ab11770b01..3934aca885 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/ResultsWriter.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/writer/ResultsWriter.java @@ -15,25 +15,36 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.metron.pcap.query; - -import org.apache.metron.pcap.PcapMerger; +package org.apache.metron.pcap.writer; import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.FileOutputStream; import java.io.IOException; import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.metron.common.utils.HDFSUtils; +import org.apache.metron.pcap.PcapMerger; public class ResultsWriter { - public void write(List pcaps, String outPath) throws IOException { - File out = new File(outPath); - try (FileOutputStream fos = new FileOutputStream(out)) { - fos.write(mergePcaps(pcaps)); - } + /** + * Write out pcaps. Configuration offers ability to configure for HDFS or local FS, if desired. + * + * @param config Standard hadoop filesystem config. + * @param pcaps pcap data to write. Pre-merged format as a list of pcaps as byte arrays. + * @param outPath where to write the pcap data to. + * @throws IOException I/O issue encountered. + */ + public void write(Configuration config, List pcaps, String outPath) throws IOException { + HDFSUtils.write(config, mergePcaps(pcaps), outPath); } + /** + * Creates a pcap file with proper global header from individual pcaps. + * + * @param pcaps pcap records to merge into a pcap file with header. + * @return merged result. + * @throws IOException I/O issue encountered. + */ public byte[] mergePcaps(List pcaps) throws IOException { if (pcaps == null) { return new byte[]{}; From 976e1bcdf9619cc0672de4bcd7dd0d865a01cc1b Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Mon, 25 Jun 2018 13:00:03 -0600 Subject: [PATCH 03/36] Rev metron-job to 0.5.1 after merge with master --- metron-platform/metron-job/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metron-platform/metron-job/pom.xml b/metron-platform/metron-job/pom.xml index fa36ff7125..49cc3cb8bc 100644 --- a/metron-platform/metron-job/pom.xml +++ b/metron-platform/metron-job/pom.xml @@ -20,7 +20,7 @@ org.apache.metron metron-platform - 0.5.0 + 0.5.1 metron-job From f9fc106f4a33f323dc06260d861fc543726e2518 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Tue, 26 Jun 2018 16:03:21 -0600 Subject: [PATCH 04/36] Move result writing code to pcapjob. Get pcapclitest working again. --- .../org/apache/metron/pcap/query/PcapCli.java | 1 + .../apache/metron/pcap/query/PcapCliTest.java | 51 +++++++++++-------- .../org/apache/metron/pcap/mr/PcapJob.java | 12 ++--- 3 files changed, 38 insertions(+), 26 deletions(-) diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java index 06545b6894..3328bb4416 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java @@ -157,6 +157,7 @@ public int run(String[] args) { } try { + // write to local FS in the executing directory jobRunner.writeResults(results, resultsWriter, new Path("file://."), commonConfig.getNumRecordsPerFile(), commonConfig.getPrefix()); diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java index 0d16651adb..fcb4d1995c 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java @@ -17,6 +17,30 @@ */ package org.apache.metron.pcap.query; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.isA; +import static org.mockito.Mockito.doCallRealMethod; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.BufferedOutputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.PrintStream; +import java.nio.charset.StandardCharsets; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Arrays; +import java.util.Date; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -35,20 +59,6 @@ import org.mockito.Mockito; import org.mockito.MockitoAnnotations; -import java.io.BufferedOutputStream; -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; -import java.nio.charset.StandardCharsets; -import java.text.ParseException; -import java.text.SimpleDateFormat; -import java.util.*; - -import static org.hamcrest.CoreMatchers.equalTo; -import static org.junit.Assert.assertThat; -import static org.mockito.Matchers.*; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - public class PcapCliTest { @Mock @@ -59,8 +69,9 @@ public class PcapCliTest { private Clock clock; @Before - public void setup() { + public void setup() throws IOException { MockitoAnnotations.initMocks(this); + doCallRealMethod().when(jobRunner).writeResults(anyObject(), anyObject(), anyObject(), anyInt(), anyObject()); } @Test @@ -96,7 +107,7 @@ public void runs_fixed_pcap_filter_job_with_default_argument_list() throws Excep PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(pcaps, "pcap-data-random_prefix+0001.pcap"); + Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file://./pcap-data-random_prefix+0001.pcap")); } @Test @@ -136,7 +147,7 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list_and_default_datef PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(pcaps, "pcap-data-random_prefix+0001.pcap"); + Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file://./pcap-data-random_prefix+0001.pcap")); } @Test @@ -179,7 +190,7 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list() throws Exceptio PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(pcaps, "pcap-data-random_prefix+0001.pcap"); + Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file://./pcap-data-random_prefix+0001.pcap")); } private long asNanos(String inDate, String format) throws ParseException { @@ -212,7 +223,7 @@ public void runs_query_pcap_filter_job_with_default_argument_list() throws Excep PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(pcaps, "pcap-data-random_prefix+0001.pcap"); + Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file://./pcap-data-random_prefix+0001.pcap")); } @Test @@ -240,7 +251,7 @@ public void runs_query_pcap_filter_job_with_full_argument_list() throws Exceptio PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(pcaps, "pcap-data-random_prefix+0001.pcap"); + Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file://./pcap-data-random_prefix+0001.pcap")); } // INVALID OPTION CHECKS diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index deee1834fb..1992b6b1b2 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -83,12 +83,12 @@ public static class PcapPartitioner extends Partitioner numPartitions) { + if (ret > numPartitions) { throw new IllegalArgumentException(String.format("Bad partition: key=%s, width=%d, partition=%d, numPartitions=%d" , Long.toUnsignedString(x), width, ret, numPartitions) ); @@ -112,6 +112,7 @@ public Configuration getConf() { return configuration; } } + public static class PcapMapper extends Mapper { PcapFilter filter; @@ -137,7 +138,7 @@ protected void map(LongWritable key, BytesWritable value, Context context) throw List packetInfos; try { packetInfos = PcapHelper.toPacketInfo(value.copyBytes()); - } catch(Exception e) { + } catch (Exception e) { // toPacketInfo is throwing RuntimeExceptions. Attempt to catch and count errors with malformed packets context.getCounter(PCAP_COUNTER.MALFORMED_PACKET_COUNT).increment(1); return; @@ -157,7 +158,7 @@ private Stream filteredPacketInfo(List packetInfos) thro public static class PcapReducer extends Reducer { @Override protected void reduce(LongWritable key, Iterable values, Context context) throws IOException, InterruptedException { - for(BytesWritable value : values) { + for (BytesWritable value : values) { context.write(key, value); } } @@ -176,7 +177,7 @@ public SequenceFileIterable query(Path basePath , FileSystem fs , PcapFilterConfigurator filterImpl ) throws IOException, ClassNotFoundException, InterruptedException { - Statusable statusable = query(basePath, baseOutputPath, finalBaseOutputPath, beginNS, endNS, numReducers, fields, + Statusable statusable = query(basePath, baseOutputPath, beginNS, endNS, numReducers, fields, conf, fs, filterImpl, true); JobStatus jobStatus = statusable.getStatus(); @@ -195,7 +196,6 @@ public SequenceFileIterable query(Path basePath */ public Statusable query(Path basePath, Path baseOutputPath, - Path finalBaseOutputPath, long beginNS, long endNS, int numReducers, From 8b4ef9cdc37235c333f3cbd294f20ae15a7438aa Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Tue, 26 Jun 2018 23:31:21 -0600 Subject: [PATCH 05/36] Add Pageable results. --- .../java/org/apache/metron/job/JobStatus.java | 22 +++------- .../org/apache/metron/pcap/query/PcapCli.java | 10 +---- .../org/apache/metron/pcap/PcapJobTest.java | 26 +++++------- .../org/apache/metron/pcap/PcapFiles.java | 42 +++++++++++++++++++ .../org/apache/metron/pcap/mr/PcapJob.java | 24 ++++++----- 5 files changed, 74 insertions(+), 50 deletions(-) create mode 100644 metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java index d3cda65e54..8b67487706 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java @@ -21,11 +21,11 @@ import org.apache.hadoop.fs.Path; /** - * Capture metadata about a batch job + * Capture metadata about a batch job. */ -public class JobStatus { +public class JobStatus { - public enum STATE { + public enum State { NOT_RUNNING, RUNNING, SUCCEEDED, @@ -33,12 +33,11 @@ public enum STATE { KILLED } - private STATE state = STATE.NOT_RUNNING; + private State state = State.NOT_RUNNING; private double percentComplete = 0.0; private Path resultPath; - private Pageable pagedResults; - public JobStatus withState(STATE state) { + public JobStatus withState(State state) { this.state = state; return this; } @@ -53,12 +52,7 @@ public JobStatus withResultPath(Path resultPath) { return this; } - public JobStatus withPagedResults(Pageable pagedResults) { - this.pagedResults = pagedResults; - return this; - } - - public STATE getState() { + public State getState() { return state; } @@ -70,8 +64,4 @@ public Path getResultPath() { return resultPath; } - public Pageable getPagedResults() { - return pagedResults; - } - } diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java index 3328bb4416..61f83c61fd 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java @@ -107,10 +107,7 @@ public int run(String[] args) { hadoopConf, FileSystem.get(hadoopConf), new FixedPcapFilter.Configurator()); - } catch (IOException | ClassNotFoundException e) { - LOGGER.error("Failed to execute fixed filter job: {}", e.getMessage(), e); - return -1; - } catch (InterruptedException e) { + } catch (IOException | ClassNotFoundException | InterruptedException e) { LOGGER.error("Failed to execute fixed filter job: {}", e.getMessage(), e); return -1; } @@ -144,10 +141,7 @@ public int run(String[] args) { hadoopConf, FileSystem.get(hadoopConf), new QueryPcapFilter.Configurator()); - } catch (IOException | ClassNotFoundException e) { - LOGGER.error("Failed to execute query filter job: {}", e.getMessage(), e); - return -1; - } catch (InterruptedException e) { + } catch (IOException | ClassNotFoundException | InterruptedException e) { LOGGER.error("Failed to execute query filter job: {}", e.getMessage(), e); return -1; } diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java index e24b43f316..af1448fe3e 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java @@ -20,7 +20,6 @@ import static java.lang.Long.toUnsignedString; import static java.lang.String.format; -import static org.hamcrest.CoreMatchers.endsWith; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.startsWith; @@ -37,10 +36,9 @@ import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.JobStatus.State; import org.apache.metron.common.utils.timestamp.TimestampConverters; import org.apache.metron.job.JobStatus; -import org.apache.metron.job.JobStatus.STATE; +import org.apache.metron.job.JobStatus.State; import org.apache.metron.job.Statusable; import org.apache.metron.pcap.filter.PcapFilterConfigurator; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; @@ -106,7 +104,7 @@ public Job createJob(Path basePath, Path outputPath, long beginNS, long endN @Test public void job_succeeds_synchronously() throws Exception { when(job.isComplete()).thenReturn(true); - when(mrStatus.getState()).thenReturn(State.SUCCEEDED); + when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.SUCCEEDED); when(job.getStatus()).thenReturn(mrStatus); TestJob testJob = new TestJob(); Statusable statusable = testJob.query(basePath, @@ -121,19 +119,17 @@ public void job_succeeds_synchronously() throws Exception { true); verify(job, times(1)).waitForCompletion(true); JobStatus status = statusable.getStatus(); - Assert.assertThat(status.getState(), equalTo(STATE.SUCCEEDED)); + Assert.assertThat(status.getState(), equalTo(State.SUCCEEDED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); Assert.assertThat(status.getResultPath(), notNullValue()); Assert.assertThat(status.getResultPath().toString(), startsWith(expectedOutPath)); - Assert.assertThat(status.getPagedResults(), notNullValue()); - Assert.assertThat(((Path)status.getPagedResults().getPage(0)).getName(), endsWith("pcap-data-1234+0001.pcap")); } @Test public void job_fails_synchronously() throws Exception { when(job.isComplete()).thenReturn(true); - when(mrStatus.getState()).thenReturn(State.FAILED); + when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.FAILED); when(job.getStatus()).thenReturn(mrStatus); TestJob testJob = new TestJob(); Statusable statusable = testJob.query(basePath, @@ -147,7 +143,7 @@ public void job_fails_synchronously() throws Exception { new FixedPcapFilter.Configurator(), true); JobStatus status = statusable.getStatus(); - Assert.assertThat(status.getState(), equalTo(STATE.FAILED)); + Assert.assertThat(status.getState(), equalTo(State.FAILED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); Assert.assertThat(status.getResultPath(), notNullValue()); @@ -157,7 +153,7 @@ public void job_fails_synchronously() throws Exception { @Test public void job_fails_with_killed_status_synchronously() throws Exception { when(job.isComplete()).thenReturn(true); - when(mrStatus.getState()).thenReturn(State.KILLED); + when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.KILLED); when(job.getStatus()).thenReturn(mrStatus); TestJob testJob = new TestJob(); Statusable statusable = testJob.query(basePath, @@ -171,7 +167,7 @@ public void job_fails_with_killed_status_synchronously() throws Exception { new FixedPcapFilter.Configurator(), true); JobStatus status = statusable.getStatus(); - Assert.assertThat(status.getState(), equalTo(STATE.KILLED)); + Assert.assertThat(status.getState(), equalTo(State.KILLED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); Assert.assertThat(status.getResultPath(), notNullValue()); @@ -181,7 +177,7 @@ public void job_fails_with_killed_status_synchronously() throws Exception { @Test public void job_succeeds_asynchronously() throws Exception { when(job.isComplete()).thenReturn(true); - when(mrStatus.getState()).thenReturn(State.SUCCEEDED); + when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.SUCCEEDED); when(job.getStatus()).thenReturn(mrStatus); TestJob testJob = new TestJob(); Statusable statusable = testJob.query(basePath, @@ -195,7 +191,7 @@ public void job_succeeds_asynchronously() throws Exception { new FixedPcapFilter.Configurator(), false); JobStatus status = statusable.getStatus(); - Assert.assertThat(status.getState(), equalTo(STATE.SUCCEEDED)); + Assert.assertThat(status.getState(), equalTo(State.SUCCEEDED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); Assert.assertThat(status.getResultPath(), notNullValue()); @@ -205,7 +201,7 @@ public void job_succeeds_asynchronously() throws Exception { @Test public void job_reports_percent_complete() throws Exception { when(job.isComplete()).thenReturn(false); - when(mrStatus.getState()).thenReturn(State.RUNNING); + when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.RUNNING); when(job.getStatus()).thenReturn(mrStatus); TestJob testJob = new TestJob(); Statusable statusable = testJob.query(basePath, @@ -221,7 +217,7 @@ public void job_reports_percent_complete() throws Exception { when(job.mapProgress()).thenReturn(0.5f); when(job.reduceProgress()).thenReturn(0f); JobStatus status = statusable.getStatus(); - Assert.assertThat(status.getState(), equalTo(STATE.RUNNING)); + Assert.assertThat(status.getState(), equalTo(State.RUNNING)); Assert.assertThat(status.getPercentComplete(), equalTo(25.0)); when(job.mapProgress()).thenReturn(1.0f); when(job.reduceProgress()).thenReturn(0.5f); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java new file mode 100644 index 0000000000..997c5f79d8 --- /dev/null +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java @@ -0,0 +1,42 @@ +/** + * 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.metron.pcap; + +import java.util.List; +import org.apache.hadoop.fs.Path; +import org.apache.metron.job.Pageable; + +public class PcapFiles implements Pageable { + + private List files; + + public PcapFiles(List files) { + this.files = files; + } + + @Override + public Iterable asIterable() { + return files; + } + + @Override + public Path getPage(int num) { + return files.get(num); + } +} diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index 1992b6b1b2..b7a0edcd0e 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -51,9 +51,11 @@ import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; import org.apache.metron.common.hadoop.SequenceFileIterable; import org.apache.metron.job.JobStatus; -import org.apache.metron.job.JobStatus.STATE; +import org.apache.metron.job.JobStatus.State; +import org.apache.metron.job.Pageable; import org.apache.metron.job.Statusable; import org.apache.metron.pcap.PacketInfo; +import org.apache.metron.pcap.PcapFiles; import org.apache.metron.pcap.PcapHelper; import org.apache.metron.pcap.filter.PcapFilter; import org.apache.metron.pcap.filter.PcapFilterConfigurator; @@ -181,7 +183,7 @@ public SequenceFileIterable query(Path basePath conf, fs, filterImpl, true); JobStatus jobStatus = statusable.getStatus(); - if (jobStatus.getState() == STATE.SUCCEEDED) { + if (jobStatus.getState() == State.SUCCEEDED) { Path resultPath = jobStatus.getResultPath(); return readResults(resultPath, conf, fs); } else { @@ -205,7 +207,7 @@ public Statusable query(Path basePath, PcapFilterConfigurator filterImpl, boolean sync) throws IOException, ClassNotFoundException, InterruptedException { - String fileName = Joiner.on("_").join(beginNS, endNS, filterImpl.queryToString(fields), UUID.randomUUID().toString()); + String outputDirName = Joiner.on("_").join(beginNS, endNS, filterImpl.queryToString(fields), UUID.randomUUID().toString()); if(LOG.isDebugEnabled()) { DateFormat format = SimpleDateFormat.getDateTimeInstance( SimpleDateFormat.LONG , SimpleDateFormat.LONG @@ -214,7 +216,7 @@ public Statusable query(Path basePath, String to = format.format(new Date(Long.divideUnsigned(endNS, 1000000))); LOG.debug("Executing query {} on timerange from {} to {}", filterImpl.queryToString(fields), from, to); } - outputPath = new Path(baseOutputPath, fileName); + outputPath = new Path(baseOutputPath, outputDirName); job = createJob( basePath , outputPath , beginNS @@ -255,7 +257,7 @@ private SequenceFileIterable readResults(Path outputPath, Configuration config, return new SequenceFileIterable(files, config); } - public List writeResults(SequenceFileIterable results, ResultsWriter resultsWriter, + public Pageable writeResults(SequenceFileIterable results, ResultsWriter resultsWriter, Path outPath, int recPerFile, String prefix) throws IOException { List outFiles = new ArrayList<>(); try { @@ -279,7 +281,7 @@ public List writeResults(SequenceFileIterable results, ResultsWriter resul LOG.warn("Unable to cleanup files in HDFS", e); } } - return outFiles; + return new PcapFiles(outFiles); } /** @@ -340,27 +342,27 @@ public JobStatus getStatus() { // Note: this method is only reading state from the underlying job, so locking not needed JobStatus status = new JobStatus().withResultPath(outputPath); if (job == null) { - status.withPercentComplete(100).withState(STATE.SUCCEEDED); + status.withPercentComplete(100).withState(State.SUCCEEDED); } else { try { if (job.isComplete()) { status.withPercentComplete(100); switch (job.getStatus().getState()) { case SUCCEEDED: - status.withState(STATE.SUCCEEDED); + status.withState(State.SUCCEEDED); break; case FAILED: - status.withState(STATE.FAILED); + status.withState(State.FAILED); break; case KILLED: - status.withState(STATE.KILLED); + status.withState(State.KILLED); break; } } else { float mapProg = job.mapProgress(); float reduceProg = job.reduceProgress(); float totalProgress = ((mapProg / 2) + (reduceProg / 2)) * 100; - status.withPercentComplete(totalProgress).withState(STATE.RUNNING); + status.withPercentComplete(totalProgress).withState(State.RUNNING); } } catch (Exception e) { throw new RuntimeException("Error occurred while attempting to retrieve job status.", e); From cd302a989320a6a8970b18a94df8fbf4f11576b1 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Wed, 27 Jun 2018 09:12:42 -0600 Subject: [PATCH 06/36] change pom change with spaces to tabs --- metron-platform/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metron-platform/pom.xml b/metron-platform/pom.xml index 6866386330..cb64f9efa3 100644 --- a/metron-platform/pom.xml +++ b/metron-platform/pom.xml @@ -46,7 +46,7 @@ metron-enrichment metron-solr metron-parsers - metron-job + metron-job metron-pcap-backend metron-data-management metron-pcap From 7c1d4a0bedb23e86e323b0daec21c73113065360 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Wed, 27 Jun 2018 17:20:16 -0600 Subject: [PATCH 07/36] Address review comments. Fix local FS write path problem. --- .../apache/metron/common/utils/HDFSUtils.java | 25 +++++++---- .../metron/common/utils/HDFSUtilsTest.java | 13 +++++- .../java/org/apache/metron/job/JobStatus.java | 16 ++++++++ .../org/apache/metron/pcap/query/PcapCli.java | 8 ++-- .../org/apache/metron/pcap/PcapJobTest.java | 31 +++++++++++--- .../PcapTopologyIntegrationTest.java | 41 ++++++++++++------- .../apache/metron/pcap/query/PcapCliTest.java | 23 ++++++----- .../org/apache/metron/pcap/mr/PcapJob.java | 23 +++++++---- 8 files changed, 130 insertions(+), 50 deletions(-) diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java index 1d03a6d5a0..ae09edf881 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java @@ -30,8 +30,11 @@ public class HDFSUtils { /** - * Reads full HDFS FS file contents into a List of Strings. Initializes file system with default - * configuration. Opens and closes the file system on each call. Never null. + * Reads full file contents into a List of Strings. Reads from local FS if file:/// used as the + * scheme. Initializes file system with default configuration. + * Automatically handles file system translation for the provided path's scheme. + * Opens and closes the file system on each call. Never null. + * Null/empty scheme defaults to default configured FS. * * @param path path to file * @return file contents as a String @@ -42,8 +45,10 @@ public static List readFile(String path) throws IOException { } /** - * Reads full HDFS FS file contents into a String. Opens and closes the file system on each call. - * Never null. + * Reads full file contents into a String. Reads from local FS if file:/// used as the scheme. + * Opens and closes the file system on each call. + * Never null. Automatically handles file system translation for the provided path's scheme. + * Null/empty scheme defaults to default configured FS. * * @param config Hadoop configuration * @param path path to file @@ -51,14 +56,16 @@ public static List readFile(String path) throws IOException { * @throws IOException */ public static List readFile(Configuration config, String path) throws IOException { - FileSystem fs = FileSystem.newInstance(config); - Path hdfsPath = new Path(path); - FSDataInputStream inputStream = fs.open(hdfsPath); + Path inPath = new Path(path); + FileSystem fs = FileSystem.newInstance(inPath.toUri(), config); + FSDataInputStream inputStream = fs.open(inPath); return IOUtils.readLines(inputStream, "UTF-8"); } /** - * Write file to HDFS. Writes to local FS if file:// used as protocol. + * Write file to HDFS. Writes to local FS if file:/// used as the scheme. + * Automatically handles file system translation for the provided path's scheme. + * Null/empty scheme defaults to default configured FS. * * @param config filesystem configuration * @param bytes bytes to write @@ -66,8 +73,8 @@ public static List readFile(Configuration config, String path) throws IO * @throws IOException This is the generic Hadoop "everything is an IOException." */ public static void write(Configuration config, byte[] bytes, String path) throws IOException { - FileSystem fs = FileSystem.newInstance(config); Path outPath = new Path(path); + FileSystem fs = FileSystem.newInstance(outPath.toUri(), config); fs.mkdirs(outPath.getParent()); try (FSDataOutputStream outputStream = fs.create(outPath)) { outputStream.write(bytes); diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/HDFSUtilsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/HDFSUtilsTest.java index 53b6f299c2..a572e24408 100644 --- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/HDFSUtilsTest.java +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/HDFSUtilsTest.java @@ -37,7 +37,7 @@ public class HDFSUtilsTest { @Test public void writes_file_to_local_fs() throws Exception { String outText = "small brown bike and casket lottery"; - String outFile = tempDir.getRoot().getAbsolutePath() + "outfile.txt"; + String outFile = tempDir.getRoot().getAbsolutePath() + "/outfile.txt"; Configuration config = new Configuration(); config.set("fs.default.name", "file:///"); HDFSUtils.write(config, outText.getBytes(StandardCharsets.UTF_8), outFile); @@ -45,4 +45,15 @@ public void writes_file_to_local_fs() throws Exception { assertThat("Text should match", actual, equalTo(outText)); } + @Test + public void writes_file_to_local_fs_with_scheme_defined_only_in_uri() throws Exception { + String outText = "small brown bike and casket lottery"; + String outFile = tempDir.getRoot().getAbsolutePath() + "/outfile.txt"; + Configuration config = new Configuration(); + HDFSUtils.write(config, outText.getBytes(StandardCharsets.UTF_8), "file:///" + outFile); + + String actual = TestUtils.read(new File(outFile)); + assertThat("Text should match", actual, equalTo(outText)); + } + } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java index 8b67487706..ec006fb2c7 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java @@ -33,10 +33,17 @@ public enum State { KILLED } + private String jobId; private State state = State.NOT_RUNNING; private double percentComplete = 0.0; + private String description; private Path resultPath; + public JobStatus withJobId(String jobId) { + this.jobId = jobId; + return this; + } + public JobStatus withState(State state) { this.state = state; return this; @@ -47,6 +54,11 @@ public JobStatus withPercentComplete(double percentComplete) { return this; } + public JobStatus withDescription(String description) { + this.description = description; + return this; + } + public JobStatus withResultPath(Path resultPath) { this.resultPath = resultPath; return this; @@ -60,6 +72,10 @@ public double getPercentComplete() { return percentComplete; } + public String getDescription() { + return description; + } + public Path getResultPath() { return resultPath; } diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java index 61f83c61fd..e28b7f42c0 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.Arrays; +import java.util.Optional; import java.util.UUID; import org.apache.commons.cli.ParseException; import org.apache.commons.lang3.tuple.Pair; @@ -97,7 +98,7 @@ public int run(String[] args) { long endTime = time.getRight(); try { - results = jobRunner.query( + results = jobRunner.query(Optional.empty(), new Path(config.getBasePath()), new Path(config.getBaseOutputPath()), startTime, @@ -131,7 +132,7 @@ public int run(String[] args) { long endTime = time.getRight(); try { - results = jobRunner.query( + results = jobRunner.query(Optional.empty(), new Path(config.getBasePath()), new Path(config.getBaseOutputPath()), startTime, @@ -152,7 +153,8 @@ public int run(String[] args) { try { // write to local FS in the executing directory - jobRunner.writeResults(results, resultsWriter, new Path("file://."), + String execDir = System.getProperty("user.dir"); + jobRunner.writeResults(results, resultsWriter, new Path("file:///" + execDir), commonConfig.getNumRecordsPerFile(), commonConfig.getPrefix()); } catch (IOException e) { diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java index af1448fe3e..5a5d4063e6 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java @@ -30,12 +30,14 @@ import java.io.IOException; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobID; import org.apache.metron.common.utils.timestamp.TimestampConverters; import org.apache.metron.job.JobStatus; import org.apache.metron.job.JobStatus.State; @@ -55,6 +57,9 @@ public class PcapJobTest { private Job job; @Mock private org.apache.hadoop.mapreduce.JobStatus mrStatus; + @Mock + private JobID jobId; + private static final String JOB_ID_VAL = "job_abc_123"; private Path basePath; private Path baseOutPath; private long startTime; @@ -74,6 +79,8 @@ public void setup() { fixedFields = new HashMap<>(); fixedFields.put("ip_src_addr", "192.168.1.1"); hadoopConfig = new Configuration(); + when(jobId.toString()).thenReturn(JOB_ID_VAL); + when(mrStatus.getJobID()).thenReturn(jobId); } @Test @@ -94,7 +101,7 @@ public void partition_gives_value_in_range() throws Exception { private class TestJob extends PcapJob { @Override - public Job createJob(Path basePath, Path outputPath, long beginNS, long endNS, + public Job createJob(Optional jobName, Path basePath, Path outputPath, long beginNS, long endNS, int numReducers, T fields, Configuration conf, FileSystem fs, PcapFilterConfigurator filterImpl) throws IOException { return job; @@ -107,7 +114,9 @@ public void job_succeeds_synchronously() throws Exception { when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.SUCCEEDED); when(job.getStatus()).thenReturn(mrStatus); TestJob testJob = new TestJob(); - Statusable statusable = testJob.query(basePath, + Statusable statusable = testJob.query( + Optional.empty(), + basePath, baseOutPath, startTime, endTime, @@ -132,7 +141,9 @@ public void job_fails_synchronously() throws Exception { when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.FAILED); when(job.getStatus()).thenReturn(mrStatus); TestJob testJob = new TestJob(); - Statusable statusable = testJob.query(basePath, + Statusable statusable = testJob.query( + Optional.empty(), + basePath, baseOutPath, startTime, endTime, @@ -156,7 +167,9 @@ public void job_fails_with_killed_status_synchronously() throws Exception { when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.KILLED); when(job.getStatus()).thenReturn(mrStatus); TestJob testJob = new TestJob(); - Statusable statusable = testJob.query(basePath, + Statusable statusable = testJob.query( + Optional.empty(), + basePath, baseOutPath, startTime, endTime, @@ -180,7 +193,9 @@ public void job_succeeds_asynchronously() throws Exception { when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.SUCCEEDED); when(job.getStatus()).thenReturn(mrStatus); TestJob testJob = new TestJob(); - Statusable statusable = testJob.query(basePath, + Statusable statusable = testJob.query( + Optional.empty(), + basePath, baseOutPath, startTime, endTime, @@ -204,7 +219,9 @@ public void job_reports_percent_complete() throws Exception { when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.RUNNING); when(job.getStatus()).thenReturn(mrStatus); TestJob testJob = new TestJob(); - Statusable statusable = testJob.query(basePath, + Statusable statusable = testJob.query( + Optional.empty(), + basePath, baseOutPath, startTime, endTime, @@ -218,11 +235,13 @@ public void job_reports_percent_complete() throws Exception { when(job.reduceProgress()).thenReturn(0f); JobStatus status = statusable.getStatus(); Assert.assertThat(status.getState(), equalTo(State.RUNNING)); + Assert.assertThat(status.getDescription(), equalTo("map: 50.0%, reduce: 0.0%")); Assert.assertThat(status.getPercentComplete(), equalTo(25.0)); when(job.mapProgress()).thenReturn(1.0f); when(job.reduceProgress()).thenReturn(0.5f); status = statusable.getStatus(); Assert.assertThat(status.getPercentComplete(), equalTo(75.0)); + Assert.assertThat(status.getDescription(), equalTo("map: 100.0%, reduce: 50.0%")); } } diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java index 29c68d0606..b32fa1808f 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java @@ -29,11 +29,11 @@ import java.io.IOException; import java.util.AbstractMap; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import javax.annotation.Nullable; import kafka.consumer.ConsumerIterator; @@ -238,7 +238,8 @@ public ProcessorResult getResult() { { //Ensure that only two pcaps are returned when we look at 4 and 5 Iterable results = - job.query(new Path(outDir.getAbsolutePath()) + job.query(Optional.empty() + , new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(4, pcapEntries) , getTimestamp(5, pcapEntries) @@ -255,7 +256,8 @@ public ProcessorResult getResult() { // Ensure that only two pcaps are returned when we look at 4 and 5 // test with empty query filter Iterable results = - job.query(new Path(outDir.getAbsolutePath()) + job.query(Optional.empty() + , new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(4, pcapEntries) , getTimestamp(5, pcapEntries) @@ -271,7 +273,8 @@ public ProcessorResult getResult() { { //ensure that none get returned since that destination IP address isn't in the dataset Iterable results = - job.query(new Path(outDir.getAbsolutePath()) + job.query(Optional.empty() + , new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(1, pcapEntries) @@ -290,7 +293,8 @@ public ProcessorResult getResult() { // ensure that none get returned since that destination IP address isn't in the dataset // test with query filter Iterable results = - job.query(new Path(outDir.getAbsolutePath()) + job.query(Optional.empty() + , new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(1, pcapEntries) @@ -306,7 +310,8 @@ public ProcessorResult getResult() { { //same with protocol as before with the destination addr Iterable results = - job.query(new Path(outDir.getAbsolutePath()) + job.query(Optional.empty() + , new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(1, pcapEntries) @@ -325,7 +330,8 @@ public ProcessorResult getResult() { //same with protocol as before with the destination addr //test with query filter Iterable results = - job.query(new Path(outDir.getAbsolutePath()) + job.query(Optional.empty() + , new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(1, pcapEntries) @@ -341,7 +347,8 @@ public ProcessorResult getResult() { { //make sure I get them all. Iterable results = - job.query(new Path(outDir.getAbsolutePath()) + job.query(Optional.empty() + , new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 @@ -358,7 +365,8 @@ public ProcessorResult getResult() { //make sure I get them all. //with query filter Iterable results = - job.query(new Path(outDir.getAbsolutePath()) + job.query(Optional.empty() + , new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 @@ -373,7 +381,8 @@ public ProcessorResult getResult() { } { Iterable results = - job.query(new Path(outDir.getAbsolutePath()) + job.query(Optional.empty() + , new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 @@ -404,7 +413,8 @@ public boolean apply(@Nullable JSONObject input) { { //test with query filter and byte array matching Iterable results = - job.query(new Path(outDir.getAbsolutePath()) + job.query(Optional.empty() + , new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 @@ -423,7 +433,8 @@ public boolean apply(@Nullable JSONObject input) { { //test with query filter Iterable results = - job.query(new Path(outDir.getAbsolutePath()) + job.query(Optional.empty() + , new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 @@ -453,7 +464,8 @@ public boolean apply(@Nullable JSONObject input) { { //test with query filter Iterable results = - job.query(new Path(outDir.getAbsolutePath()) + job.query(Optional.empty() + , new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 @@ -483,7 +495,8 @@ public boolean apply(@Nullable JSONObject input) { { //test with query filter Iterable results = - job.query(new Path(outDir.getAbsolutePath()) + job.query(Optional.empty() + , new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java index fcb4d1995c..dc2434cd72 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java @@ -41,6 +41,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -67,11 +68,13 @@ public class PcapCliTest { private ResultsWriter resultsWriter; @Mock private Clock clock; + private String execDir; @Before public void setup() throws IOException { MockitoAnnotations.initMocks(this); doCallRealMethod().when(jobRunner).writeResults(anyObject(), anyObject(), anyObject(), anyInt(), anyObject()); + execDir = System.getProperty("user.dir"); } @Test @@ -103,11 +106,11 @@ public void runs_fixed_pcap_filter_job_with_default_argument_list() throws Excep put(PcapHelper.PacketFields.PACKET_FILTER.getName(), "`casey`"); }}; - when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); + when(jobRunner.query(eq(Optional.empty()), eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file://./pcap-data-random_prefix+0001.pcap")); + Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } @Test @@ -143,11 +146,11 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list_and_default_datef put(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName(), "true"); }}; - when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); + when(jobRunner.query(eq(Optional.empty()), eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file://./pcap-data-random_prefix+0001.pcap")); + Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } @Test @@ -186,11 +189,11 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list() throws Exceptio long startAsNanos = asNanos("2016-06-13-18:35.00", "yyyy-MM-dd-HH:mm.ss"); long endAsNanos = asNanos("2016-06-15-18:35.00", "yyyy-MM-dd-HH:mm.ss"); - when(jobRunner.query(eq(base_path), eq(base_output_path), eq(startAsNanos), eq(endAsNanos), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); + when(jobRunner.query(eq(Optional.empty()), eq(base_path), eq(base_output_path), eq(startAsNanos), eq(endAsNanos), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file://./pcap-data-random_prefix+0001.pcap")); + Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } private long asNanos(String inDate, String format) throws ParseException { @@ -219,11 +222,11 @@ public void runs_query_pcap_filter_job_with_default_argument_list() throws Excep Path base_output_path = new Path(CliParser.BASE_OUTPUT_PATH_DEFAULT); String query = "some query string"; - when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(QueryPcapFilter.Configurator.class))).thenReturn(iterable); + when(jobRunner.query(eq(Optional.empty()), eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(QueryPcapFilter.Configurator.class))).thenReturn(iterable); PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file://./pcap-data-random_prefix+0001.pcap")); + Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } @Test @@ -247,11 +250,11 @@ public void runs_query_pcap_filter_job_with_full_argument_list() throws Exceptio Path base_output_path = new Path("/base/output/path"); String query = "some query string"; - when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(QueryPcapFilter.Configurator.class))).thenReturn(iterable); + when(jobRunner.query(eq(Optional.empty()), eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(QueryPcapFilter.Configurator.class))).thenReturn(iterable); PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file://./pcap-data-random_prefix+0001.pcap")); + Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } // INVALID OPTION CHECKS diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index b7a0edcd0e..02fa8aa0da 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -32,6 +32,7 @@ import java.util.Date; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.UUID; import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; @@ -169,7 +170,8 @@ protected void reduce(LongWritable key, Iterable values, Context /** * Run query synchronously. */ - public SequenceFileIterable query(Path basePath + public SequenceFileIterable query(Optional jobName + ,Path basePath , Path baseOutputPath , long beginNS , long endNS @@ -179,7 +181,7 @@ public SequenceFileIterable query(Path basePath , FileSystem fs , PcapFilterConfigurator filterImpl ) throws IOException, ClassNotFoundException, InterruptedException { - Statusable statusable = query(basePath, baseOutputPath, beginNS, endNS, numReducers, fields, + Statusable statusable = query(jobName, basePath, baseOutputPath, beginNS, endNS, numReducers, fields, conf, fs, filterImpl, true); JobStatus jobStatus = statusable.getStatus(); @@ -196,7 +198,8 @@ public SequenceFileIterable query(Path basePath * Run query sync OR async based on flag. Async mode allows the client to check the returned * statusable object for status details. */ - public Statusable query(Path basePath, + public Statusable query(Optional jobName, + Path basePath, Path baseOutputPath, long beginNS, long endNS, @@ -217,7 +220,8 @@ public Statusable query(Path basePath, LOG.debug("Executing query {} on timerange from {} to {}", filterImpl.queryToString(fields), from, to); } outputPath = new Path(baseOutputPath, outputDirName); - job = createJob( basePath + job = createJob(jobName + , basePath , outputPath , beginNS , endNS @@ -287,7 +291,8 @@ public Pageable writeResults(SequenceFileIterable results, ResultsWriter r /** * Creates, but does not submit the job. */ - public Job createJob( Path basePath + public Job createJob(Optional jobName + ,Path basePath , Path outputPath , long beginNS , long endNS @@ -303,6 +308,7 @@ public Job createJob( Path basePath conf.set(WIDTH_CONF, "" + findWidth(beginNS, endNS, numReducers)); filterImpl.addToConfig(fields, conf); Job job = Job.getInstance(conf); + jobName.ifPresent(job::setJobName); job.setJarByClass(PcapJob.class); job.setMapperClass(PcapJob.PcapMapper.class); job.setMapOutputKeyClass(LongWritable.class); @@ -345,11 +351,12 @@ public JobStatus getStatus() { status.withPercentComplete(100).withState(State.SUCCEEDED); } else { try { + status.withJobId(job.getStatus().getJobID().toString()); if (job.isComplete()) { status.withPercentComplete(100); switch (job.getStatus().getState()) { case SUCCEEDED: - status.withState(State.SUCCEEDED); + status.withState(State.SUCCEEDED).withDescription(State.SUCCEEDED.toString()); break; case FAILED: status.withState(State.FAILED); @@ -362,7 +369,9 @@ public JobStatus getStatus() { float mapProg = job.mapProgress(); float reduceProg = job.reduceProgress(); float totalProgress = ((mapProg / 2) + (reduceProg / 2)) * 100; - status.withPercentComplete(totalProgress).withState(State.RUNNING); + String description = String.format("map: %s%%, reduce: %s%%", mapProg * 100, reduceProg * 100); + status.withPercentComplete(totalProgress).withState(State.RUNNING) + .withDescription(description); } } catch (Exception e) { throw new RuntimeException("Error occurred while attempting to retrieve job status.", e); From a00e300d8bd812589fd09970ce4db1f25dc29165 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Wed, 27 Jun 2018 21:31:02 -0600 Subject: [PATCH 08/36] Let's try this again. --- .../rest/service/impl/PcapServiceImpl.java | 11 +++-- .../apache/metron/rest/mock/MockPcapJob.java | 15 +++---- .../org/apache/metron/pcap/query/PcapCli.java | 5 +-- .../PcapTopologyIntegrationTest.java | 40 ++++++------------- .../apache/metron/pcap/query/PcapCliTest.java | 11 +++-- .../org/apache/metron/pcap/mr/PcapJob.java | 5 +-- 6 files changed, 33 insertions(+), 54 deletions(-) diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java index 4dae1e5ec6..dd4af5cb03 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java @@ -17,6 +17,11 @@ */ package org.apache.metron.rest.service.impl; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -35,12 +40,6 @@ import org.springframework.core.env.Environment; import org.springframework.stereotype.Service; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - @Service public class PcapServiceImpl implements PcapService { diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java index 3aa9ce37ed..a7eca319b4 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java @@ -17,6 +17,12 @@ */ package org.apache.metron.rest.mock; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.List; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -24,15 +30,6 @@ import org.apache.metron.pcap.filter.PcapFilterConfigurator; import org.apache.metron.pcap.mr.PcapJob; -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.Map; - -import static org.mockito.Matchers.anyList; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - public class MockPcapJob extends PcapJob { private String basePath; diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java index e28b7f42c0..0fda8011bf 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.Arrays; -import java.util.Optional; import java.util.UUID; import org.apache.commons.cli.ParseException; import org.apache.commons.lang3.tuple.Pair; @@ -98,7 +97,7 @@ public int run(String[] args) { long endTime = time.getRight(); try { - results = jobRunner.query(Optional.empty(), + results = jobRunner.query( new Path(config.getBasePath()), new Path(config.getBaseOutputPath()), startTime, @@ -132,7 +131,7 @@ public int run(String[] args) { long endTime = time.getRight(); try { - results = jobRunner.query(Optional.empty(), + results = jobRunner.query( new Path(config.getBasePath()), new Path(config.getBaseOutputPath()), startTime, diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java index b32fa1808f..c7292abc08 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java @@ -33,7 +33,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Properties; import javax.annotation.Nullable; import kafka.consumer.ConsumerIterator; @@ -238,8 +237,7 @@ public ProcessorResult getResult() { { //Ensure that only two pcaps are returned when we look at 4 and 5 Iterable results = - job.query(Optional.empty() - , new Path(outDir.getAbsolutePath()) + job.query(new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(4, pcapEntries) , getTimestamp(5, pcapEntries) @@ -256,8 +254,7 @@ public ProcessorResult getResult() { // Ensure that only two pcaps are returned when we look at 4 and 5 // test with empty query filter Iterable results = - job.query(Optional.empty() - , new Path(outDir.getAbsolutePath()) + job.query(new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(4, pcapEntries) , getTimestamp(5, pcapEntries) @@ -273,8 +270,7 @@ public ProcessorResult getResult() { { //ensure that none get returned since that destination IP address isn't in the dataset Iterable results = - job.query(Optional.empty() - , new Path(outDir.getAbsolutePath()) + job.query(new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(1, pcapEntries) @@ -293,8 +289,7 @@ public ProcessorResult getResult() { // ensure that none get returned since that destination IP address isn't in the dataset // test with query filter Iterable results = - job.query(Optional.empty() - , new Path(outDir.getAbsolutePath()) + job.query(new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(1, pcapEntries) @@ -310,8 +305,7 @@ public ProcessorResult getResult() { { //same with protocol as before with the destination addr Iterable results = - job.query(Optional.empty() - , new Path(outDir.getAbsolutePath()) + job.query(new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(1, pcapEntries) @@ -330,8 +324,7 @@ public ProcessorResult getResult() { //same with protocol as before with the destination addr //test with query filter Iterable results = - job.query(Optional.empty() - , new Path(outDir.getAbsolutePath()) + job.query(new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(1, pcapEntries) @@ -347,8 +340,7 @@ public ProcessorResult getResult() { { //make sure I get them all. Iterable results = - job.query(Optional.empty() - , new Path(outDir.getAbsolutePath()) + job.query(new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 @@ -365,8 +357,7 @@ public ProcessorResult getResult() { //make sure I get them all. //with query filter Iterable results = - job.query(Optional.empty() - , new Path(outDir.getAbsolutePath()) + job.query(new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 @@ -381,8 +372,7 @@ public ProcessorResult getResult() { } { Iterable results = - job.query(Optional.empty() - , new Path(outDir.getAbsolutePath()) + job.query(new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 @@ -413,8 +403,7 @@ public boolean apply(@Nullable JSONObject input) { { //test with query filter and byte array matching Iterable results = - job.query(Optional.empty() - , new Path(outDir.getAbsolutePath()) + job.query(new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 @@ -433,8 +422,7 @@ public boolean apply(@Nullable JSONObject input) { { //test with query filter Iterable results = - job.query(Optional.empty() - , new Path(outDir.getAbsolutePath()) + job.query(new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 @@ -464,8 +452,7 @@ public boolean apply(@Nullable JSONObject input) { { //test with query filter Iterable results = - job.query(Optional.empty() - , new Path(outDir.getAbsolutePath()) + job.query(new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 @@ -495,8 +482,7 @@ public boolean apply(@Nullable JSONObject input) { { //test with query filter Iterable results = - job.query(Optional.empty() - , new Path(outDir.getAbsolutePath()) + job.query(new Path(outDir.getAbsolutePath()) , new Path(queryDir.getAbsolutePath()) , getTimestamp(0, pcapEntries) , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java index dc2434cd72..3468a7c658 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java @@ -41,7 +41,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -106,7 +105,7 @@ public void runs_fixed_pcap_filter_job_with_default_argument_list() throws Excep put(PcapHelper.PacketFields.PACKET_FILTER.getName(), "`casey`"); }}; - when(jobRunner.query(eq(Optional.empty()), eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); + when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); @@ -146,7 +145,7 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list_and_default_datef put(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName(), "true"); }}; - when(jobRunner.query(eq(Optional.empty()), eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); + when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); @@ -189,7 +188,7 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list() throws Exceptio long startAsNanos = asNanos("2016-06-13-18:35.00", "yyyy-MM-dd-HH:mm.ss"); long endAsNanos = asNanos("2016-06-15-18:35.00", "yyyy-MM-dd-HH:mm.ss"); - when(jobRunner.query(eq(Optional.empty()), eq(base_path), eq(base_output_path), eq(startAsNanos), eq(endAsNanos), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); + when(jobRunner.query(eq(base_path), eq(base_output_path), eq(startAsNanos), eq(endAsNanos), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); @@ -222,7 +221,7 @@ public void runs_query_pcap_filter_job_with_default_argument_list() throws Excep Path base_output_path = new Path(CliParser.BASE_OUTPUT_PATH_DEFAULT); String query = "some query string"; - when(jobRunner.query(eq(Optional.empty()), eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(QueryPcapFilter.Configurator.class))).thenReturn(iterable); + when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(QueryPcapFilter.Configurator.class))).thenReturn(iterable); PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); @@ -250,7 +249,7 @@ public void runs_query_pcap_filter_job_with_full_argument_list() throws Exceptio Path base_output_path = new Path("/base/output/path"); String query = "some query string"; - when(jobRunner.query(eq(Optional.empty()), eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(QueryPcapFilter.Configurator.class))).thenReturn(iterable); + when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(QueryPcapFilter.Configurator.class))).thenReturn(iterable); PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index 02fa8aa0da..269f69b098 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -170,8 +170,7 @@ protected void reduce(LongWritable key, Iterable values, Context /** * Run query synchronously. */ - public SequenceFileIterable query(Optional jobName - ,Path basePath + public SequenceFileIterable query(Path basePath , Path baseOutputPath , long beginNS , long endNS @@ -181,7 +180,7 @@ public SequenceFileIterable query(Optional jobName , FileSystem fs , PcapFilterConfigurator filterImpl ) throws IOException, ClassNotFoundException, InterruptedException { - Statusable statusable = query(jobName, basePath, baseOutputPath, beginNS, endNS, numReducers, fields, + Statusable statusable = query(Optional.empty(), basePath, baseOutputPath, beginNS, endNS, numReducers, fields, conf, fs, filterImpl, true); JobStatus jobStatus = statusable.getStatus(); From ba6cf9030f45b6b1a3cf270a376e4b1f5e67d6fb Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Fri, 29 Jun 2018 06:14:44 -0600 Subject: [PATCH 09/36] Job manager and job service implementations --- .../org/apache/metron/job/JobException.java | 31 ++++ .../java/org/apache/metron/job/JobStatus.java | 4 + .../org/apache/metron/job/Statusable.java | 16 +- .../apache/metron/job/manager/JobManager.java | 66 +++++++++ .../metron/job/service/HdfsJobService.java | 97 ++++++++++++ .../apache/metron/job/service/JobService.java | 33 +++++ .../job/service/JobServiceStrategies.java | 52 +++++++ .../metron/job/manager/JobManagerTest.java | 115 +++++++++++++++ .../job/service/HdfsJobServiceTest.java | 103 +++++++++++++ .../org/apache/metron/pcap/PcapJobTest.java | 5 +- .../org/apache/metron/pcap/mr/PcapJob.java | 47 +++++- .../metron/pcap/mr/PcapMRJobConfig.java | 139 ++++++++++++++++++ 12 files changed, 699 insertions(+), 9 deletions(-) create mode 100644 metron-platform/metron-job/src/main/java/org/apache/metron/job/JobException.java create mode 100644 metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java create mode 100644 metron-platform/metron-job/src/main/java/org/apache/metron/job/service/HdfsJobService.java create mode 100644 metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobService.java create mode 100644 metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobServiceStrategies.java create mode 100644 metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/JobManagerTest.java create mode 100644 metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java create mode 100644 metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapMRJobConfig.java diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobException.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobException.java new file mode 100644 index 0000000000..10096cd2ba --- /dev/null +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobException.java @@ -0,0 +1,31 @@ +/** + * 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.metron.job; + +public class JobException extends Exception { + + public JobException(String message) { + super(message); + } + + public JobException(String message, Throwable cause) { + super(message, cause); + } + +} diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java index ec006fb2c7..9d0e7c44ce 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java @@ -64,6 +64,10 @@ public JobStatus withResultPath(Path resultPath) { return this; } + public String getJobId() { + return jobId; + } + public State getState() { return state; } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java index 7a8fc02cef..0808dddd2b 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java @@ -18,7 +18,6 @@ package org.apache.metron.job; -import java.io.IOException; import java.util.Map; /** @@ -26,6 +25,13 @@ */ public interface Statusable { + enum JobType { + MAP_REDUCE, + SPARK; + } + + JobType getJobType(); + /** * Current job status. * @@ -43,7 +49,7 @@ public interface Statusable { /** * Kill job. */ - void kill() throws IOException; + void kill() throws JobException; /** * Validate job after submitted. @@ -53,4 +59,10 @@ public interface Statusable { */ boolean validate(Map configuration); + /** + * Submit the job. + * + * @return self + */ + Statusable submit() throws JobException; } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java new file mode 100644 index 0000000000..a4ab7cb910 --- /dev/null +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java @@ -0,0 +1,66 @@ +/** + * 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.metron.job.manager; + +import java.util.Map; +import org.apache.metron.job.JobException; +import org.apache.metron.job.JobStatus; +import org.apache.metron.job.Statusable; +import org.apache.metron.job.service.JobService; +import org.apache.metron.job.service.JobServiceStrategies; + +public class JobManager { + + private JobService jobs; + + public JobManager() { + jobs = JobServiceStrategies.HDFS; + } + + public JobManager(JobService jobs) { + this.jobs = jobs; + } + + public JobStatus submit(Statusable job, Map configuration, String username) + throws JobException { + JobStatus status = new JobStatus(); + if (job.validate(configuration)) { + status = job.submit().getStatus(); + } + jobs.add(job, status.getJobId(), username); + return status; + } + + public JobStatus getStatus(String username, String jobId) { + return jobs.getJob(username, jobId).getStatus(); + } + + public boolean done(String username, String jobId) { + return jobs.getJob(username, jobId).isDone(); + } + + public void killJob(String username, String jobId) throws JobException { + jobs.getJob(username, jobId).kill(); + } + + public Statusable getJob(String username, String jobId) { + return jobs.getJob(username, jobId); + } + +} diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/HdfsJobService.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/HdfsJobService.java new file mode 100644 index 0000000000..27806ec5f4 --- /dev/null +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/HdfsJobService.java @@ -0,0 +1,97 @@ +/** + * 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.metron.job.service; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.HashMap; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.metron.common.utils.HDFSUtils; +import org.apache.metron.job.Statusable; +import org.apache.metron.job.Statusable.JobType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * HDFS-backed implementation of a job service + */ +public class HdfsJobService implements JobService { + + private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + private Map> jobs; + private String basePath; + private String jobsPath; + + public HdfsJobService() { + jobs = new HashMap<>(); + basePath = "/apps/metron"; + jobsPath = basePath + "/jobs"; + } + + @Override + public void configure(Map config) { + if (config != null) { + if (config.containsKey("basePath")) { + basePath = (String) config.get("basePath"); + jobsPath = basePath + "/jobs"; + } + } + } + + @Override + public void add(Statusable job, String username, String jobId) { + Map jobMapping = new HashMap<>(); + jobMapping.put(jobId, job); + jobs.put(username, jobMapping); + try { + writeToHdfs(jobsPath, job.getJobType(), username, jobId); + } catch (IOException e) { + LOG.error("Unable to save running job information to HDFS: {}, {}, {}", job.getJobType(), + username, jobId, e); + } + } + + private void writeToHdfs(String basePath, JobType jobType, String username, String jobId) + throws IOException { + String path = String.format("%s/%s/%s/%s", basePath, username, jobType.toString(), jobId); + HDFSUtils.write(new Configuration(), new byte[]{}, path); + } + + @Override + public boolean jobExists(String username, String jobId) { + if (jobs.containsKey(username)) { + Map jobsByUser = jobs.get(username); + return jobsByUser.containsKey(jobId); + } else { + return false; + } + } + + @Override + public Statusable getJob(String username, String jobId) { + if (jobs.containsKey(username)) { + Map jobsByUser = jobs.get(username); + return jobsByUser.get(jobId); + } else { + return null; + } + } + +} diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobService.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobService.java new file mode 100644 index 0000000000..da21249d01 --- /dev/null +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobService.java @@ -0,0 +1,33 @@ +/** + * 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.metron.job.service; + +import java.util.Map; +import org.apache.metron.job.Statusable; + +public interface JobService { + + void configure(Map config); + + void add(Statusable job, String username, String jobId); + + boolean jobExists(String username, String jobId); + + Statusable getJob(String username, String jobId); +} diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobServiceStrategies.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobServiceStrategies.java new file mode 100644 index 0000000000..e4da37f6fe --- /dev/null +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobServiceStrategies.java @@ -0,0 +1,52 @@ +/** + * 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.metron.job.service; + +import java.util.Map; +import org.apache.metron.job.Statusable; + +public enum JobServiceStrategies implements JobService { + HDFS(new HdfsJobService()); + + private JobService jobService; + + JobServiceStrategies(JobService jobService) { + this.jobService = jobService; + } + + @Override + public void configure(Map config) { + jobService.configure(config); + } + + @Override + public void add(Statusable job, String username, String jobId) { + jobService.add(job, username, jobId); + } + + @Override + public boolean jobExists(String username, String jobId) { + return jobService.jobExists(username, jobId); + } + + @Override + public Statusable getJob(String username, String jobId) { + return jobService.getJob(username, jobId); + } +} diff --git a/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/JobManagerTest.java b/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/JobManagerTest.java new file mode 100644 index 0000000000..04ced113d9 --- /dev/null +++ b/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/JobManagerTest.java @@ -0,0 +1,115 @@ +package org.apache.metron.job.manager; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.util.HashMap; +import org.apache.metron.job.JobException; +import org.apache.metron.job.JobStatus; +import org.apache.metron.job.JobStatus.State; +import org.apache.metron.job.Statusable; +import org.apache.metron.job.manager.JobManager; +import org.apache.metron.job.service.JobService; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +/** + * 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. + */ + +public class JobManagerTest { + + private JobManager jm; + @Mock + private JobService jobService; + @Mock + private Statusable job; + private String username; + private String jobId; + + @Before + public void setup() { + MockitoAnnotations.initMocks(this); + jm = new JobManager(jobService); + username = "userabc"; + jobId = "job_abc"; + } + + @Test + public void submits_job_and_returns_status() throws JobException { + when(job.validate(any())).thenReturn(true); + when(job.submit()).thenReturn(job); + when(job.getStatus()).thenReturn(new JobStatus().withState(State.SUCCEEDED).withJobId(jobId)); + JobStatus status = jm.submit(job, new HashMap<>(), username); + verify(jobService).add(job, jobId, username); + assertThat(status.getState(), equalTo(State.SUCCEEDED)); + } + + @Test + public void returns_job_status() { + JobStatus expected = new JobStatus().withState(State.SUCCEEDED).withJobId(jobId); + when(job.getStatus()).thenReturn(expected); + when(jobService.getJob(username, jobId)).thenReturn(job); + JobStatus status = jm.getStatus(username, jobId); + assertThat(status, equalTo(expected)); + } + + @Test + public void returns_job_is_done() { + JobStatus expected = new JobStatus().withState(State.SUCCEEDED).withJobId(jobId); + when(job.isDone()).thenReturn(true); + when(jobService.getJob(username, jobId)).thenReturn(job); + boolean done = jm.done(username, jobId); + assertThat(done, equalTo(true)); + } + + @Test + public void kills_job() throws JobException { + when(jobService.getJob(username, jobId)).thenReturn(job); + jm.killJob(username, jobId); + verify(job).kill(); + } + + @Test + public void returns_statusable_job() { + when(job.isDone()).thenReturn(true); + when(jobService.getJob(username, jobId)).thenReturn(job); + Statusable done = jm.getJob(username, jobId); + assertThat(done, equalTo(job)); + } + + @Rule + public ExpectedException exception = ExpectedException.none(); + + @Test + public void job_submission_exception_returned() throws JobException { + JobException expected = new JobException("test exception"); + when(job.validate(any())).thenReturn(true); + when(job.submit()).thenThrow(expected); + exception.expect(JobException.class); + exception.expect(equalTo(expected)); + jm.submit(job, new HashMap<>(), username); + } + +} diff --git a/metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java b/metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java new file mode 100644 index 0000000000..1a89c4b370 --- /dev/null +++ b/metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java @@ -0,0 +1,103 @@ +/** + * 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.metron.job.service; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.when; + +import java.io.File; +import java.util.HashMap; +import java.util.Map; +import org.apache.metron.job.Statusable; +import org.apache.metron.job.Statusable.JobType; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +public class HdfsJobServiceTest { + + @Rule + public TemporaryFolder tempDir = new TemporaryFolder(); + + @Mock + private Statusable job; + private String username; + private String jobId; + private String basePath; + private HdfsJobService js; + + @Before + public void setup() { + MockitoAnnotations.initMocks(this); + username = "user123"; + jobId = "job_abc_123"; + when(job.getJobType()).thenReturn(JobType.MAP_REDUCE); + basePath = tempDir.getRoot().getAbsolutePath(); + Map config = new HashMap<>(); + config.put("basePath", basePath); + js = new HdfsJobService(); + js.configure(config); + } + + @Test + public void adds_job() { + js.add(job, username, jobId); + Statusable actual = js.getJob(username, jobId); + assertThat(actual, equalTo(job)); + } + + @Test + public void job_exists_true_for_submitted_jobs() { + js.add(job, username, jobId); + boolean actual = js.jobExists(username, jobId); + assertThat(actual, equalTo(true)); + } + + @Test + public void job_exists_false_for_non_existent_jobs() { + boolean actual = js.jobExists(username, jobId); + assertThat(actual, equalTo(false)); + js.add(job, username, jobId); + actual = js.jobExists(username, "this_job_id_does_not_exist"); + assertThat(actual, equalTo(false)); + } + + @Test + public void returns_null_for_non_existent_job() { + Statusable actual = js.getJob(username, jobId); + assertThat(actual, equalTo(null)); + js.add(job, username, jobId); + actual = js.getJob(username, "this_job_id_does_not_exist"); + assertThat(actual, equalTo(null)); + } + + @Test + public void writes_job_info_to_hdfs() { + // /base/path/jobs/metron_user/MAP_REDUCE/job_abc_123 + js.add(job, username, jobId); + File jobFile = new File(String.format("%s/jobs/%s/%s/%s", basePath, username, JobType.MAP_REDUCE.toString(), jobId)); + assertThat("File should exist", jobFile.exists(), equalTo(true)); + assertThat("File should be a file", jobFile.isFile(), equalTo(true)); + } + +} diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java index 5a5d4063e6..ac08f92ca0 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java @@ -59,7 +59,7 @@ public class PcapJobTest { private org.apache.hadoop.mapreduce.JobStatus mrStatus; @Mock private JobID jobId; - private static final String JOB_ID_VAL = "job_abc_123"; + private String jobIdVal = "job_abc_123"; private Path basePath; private Path baseOutPath; private long startTime; @@ -79,7 +79,7 @@ public void setup() { fixedFields = new HashMap<>(); fixedFields.put("ip_src_addr", "192.168.1.1"); hadoopConfig = new Configuration(); - when(jobId.toString()).thenReturn(JOB_ID_VAL); + when(jobId.toString()).thenReturn(jobIdVal); when(mrStatus.getJobID()).thenReturn(jobId); } @@ -130,6 +130,7 @@ public void job_succeeds_synchronously() throws Exception { JobStatus status = statusable.getStatus(); Assert.assertThat(status.getState(), equalTo(State.SUCCEEDED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); + Assert.assertThat(status.getJobId(), equalTo(jobIdVal)); String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); Assert.assertThat(status.getResultPath(), notNullValue()); Assert.assertThat(status.getResultPath().toString(), startsWith(expectedOutPath)); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index 269f69b098..bc09f9cefb 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -51,6 +51,7 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; import org.apache.metron.common.hadoop.SequenceFileIterable; +import org.apache.metron.job.JobException; import org.apache.metron.job.JobStatus; import org.apache.metron.job.JobStatus.State; import org.apache.metron.job.Pageable; @@ -74,6 +75,7 @@ public class PcapJob implements Statusable { public static final String WIDTH_CONF = "width"; private Job job; // store a running MR job reference for async status check private Path outputPath; + private PcapMRJobConfig config; public static enum PCAP_COUNTER { MALFORMED_PACKET_COUNT @@ -167,6 +169,13 @@ protected void reduce(LongWritable key, Iterable values, Context } } + public PcapJob() { + } + + public PcapJob(PcapMRJobConfig config) { + this.config = config; + } + /** * Run query synchronously. */ @@ -180,9 +189,8 @@ public SequenceFileIterable query(Path basePath , FileSystem fs , PcapFilterConfigurator filterImpl ) throws IOException, ClassNotFoundException, InterruptedException { - Statusable statusable = query(Optional.empty(), basePath, baseOutputPath, beginNS, endNS, numReducers, fields, - conf, - fs, filterImpl, true); + Statusable statusable = query(Optional.empty(), basePath, baseOutputPath, beginNS, endNS, + numReducers, fields, conf, fs, filterImpl, true); JobStatus jobStatus = statusable.getStatus(); if (jobStatus.getState() == State.SUCCEEDED) { Path resultPath = jobStatus.getResultPath(); @@ -260,6 +268,26 @@ private SequenceFileIterable readResults(Path outputPath, Configuration config, return new SequenceFileIterable(files, config); } + @Override + public Statusable submit() throws JobException { + try { + return query( + config.getJobName(), + config.getBasePath(), + config.getBaseOutputPath(), + config.getBeginNS(), + config.getEndNS(), + config.getNumReducers(), + config.getFields(), + config.getConf(), + config.getFs(), + config.getFilterImpl(), + config.isSynchronous()); + } catch (IOException | ClassNotFoundException | InterruptedException e) { + throw new JobException("Unable to run pcap query.", e); + } + } + public Pageable writeResults(SequenceFileIterable results, ResultsWriter resultsWriter, Path outPath, int recPerFile, String prefix) throws IOException { List outFiles = new ArrayList<>(); @@ -342,6 +370,11 @@ protected Iterable listFiles(FileSystem fs, Path basePath) throws IOExcept return ret; } + @Override + public JobType getJobType() { + return JobType.MAP_REDUCE; + } + @Override public JobStatus getStatus() { // Note: this method is only reading state from the underlying job, so locking not needed @@ -390,8 +423,12 @@ public boolean isDone() { } @Override - public void kill() throws IOException { - job.killJob(); + public void kill() throws JobException { + try { + job.killJob(); + } catch (IOException e) { + throw new JobException("Unable to kill pcap job.", e); + } } @Override diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapMRJobConfig.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapMRJobConfig.java new file mode 100644 index 0000000000..8057096d51 --- /dev/null +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapMRJobConfig.java @@ -0,0 +1,139 @@ +/** + * 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.metron.pcap.mr; + +import java.util.Optional; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.metron.pcap.filter.PcapFilterConfigurator; + +public class PcapMRJobConfig { + + private Optional jobName; + private Path basePath; + private Path baseOutputPath; + private long beginNS; + private long endNS; + private int numReducers; + private T fields; + private Configuration conf; + private FileSystem fs; + private PcapFilterConfigurator filterImpl; + private boolean synchronous; + + public Optional getJobName() { + return jobName; + } + + public PcapMRJobConfig setJobName(Optional jobName) { + this.jobName = jobName; + return this; + } + + public Path getBasePath() { + return basePath; + } + + public PcapMRJobConfig setBasePath(Path basePath) { + this.basePath = basePath; + return this; + } + + public Path getBaseOutputPath() { + return baseOutputPath; + } + + public PcapMRJobConfig setBaseOutputPath(Path baseOutputPath) { + this.baseOutputPath = baseOutputPath; + return this; + } + + public long getBeginNS() { + return beginNS; + } + + public PcapMRJobConfig setBeginNS(long beginNS) { + this.beginNS = beginNS; + return this; + } + + public long getEndNS() { + return endNS; + } + + public PcapMRJobConfig setEndNS(long endNS) { + this.endNS = endNS; + return this; + } + + public int getNumReducers() { + return numReducers; + } + + public PcapMRJobConfig setNumReducers(int numReducers) { + this.numReducers = numReducers; + return this; + } + + public T getFields() { + return fields; + } + + public PcapMRJobConfig setFields(T fields) { + this.fields = fields; + return this; + } + + public Configuration getConf() { + return conf; + } + + public PcapMRJobConfig setConf(Configuration conf) { + this.conf = conf; + return this; + } + + public FileSystem getFs() { + return fs; + } + + public PcapMRJobConfig setFs(FileSystem fs) { + this.fs = fs; + return this; + } + + public PcapFilterConfigurator getFilterImpl() { + return filterImpl; + } + + public PcapMRJobConfig setFilterImpl(PcapFilterConfigurator filterImpl) { + this.filterImpl = filterImpl; + return this; + } + + public boolean isSynchronous() { + return synchronous; + } + + public PcapMRJobConfig setSynchronous(boolean synchronous) { + this.synchronous = synchronous; + return this; + } +} From 157aa1e3715d77373089a5c3eacfcc1dcd8d0a08 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Thu, 5 Jul 2018 11:58:55 -0600 Subject: [PATCH 10/36] Refactor jobs to have a finalize step --- .../metron/rest/model/pcap/PcapRequest.java | 2 + .../java/org/apache/metron/job/JobStatus.java | 10 +- .../org/apache/metron/job/Statusable.java | 25 +++- .../apache/metron/job/manager/JobManager.java | 19 +-- .../metron/job/service/HdfsJobService.java | 17 ++- .../apache/metron/job/service/JobService.java | 7 +- .../job/service/JobServiceStrategies.java | 11 +- .../metron/job/writer/PrefixStrategy.java | 26 ++++ .../metron/job/writer/ResultsWriter.java | 35 +++++ .../metron/job/manager/JobManagerTest.java | 14 +- .../job/service/HdfsJobServiceTest.java | 48 ++++--- .../org/apache/metron/pcap/query/PcapCli.java | 16 ++- .../org/apache/metron/pcap/PcapJobTest.java | 16 +-- .../apache/metron/pcap/query/PcapCliTest.java | 16 +-- .../org/apache/metron/pcap/mr/PcapJob.java | 123 +++++++++++------- .../metron/pcap/mr/PcapMRJobConfig.java | 64 +++++++-- ...ultsWriter.java => PcapResultsWriter.java} | 32 ++++- 17 files changed, 333 insertions(+), 148 deletions(-) create mode 100644 metron-platform/metron-job/src/main/java/org/apache/metron/job/writer/PrefixStrategy.java create mode 100644 metron-platform/metron-job/src/main/java/org/apache/metron/job/writer/ResultsWriter.java rename metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/writer/{ResultsWriter.java => PcapResultsWriter.java} (75%) diff --git a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java index 6062f94d6e..2648306d0c 100644 --- a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java +++ b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java @@ -17,6 +17,8 @@ */ package org.apache.metron.rest.model.pcap; +// TODO reconcile with pcapmrjob + public class PcapRequest { private String baseOutputPath; diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java index 9d0e7c44ce..4d61df3d57 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java @@ -37,7 +37,7 @@ public enum State { private State state = State.NOT_RUNNING; private double percentComplete = 0.0; private String description; - private Path resultPath; + private Path interimResultPath; public JobStatus withJobId(String jobId) { this.jobId = jobId; @@ -59,8 +59,8 @@ public JobStatus withDescription(String description) { return this; } - public JobStatus withResultPath(Path resultPath) { - this.resultPath = resultPath; + public JobStatus withInterimResultPath(Path interimResultPath) { + this.interimResultPath = interimResultPath; return this; } @@ -80,8 +80,8 @@ public String getDescription() { return description; } - public Path getResultPath() { - return resultPath; + public Path getInterimResultPath() { + return interimResultPath; } } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java index 0808dddd2b..02b1ab6f1c 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java @@ -23,13 +23,25 @@ /** * Abstraction for getting status on running jobs. Also provides options for killing and validating. */ -public interface Statusable { +public interface Statusable { enum JobType { MAP_REDUCE, SPARK; } + /** + * Submit the job. + * + * @return self + */ + Statusable submit() throws JobException; + + /** + * Execution framework type of this job. + * + * @return type of job + */ JobType getJobType(); /** @@ -37,14 +49,14 @@ enum JobType { * * @return status */ - JobStatus getStatus(); + JobStatus getStatus() throws JobException; /** * Completion flag. * * @return true if job is completed, whether KILLED, FAILED, SUCCEEDED. False otherwise. */ - boolean isDone(); + boolean isDone() throws JobException; /** * Kill job. @@ -60,9 +72,10 @@ enum JobType { boolean validate(Map configuration); /** - * Submit the job. + * Finalize job results. Any post-processing is done here. * - * @return self + * @return Results in a pageable fashion. */ - Statusable submit() throws JobException; + Pageable finalizeJob() throws JobException; + } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java index a4ab7cb910..57e9d9f494 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java @@ -23,21 +23,16 @@ import org.apache.metron.job.JobStatus; import org.apache.metron.job.Statusable; import org.apache.metron.job.service.JobService; -import org.apache.metron.job.service.JobServiceStrategies; -public class JobManager { +public class JobManager { - private JobService jobs; + private JobService jobs; - public JobManager() { - jobs = JobServiceStrategies.HDFS; - } - - public JobManager(JobService jobs) { + public JobManager(JobService jobs) { this.jobs = jobs; } - public JobStatus submit(Statusable job, Map configuration, String username) + public JobStatus submit(Statusable job, Map configuration, String username) throws JobException { JobStatus status = new JobStatus(); if (job.validate(configuration)) { @@ -47,11 +42,11 @@ public JobStatus submit(Statusable job, Map configuration, Strin return status; } - public JobStatus getStatus(String username, String jobId) { + public JobStatus getStatus(String username, String jobId) throws JobException { return jobs.getJob(username, jobId).getStatus(); } - public boolean done(String username, String jobId) { + public boolean done(String username, String jobId) throws JobException { return jobs.getJob(username, jobId).isDone(); } @@ -59,7 +54,7 @@ public void killJob(String username, String jobId) throws JobException { jobs.getJob(username, jobId).kill(); } - public Statusable getJob(String username, String jobId) { + public Statusable getJob(String username, String jobId) { return jobs.getJob(username, jobId); } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/HdfsJobService.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/HdfsJobService.java index 27806ec5f4..ca29b16246 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/HdfsJobService.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/HdfsJobService.java @@ -23,6 +23,7 @@ import java.util.HashMap; import java.util.Map; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.metron.common.utils.HDFSUtils; import org.apache.metron.job.Statusable; import org.apache.metron.job.Statusable.JobType; @@ -32,10 +33,10 @@ /** * HDFS-backed implementation of a job service */ -public class HdfsJobService implements JobService { +public class HdfsJobService implements JobService { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private Map> jobs; + private Map>> jobs; private String basePath; private String jobsPath; @@ -56,8 +57,8 @@ public void configure(Map config) { } @Override - public void add(Statusable job, String username, String jobId) { - Map jobMapping = new HashMap<>(); + public void add(Statusable job, String username, String jobId) { + Map> jobMapping = getJobsByUser(username); jobMapping.put(jobId, job); jobs.put(username, jobMapping); try { @@ -68,6 +69,10 @@ public void add(Statusable job, String username, String jobId) { } } + private Map> getJobsByUser(String username) { + return jobs.getOrDefault(username, new HashMap<>()); + } + private void writeToHdfs(String basePath, JobType jobType, String username, String jobId) throws IOException { String path = String.format("%s/%s/%s/%s", basePath, username, jobType.toString(), jobId); @@ -77,7 +82,7 @@ private void writeToHdfs(String basePath, JobType jobType, String username, Stri @Override public boolean jobExists(String username, String jobId) { if (jobs.containsKey(username)) { - Map jobsByUser = jobs.get(username); + Map> jobsByUser = jobs.get(username); return jobsByUser.containsKey(jobId); } else { return false; @@ -87,7 +92,7 @@ public boolean jobExists(String username, String jobId) { @Override public Statusable getJob(String username, String jobId) { if (jobs.containsKey(username)) { - Map jobsByUser = jobs.get(username); + Map> jobsByUser = jobs.get(username); return jobsByUser.get(jobId); } else { return null; diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobService.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobService.java index da21249d01..0a1013d34c 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobService.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobService.java @@ -21,13 +21,14 @@ import java.util.Map; import org.apache.metron.job.Statusable; -public interface JobService { +public interface JobService { void configure(Map config); - void add(Statusable job, String username, String jobId); + void add(Statusable job, String username, String jobId); boolean jobExists(String username, String jobId); - Statusable getJob(String username, String jobId); + Statusable getJob(String username, String jobId); + } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobServiceStrategies.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobServiceStrategies.java index e4da37f6fe..d3685371bd 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobServiceStrategies.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobServiceStrategies.java @@ -19,14 +19,15 @@ package org.apache.metron.job.service; import java.util.Map; +import org.apache.hadoop.fs.Path; import org.apache.metron.job.Statusable; -public enum JobServiceStrategies implements JobService { +public enum JobServiceStrategies implements JobService { HDFS(new HdfsJobService()); - private JobService jobService; + private JobService jobService; - JobServiceStrategies(JobService jobService) { + JobServiceStrategies(JobService jobService) { this.jobService = jobService; } @@ -36,7 +37,7 @@ public void configure(Map config) { } @Override - public void add(Statusable job, String username, String jobId) { + public void add(Statusable job, String username, String jobId) { jobService.add(job, username, jobId); } @@ -46,7 +47,7 @@ public boolean jobExists(String username, String jobId) { } @Override - public Statusable getJob(String username, String jobId) { + public Statusable getJob(String username, String jobId) { return jobService.getJob(username, jobId); } } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/writer/PrefixStrategy.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/writer/PrefixStrategy.java new file mode 100644 index 0000000000..af7ce042e4 --- /dev/null +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/writer/PrefixStrategy.java @@ -0,0 +1,26 @@ +/** + * 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.metron.job.writer; + +import java.util.function.Function; +import org.apache.metron.common.system.Clock; + +public interface PrefixStrategy extends Function { + +} diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/writer/ResultsWriter.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/writer/ResultsWriter.java new file mode 100644 index 0000000000..8ab4b18f76 --- /dev/null +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/writer/ResultsWriter.java @@ -0,0 +1,35 @@ +/** + * 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.metron.job.writer; + +import java.io.IOException; +import java.util.List; + +public interface ResultsWriter { + + /** + * Write out results. + * + * @param data data to write. + * @param outPath where to write the data to. + * @throws IOException I/O issue encountered. + */ + void write(List data, String outPath) throws IOException; + +} diff --git a/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/JobManagerTest.java b/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/JobManagerTest.java index 04ced113d9..a4e527c6b4 100644 --- a/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/JobManagerTest.java +++ b/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/JobManagerTest.java @@ -7,11 +7,11 @@ import static org.mockito.Mockito.when; import java.util.HashMap; +import org.apache.hadoop.fs.Path; import org.apache.metron.job.JobException; import org.apache.metron.job.JobStatus; import org.apache.metron.job.JobStatus.State; import org.apache.metron.job.Statusable; -import org.apache.metron.job.manager.JobManager; import org.apache.metron.job.service.JobService; import org.junit.Before; import org.junit.Rule; @@ -40,9 +40,9 @@ public class JobManagerTest { - private JobManager jm; + private JobManager jm; @Mock - private JobService jobService; + private JobService jobService; @Mock private Statusable job; private String username; @@ -51,7 +51,7 @@ public class JobManagerTest { @Before public void setup() { MockitoAnnotations.initMocks(this); - jm = new JobManager(jobService); + jm = new JobManager(jobService); username = "userabc"; jobId = "job_abc"; } @@ -67,7 +67,7 @@ public void submits_job_and_returns_status() throws JobException { } @Test - public void returns_job_status() { + public void returns_job_status() throws JobException { JobStatus expected = new JobStatus().withState(State.SUCCEEDED).withJobId(jobId); when(job.getStatus()).thenReturn(expected); when(jobService.getJob(username, jobId)).thenReturn(job); @@ -76,7 +76,7 @@ public void returns_job_status() { } @Test - public void returns_job_is_done() { + public void returns_job_is_done() throws JobException { JobStatus expected = new JobStatus().withState(State.SUCCEEDED).withJobId(jobId); when(job.isDone()).thenReturn(true); when(jobService.getJob(username, jobId)).thenReturn(job); @@ -92,7 +92,7 @@ public void kills_job() throws JobException { } @Test - public void returns_statusable_job() { + public void returns_statusable_job() throws JobException { when(job.isDone()).thenReturn(true); when(jobService.getJob(username, jobId)).thenReturn(job); Statusable done = jm.getJob(username, jobId); diff --git a/metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java b/metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java index 1a89c4b370..a0f4df8f32 100644 --- a/metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java +++ b/metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java @@ -40,9 +40,12 @@ public class HdfsJobServiceTest { public TemporaryFolder tempDir = new TemporaryFolder(); @Mock - private Statusable job; + private Statusable job1; + @Mock + private Statusable job2; private String username; - private String jobId; + private String jobId1; + private String jobId2; private String basePath; private HdfsJobService js; @@ -50,8 +53,10 @@ public class HdfsJobServiceTest { public void setup() { MockitoAnnotations.initMocks(this); username = "user123"; - jobId = "job_abc_123"; - when(job.getJobType()).thenReturn(JobType.MAP_REDUCE); + jobId1 = "job_abc_123"; + jobId2 = "job_def_456"; + when(job1.getJobType()).thenReturn(JobType.MAP_REDUCE); + when(job2.getJobType()).thenReturn(JobType.MAP_REDUCE); basePath = tempDir.getRoot().getAbsolutePath(); Map config = new HashMap<>(); config.put("basePath", basePath); @@ -60,33 +65,41 @@ public void setup() { } @Test - public void adds_job() { - js.add(job, username, jobId); - Statusable actual = js.getJob(username, jobId); - assertThat(actual, equalTo(job)); + public void adds_jobs() { + js.add(job1, username, jobId1); + Statusable actual = js.getJob(username, jobId1); + assertThat("Job 1 should exist", actual, equalTo(job1)); + js.add(job2, username, jobId2); + actual = js.getJob(username, jobId1); + assertThat("Job 1 should still exist after adding job 2", actual, equalTo(job1)); + actual = js.getJob(username, jobId2); + assertThat("Job 2 should exist", actual, equalTo(job2)); } @Test public void job_exists_true_for_submitted_jobs() { - js.add(job, username, jobId); - boolean actual = js.jobExists(username, jobId); - assertThat(actual, equalTo(true)); + js.add(job1, username, jobId1); + js.add(job2, username, jobId2); + boolean actual = js.jobExists(username, jobId1); + assertThat("Job 1 should exist", actual, equalTo(true)); + actual = js.jobExists(username, jobId2); + assertThat("Job 2 should exist", actual, equalTo(true)); } @Test public void job_exists_false_for_non_existent_jobs() { - boolean actual = js.jobExists(username, jobId); + boolean actual = js.jobExists(username, jobId1); assertThat(actual, equalTo(false)); - js.add(job, username, jobId); + js.add(job1, username, jobId1); actual = js.jobExists(username, "this_job_id_does_not_exist"); assertThat(actual, equalTo(false)); } @Test public void returns_null_for_non_existent_job() { - Statusable actual = js.getJob(username, jobId); + Statusable actual = js.getJob(username, jobId1); assertThat(actual, equalTo(null)); - js.add(job, username, jobId); + js.add(job1, username, jobId1); actual = js.getJob(username, "this_job_id_does_not_exist"); assertThat(actual, equalTo(null)); } @@ -94,8 +107,9 @@ public void returns_null_for_non_existent_job() { @Test public void writes_job_info_to_hdfs() { // /base/path/jobs/metron_user/MAP_REDUCE/job_abc_123 - js.add(job, username, jobId); - File jobFile = new File(String.format("%s/jobs/%s/%s/%s", basePath, username, JobType.MAP_REDUCE.toString(), jobId)); + js.add(job1, username, jobId1); + File jobFile = new File(String.format("%s/jobs/%s/%s/%s", basePath, username, JobType.MAP_REDUCE.toString(), + jobId1)); assertThat("File should exist", jobFile.exists(), equalTo(true)); assertThat("File should be a file", jobFile.isFile(), equalTo(true)); } diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java index 0fda8011bf..2877235997 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java @@ -30,10 +30,11 @@ import org.apache.metron.common.hadoop.SequenceFileIterable; import org.apache.metron.common.system.Clock; import org.apache.metron.common.utils.timestamp.TimestampConverters; +import org.apache.metron.job.JobException; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.filter.query.QueryPcapFilter; import org.apache.metron.pcap.mr.PcapJob; -import org.apache.metron.pcap.writer.ResultsWriter; +import org.apache.metron.pcap.writer.PcapResultsWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,15 +46,15 @@ public class PcapCli { return String.format("%s-%s", timestamp, uuid); }; private final PcapJob jobRunner; - private final ResultsWriter resultsWriter; + private final PcapResultsWriter resultsWriter; private final CliConfig.PrefixStrategy prefixStrategy; public static void main(String[] args) { - int status = new PcapCli(new PcapJob(), new ResultsWriter(), PREFIX_STRATEGY).run(args); + int status = new PcapCli(new PcapJob(), new PcapResultsWriter(), PREFIX_STRATEGY).run(args); System.exit(status); } - public PcapCli(PcapJob jobRunner, ResultsWriter resultsWriter, CliConfig.PrefixStrategy prefixStrategy) { + public PcapCli(PcapJob jobRunner, PcapResultsWriter resultsWriter, CliConfig.PrefixStrategy prefixStrategy) { this.jobRunner = jobRunner; this.resultsWriter = resultsWriter; this.prefixStrategy = prefixStrategy; @@ -68,6 +69,7 @@ public int run(String[] args) { SequenceFileIterable results = null; String[] commandArgs = Arrays.copyOfRange(args, 1, args.length); Configuration hadoopConf = new Configuration(); + resultsWriter.withConfiguration(hadoopConf); String[] otherArgs = null; try { otherArgs = new GenericOptionsParser(hadoopConf, commandArgs).getRemainingArgs(); @@ -107,7 +109,7 @@ public int run(String[] args) { hadoopConf, FileSystem.get(hadoopConf), new FixedPcapFilter.Configurator()); - } catch (IOException | ClassNotFoundException | InterruptedException e) { + } catch (IOException | ClassNotFoundException | InterruptedException | JobException e) { LOGGER.error("Failed to execute fixed filter job: {}", e.getMessage(), e); return -1; } @@ -141,7 +143,7 @@ public int run(String[] args) { hadoopConf, FileSystem.get(hadoopConf), new QueryPcapFilter.Configurator()); - } catch (IOException | ClassNotFoundException | InterruptedException e) { + } catch (IOException | ClassNotFoundException | InterruptedException | JobException e) { LOGGER.error("Failed to execute query filter job: {}", e.getMessage(), e); return -1; } @@ -153,7 +155,7 @@ public int run(String[] args) { try { // write to local FS in the executing directory String execDir = System.getProperty("user.dir"); - jobRunner.writeResults(results, resultsWriter, new Path("file:///" + execDir), + jobRunner.writeFinalResults(results, resultsWriter, new Path("file:///" + execDir), commonConfig.getNumRecordsPerFile(), commonConfig.getPrefix()); } catch (IOException e) { diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java index ac08f92ca0..c11002ffdc 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java @@ -132,8 +132,8 @@ public void job_succeeds_synchronously() throws Exception { Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); Assert.assertThat(status.getJobId(), equalTo(jobIdVal)); String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); - Assert.assertThat(status.getResultPath(), notNullValue()); - Assert.assertThat(status.getResultPath().toString(), startsWith(expectedOutPath)); + Assert.assertThat(status.getInterimResultPath(), notNullValue()); + Assert.assertThat(status.getInterimResultPath().toString(), startsWith(expectedOutPath)); } @Test @@ -158,8 +158,8 @@ public void job_fails_synchronously() throws Exception { Assert.assertThat(status.getState(), equalTo(State.FAILED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); - Assert.assertThat(status.getResultPath(), notNullValue()); - Assert.assertThat(status.getResultPath().toString(), startsWith(expectedOutPath)); + Assert.assertThat(status.getInterimResultPath(), notNullValue()); + Assert.assertThat(status.getInterimResultPath().toString(), startsWith(expectedOutPath)); } @Test @@ -184,8 +184,8 @@ public void job_fails_with_killed_status_synchronously() throws Exception { Assert.assertThat(status.getState(), equalTo(State.KILLED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); - Assert.assertThat(status.getResultPath(), notNullValue()); - Assert.assertThat(status.getResultPath().toString(), startsWith(expectedOutPath)); + Assert.assertThat(status.getInterimResultPath(), notNullValue()); + Assert.assertThat(status.getInterimResultPath().toString(), startsWith(expectedOutPath)); } @Test @@ -210,8 +210,8 @@ public void job_succeeds_asynchronously() throws Exception { Assert.assertThat(status.getState(), equalTo(State.SUCCEEDED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); - Assert.assertThat(status.getResultPath(), notNullValue()); - Assert.assertThat(status.getResultPath().toString(), startsWith(expectedOutPath)); + Assert.assertThat(status.getInterimResultPath(), notNullValue()); + Assert.assertThat(status.getInterimResultPath().toString(), startsWith(expectedOutPath)); } @Test diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java index 3468a7c658..daf3f457f2 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java @@ -52,7 +52,7 @@ import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.filter.query.QueryPcapFilter; import org.apache.metron.pcap.mr.PcapJob; -import org.apache.metron.pcap.writer.ResultsWriter; +import org.apache.metron.pcap.writer.PcapResultsWriter; import org.junit.Before; import org.junit.Test; import org.mockito.Mock; @@ -64,7 +64,7 @@ public class PcapCliTest { @Mock private PcapJob jobRunner; @Mock - private ResultsWriter resultsWriter; + private PcapResultsWriter resultsWriter; @Mock private Clock clock; private String execDir; @@ -72,7 +72,7 @@ public class PcapCliTest { @Before public void setup() throws IOException { MockitoAnnotations.initMocks(this); - doCallRealMethod().when(jobRunner).writeResults(anyObject(), anyObject(), anyObject(), anyInt(), anyObject()); + doCallRealMethod().when(jobRunner).writeFinalResults(anyObject(), anyObject(), anyObject(), anyInt(), anyObject()); execDir = System.getProperty("user.dir"); } @@ -109,7 +109,7 @@ public void runs_fixed_pcap_filter_job_with_default_argument_list() throws Excep PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); + Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } @Test @@ -149,7 +149,7 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list_and_default_datef PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); + Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } @Test @@ -192,7 +192,7 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list() throws Exceptio PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); + Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } private long asNanos(String inDate, String format) throws ParseException { @@ -225,7 +225,7 @@ public void runs_query_pcap_filter_job_with_default_argument_list() throws Excep PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); + Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } @Test @@ -253,7 +253,7 @@ public void runs_query_pcap_filter_job_with_full_argument_list() throws Exceptio PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(isA(Configuration.class), eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); + Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } // INVALID OPTION CHECKS diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index bc09f9cefb..d5a53d2376 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -56,6 +56,7 @@ import org.apache.metron.job.JobStatus.State; import org.apache.metron.job.Pageable; import org.apache.metron.job.Statusable; +import org.apache.metron.job.writer.ResultsWriter; import org.apache.metron.pcap.PacketInfo; import org.apache.metron.pcap.PcapFiles; import org.apache.metron.pcap.PcapHelper; @@ -63,18 +64,21 @@ import org.apache.metron.pcap.filter.PcapFilterConfigurator; import org.apache.metron.pcap.filter.PcapFilters; import org.apache.metron.pcap.utils.FileFilterUtil; -import org.apache.metron.pcap.writer.ResultsWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class PcapJob implements Statusable { +/** + * Encompasses MapReduce job and final writing of Pageable results to specified location. + * Cleans up MapReduce results from HDFS on completion. + */ +public class PcapJob implements Statusable { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); public static final String START_TS_CONF = "start_ts"; public static final String END_TS_CONF = "end_ts"; public static final String WIDTH_CONF = "width"; private Job job; // store a running MR job reference for async status check - private Path outputPath; + private JobStatus jobStatus; private PcapMRJobConfig config; public static enum PCAP_COUNTER { @@ -188,13 +192,14 @@ public SequenceFileIterable query(Path basePath , Configuration conf , FileSystem fs , PcapFilterConfigurator filterImpl - ) throws IOException, ClassNotFoundException, InterruptedException { + ) + throws IOException, ClassNotFoundException, InterruptedException, JobException { Statusable statusable = query(Optional.empty(), basePath, baseOutputPath, beginNS, endNS, numReducers, fields, conf, fs, filterImpl, true); JobStatus jobStatus = statusable.getStatus(); if (jobStatus.getState() == State.SUCCEEDED) { - Path resultPath = jobStatus.getResultPath(); - return readResults(resultPath, conf, fs); + Path resultPath = jobStatus.getInterimResultPath(); + return readInterimResults(resultPath, conf, fs); } else { throw new RuntimeException( "Unable to complete query due to errors. Please check logs for full errors."); @@ -226,10 +231,10 @@ public Statusable query(Optional jobName, String to = format.format(new Date(Long.divideUnsigned(endNS, 1000000))); LOG.debug("Executing query {} on timerange from {} to {}", filterImpl.queryToString(fields), from, to); } - outputPath = new Path(baseOutputPath, outputDirName); + Path interimOutputPath = new Path(baseOutputPath, outputDirName); job = createJob(jobName , basePath - , outputPath + , interimOutputPath , beginNS , endNS , numReducers @@ -238,6 +243,7 @@ public Statusable query(Optional jobName, , fs , filterImpl ); + jobStatus = new JobStatus().withInterimResultPath(interimOutputPath); if (sync) { job.waitForCompletion(true); } else { @@ -249,7 +255,7 @@ public Statusable query(Optional jobName, /** * Returns a lazily-read Iterable over a set of sequence files */ - private SequenceFileIterable readResults(Path outputPath, Configuration config, FileSystem fs) throws IOException { + public SequenceFileIterable readInterimResults(Path outputPath, Configuration config, FileSystem fs) throws IOException { List files = new ArrayList<>(); for (RemoteIterator it = fs.listFiles(outputPath, false); it.hasNext(); ) { Path p = it.next().getPath(); @@ -288,33 +294,6 @@ public Statusable submit() throws JobException { } } - public Pageable writeResults(SequenceFileIterable results, ResultsWriter resultsWriter, - Path outPath, int recPerFile, String prefix) throws IOException { - List outFiles = new ArrayList<>(); - try { - Iterable> partitions = Iterables.partition(results, recPerFile); - int part = 1; - if (partitions.iterator().hasNext()) { - for (List data : partitions) { - String outFileName = String.format("%s/pcap-data-%s+%04d.pcap", outPath, prefix, part++); - if (data.size() > 0) { - resultsWriter.write(new Configuration(), data, outFileName); - outFiles.add(new Path(outFileName)); - } - } - } else { - LOG.info("No results returned."); - } - } finally { - try { - results.cleanup(); - } catch (IOException e) { - LOG.warn("Unable to cleanup files in HDFS", e); - } - } - return new PcapFiles(outFiles); - } - /** * Creates, but does not submit the job. */ @@ -376,48 +355,94 @@ public JobType getJobType() { } @Override - public JobStatus getStatus() { + public JobStatus getStatus() throws JobException { // Note: this method is only reading state from the underlying job, so locking not needed - JobStatus status = new JobStatus().withResultPath(outputPath); if (job == null) { - status.withPercentComplete(100).withState(State.SUCCEEDED); + jobStatus.withPercentComplete(100).withState(State.SUCCEEDED); } else { try { - status.withJobId(job.getStatus().getJobID().toString()); + jobStatus.withJobId(job.getStatus().getJobID().toString()); if (job.isComplete()) { - status.withPercentComplete(100); + jobStatus.withPercentComplete(100); switch (job.getStatus().getState()) { case SUCCEEDED: - status.withState(State.SUCCEEDED).withDescription(State.SUCCEEDED.toString()); + jobStatus.withState(State.SUCCEEDED).withDescription(State.SUCCEEDED.toString()); break; case FAILED: - status.withState(State.FAILED); + jobStatus.withState(State.FAILED).withDescription(State.FAILED.toString()); break; case KILLED: - status.withState(State.KILLED); + jobStatus.withState(State.KILLED).withDescription(State.KILLED.toString()); break; + default: + throw new IllegalStateException( + "Unknown job state reported as 'complete' by mapreduce framework: " + job + .getStatus().getState()); + } } else { float mapProg = job.mapProgress(); float reduceProg = job.reduceProgress(); float totalProgress = ((mapProg / 2) + (reduceProg / 2)) * 100; String description = String.format("map: %s%%, reduce: %s%%", mapProg * 100, reduceProg * 100); - status.withPercentComplete(totalProgress).withState(State.RUNNING) + jobStatus.withPercentComplete(totalProgress).withState(State.RUNNING) .withDescription(description); } } catch (Exception e) { throw new RuntimeException("Error occurred while attempting to retrieve job status.", e); } } - return status; + return jobStatus; } + @SuppressWarnings("unchecked") @Override - public boolean isDone() { - // Note: this method is only reading state from the underlying job, so locking not needed + public Pageable finalizeJob() throws JobException { + try { + SequenceFileIterable interimResults = readInterimResults(jobStatus.getInterimResultPath(), + config.getConf(), config.getFs()); + return writeFinalResults(interimResults, config.getResultsWriter(), + config.getFinalOutputPath(), + config.getNumRecordsPerFile(), + config.getOutputFilePrefix()); + } catch (IOException e) { + throw new JobException("Unable to read intermediate pcap MapReduce results.", e); + } + } + + public Pageable writeFinalResults(SequenceFileIterable results, ResultsWriter resultsWriter, + Path outPath, int recPerFile, String prefix) throws IOException { + List outFiles = new ArrayList<>(); + try { + Iterable> partitions = Iterables.partition(results, recPerFile); + int part = 1; + if (partitions.iterator().hasNext()) { + for (List data : partitions) { + String outFileName = String.format("%s/pcap-data-%s+%04d.pcap", outPath, prefix, part++); + if (data.size() > 0) { + resultsWriter.write(data, outFileName); + outFiles.add(new Path(outFileName)); + } + } + } else { + LOG.info("No results returned."); + } + } finally { + try { + results.cleanup(); + } catch (IOException e) { + LOG.warn("Unable to cleanup files in HDFS", e); + } + } + return new PcapFiles(outFiles); + } + + @Override + public boolean isDone() throws JobException { + getStatus(); try { return job.isComplete(); - } catch (Exception e) { + } catch (IOException e) { throw new RuntimeException("Error occurred while attempting to retrieve job status.", e); } } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapMRJobConfig.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapMRJobConfig.java index 8057096d51..dce7fe78b3 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapMRJobConfig.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapMRJobConfig.java @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.metron.job.writer.ResultsWriter; import org.apache.metron.pcap.filter.PcapFilterConfigurator; public class PcapMRJobConfig { @@ -29,6 +30,10 @@ public class PcapMRJobConfig { private Optional jobName; private Path basePath; private Path baseOutputPath; + private Path finalOutputPath; + private ResultsWriter resultsWriter; + private int numRecordsPerFile; + private String outputFilePrefix; private long beginNS; private long endNS; private int numReducers; @@ -42,7 +47,7 @@ public Optional getJobName() { return jobName; } - public PcapMRJobConfig setJobName(Optional jobName) { + public PcapMRJobConfig setJobName(Optional jobName) { this.jobName = jobName; return this; } @@ -51,7 +56,7 @@ public Path getBasePath() { return basePath; } - public PcapMRJobConfig setBasePath(Path basePath) { + public PcapMRJobConfig setBasePath(Path basePath) { this.basePath = basePath; return this; } @@ -60,7 +65,7 @@ public Path getBaseOutputPath() { return baseOutputPath; } - public PcapMRJobConfig setBaseOutputPath(Path baseOutputPath) { + public PcapMRJobConfig setBaseOutputPath(Path baseOutputPath) { this.baseOutputPath = baseOutputPath; return this; } @@ -69,7 +74,7 @@ public long getBeginNS() { return beginNS; } - public PcapMRJobConfig setBeginNS(long beginNS) { + public PcapMRJobConfig setBeginNS(long beginNS) { this.beginNS = beginNS; return this; } @@ -78,7 +83,7 @@ public long getEndNS() { return endNS; } - public PcapMRJobConfig setEndNS(long endNS) { + public PcapMRJobConfig setEndNS(long endNS) { this.endNS = endNS; return this; } @@ -87,7 +92,7 @@ public int getNumReducers() { return numReducers; } - public PcapMRJobConfig setNumReducers(int numReducers) { + public PcapMRJobConfig setNumReducers(int numReducers) { this.numReducers = numReducers; return this; } @@ -96,7 +101,7 @@ public T getFields() { return fields; } - public PcapMRJobConfig setFields(T fields) { + public PcapMRJobConfig setFields(T fields) { this.fields = fields; return this; } @@ -105,7 +110,7 @@ public Configuration getConf() { return conf; } - public PcapMRJobConfig setConf(Configuration conf) { + public PcapMRJobConfig setConf(Configuration conf) { this.conf = conf; return this; } @@ -114,7 +119,7 @@ public FileSystem getFs() { return fs; } - public PcapMRJobConfig setFs(FileSystem fs) { + public PcapMRJobConfig setFs(FileSystem fs) { this.fs = fs; return this; } @@ -123,7 +128,7 @@ public PcapFilterConfigurator getFilterImpl() { return filterImpl; } - public PcapMRJobConfig setFilterImpl(PcapFilterConfigurator filterImpl) { + public PcapMRJobConfig setFilterImpl(PcapFilterConfigurator filterImpl) { this.filterImpl = filterImpl; return this; } @@ -132,8 +137,45 @@ public boolean isSynchronous() { return synchronous; } - public PcapMRJobConfig setSynchronous(boolean synchronous) { + public PcapMRJobConfig setSynchronous(boolean synchronous) { this.synchronous = synchronous; return this; } + + public Path getFinalOutputPath() { + return finalOutputPath; + } + + public PcapMRJobConfig setFinalOutputPath(Path finalOutputPath) { + this.finalOutputPath = finalOutputPath; + return this; + } + + public ResultsWriter getResultsWriter() { + return resultsWriter; + } + + public PcapMRJobConfig setResultsWriter(ResultsWriter resultsWriter) { + this.resultsWriter = resultsWriter; + return this; + } + + public int getNumRecordsPerFile() { + return numRecordsPerFile; + } + + public PcapMRJobConfig setNumRecordsPerFile(int numRecordsPerFile) { + this.numRecordsPerFile = numRecordsPerFile; + return this; + } + + public String getOutputFilePrefix() { + return outputFilePrefix; + } + + public PcapMRJobConfig setOutputFilePrefix(String outputFilePrefix) { + this.outputFilePrefix = outputFilePrefix; + return this; + } + } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/writer/ResultsWriter.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/writer/PcapResultsWriter.java similarity index 75% rename from metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/writer/ResultsWriter.java rename to metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/writer/PcapResultsWriter.java index 3934aca885..3ca834f4c1 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/writer/ResultsWriter.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/writer/PcapResultsWriter.java @@ -22,19 +22,42 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.metron.common.utils.HDFSUtils; +import org.apache.metron.job.writer.ResultsWriter; import org.apache.metron.pcap.PcapMerger; -public class ResultsWriter { +public class PcapResultsWriter implements ResultsWriter { + + private Configuration config; /** - * Write out pcaps. Configuration offers ability to configure for HDFS or local FS, if desired. + * Creates default hadoop configuration. + */ + public PcapResultsWriter() { + this.config = new Configuration(); + } + + /** + * Pass in hadoop config * * @param config Standard hadoop filesystem config. + */ + public PcapResultsWriter(Configuration config) { + this.config = config; + } + + public void withConfiguration(Configuration config) { + this.config = config; + } + + /** + * Write out pcaps. Configuration offers ability to configure for HDFS or local FS, if desired. + * * @param pcaps pcap data to write. Pre-merged format as a list of pcaps as byte arrays. * @param outPath where to write the pcap data to. * @throws IOException I/O issue encountered. */ - public void write(Configuration config, List pcaps, String outPath) throws IOException { + @Override + public void write(List pcaps, String outPath) throws IOException { HDFSUtils.write(config, mergePcaps(pcaps), outPath); } @@ -45,7 +68,7 @@ public void write(Configuration config, List pcaps, String outPath) thro * @return merged result. * @throws IOException I/O issue encountered. */ - public byte[] mergePcaps(List pcaps) throws IOException { + protected byte[] mergePcaps(List pcaps) throws IOException { if (pcaps == null) { return new byte[]{}; } @@ -56,4 +79,5 @@ public byte[] mergePcaps(List pcaps) throws IOException { PcapMerger.merge(baos, pcaps); return baos.toByteArray(); } + } From 9fb3a81c144cab23f81de94007e85e33552fe10c Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Tue, 10 Jul 2018 10:57:05 -0600 Subject: [PATCH 11/36] Prototyping rest api bits --- .../rest/controller/PcapController.java | 32 +++++-- .../metron/rest/service/PcapService.java | 7 +- .../rest/service/impl/PcapServiceImpl.java | 92 ++++++++++++++----- .../service/impl/PcapServiceImplTest.java | 32 ++----- .../apache/metron/common/utils/HDFSUtils.java | 7 ++ .../org/apache/metron/job/Statusable.java | 7 ++ .../org/apache/metron/pcap/mr/PcapJob.java | 31 +++++-- 7 files changed, 146 insertions(+), 62 deletions(-) diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/PcapController.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/PcapController.java index 11ff1cd759..3524a8cd67 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/PcapController.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/PcapController.java @@ -21,20 +21,20 @@ import io.swagger.annotations.ApiParam; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; +import org.apache.metron.job.JobStatus; +import org.apache.metron.rest.RestException; import org.apache.metron.rest.model.PcapResponse; import org.apache.metron.rest.model.pcap.FixedPcapRequest; -import org.apache.metron.rest.RestException; import org.apache.metron.rest.service.PcapService; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.http.HttpStatus; import org.springframework.http.ResponseEntity; +import org.springframework.web.bind.annotation.PathVariable; import org.springframework.web.bind.annotation.RequestBody; import org.springframework.web.bind.annotation.RequestMapping; import org.springframework.web.bind.annotation.RequestMethod; import org.springframework.web.bind.annotation.RestController; -import java.io.IOException; - @RestController @RequestMapping("/api/v1/pcap") public class PcapController { @@ -43,11 +43,29 @@ public class PcapController { private PcapService pcapQueryService; @ApiOperation(value = "Executes a Fixed Pcap Query.") - @ApiResponses(value = { @ApiResponse(message = "Returns a PcapResponse containing an array of pcaps.", code = 200)}) + @ApiResponses(value = { @ApiResponse(message = "Returns a job status with job ID.", code = 200)}) @RequestMapping(value = "/fixed", method = RequestMethod.POST) - ResponseEntity fixed(@ApiParam(name="fixedPcapRequest", value="A Fixed Pcap Request" + ResponseEntity fixed(@ApiParam(name="fixedPcapRequest", value="A Fixed Pcap Request" + " which includes fixed filter fields like ip source address and protocol.", required=true)@RequestBody FixedPcapRequest fixedPcapRequest) throws RestException { - PcapResponse pcapsResponse = pcapQueryService.fixed(fixedPcapRequest); - return new ResponseEntity<>(pcapsResponse, HttpStatus.OK); + JobStatus jobStatus = pcapQueryService.fixed(fixedPcapRequest); + return new ResponseEntity<>(jobStatus, HttpStatus.OK); + } + + @ApiOperation(value = "Gets job status for running job.") + @ApiResponses(value = { @ApiResponse(message = "Returns a job status for the passed job.", code = 200)}) + @RequestMapping(value = "/getStatus", method = RequestMethod.GET) + ResponseEntity getStatus(@ApiParam(name="jobId", value="Job ID of submitted job" + + " which includes fixed filter fields like ip source address and protocol.", required=true)@PathVariable String jobId) throws RestException { + JobStatus jobStatus = pcapQueryService.getJobStatus("metron", jobId); + return new ResponseEntity<>(jobStatus, HttpStatus.OK); + } + + @ApiOperation(value = "Gets results of a pcap job.") + @ApiResponses(value = { @ApiResponse(message = "Returns a PcapResponse containing an array of pcaps.", code = 200)}) + @RequestMapping(value = "/getPage", method = RequestMethod.GET) + ResponseEntity getPage(@ApiParam(name="fixedPcapRequest", value="Job ID of submitted job" + + " which includes fixed filter fields like ip source address and protocol.", required=true)@RequestBody String jobId, int pageNum) throws RestException { + PcapResponse pcapResponse = pcapQueryService.getPage("metron", jobId, pageNum); + return new ResponseEntity<>(pcapResponse, HttpStatus.OK); } } diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/PcapService.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/PcapService.java index ce8372c5b7..a6fdf161b5 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/PcapService.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/PcapService.java @@ -17,11 +17,16 @@ */ package org.apache.metron.rest.service; +import org.apache.metron.job.JobStatus; import org.apache.metron.rest.RestException; import org.apache.metron.rest.model.PcapResponse; import org.apache.metron.rest.model.pcap.FixedPcapRequest; public interface PcapService { - PcapResponse fixed(FixedPcapRequest fixedPcapRequest) throws RestException; + JobStatus fixed(FixedPcapRequest fixedPcapRequest) throws RestException; + + JobStatus getJobStatus(String username, String jobId) throws RestException; + + PcapResponse getPage(String username, String jobId, int pageNum) throws RestException; } diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java index dd4af5cb03..4acf36061d 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java @@ -22,15 +22,26 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.metron.common.Constants; -import org.apache.metron.common.hadoop.SequenceFileIterable; +import org.apache.metron.common.system.Clock; +import org.apache.metron.common.utils.HDFSUtils; import org.apache.metron.common.utils.timestamp.TimestampConverters; +import org.apache.metron.job.JobException; +import org.apache.metron.job.JobStatus; +import org.apache.metron.job.Pageable; +import org.apache.metron.job.Statusable; +import org.apache.metron.job.manager.JobManager; +import org.apache.metron.job.service.JobServiceStrategies; +import org.apache.metron.job.writer.PrefixStrategy; import org.apache.metron.pcap.PcapHelper; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.mr.PcapJob; +import org.apache.metron.pcap.mr.PcapMRJobConfig; +import org.apache.metron.pcap.writer.PcapResultsWriter; import org.apache.metron.rest.MetronRestConstants; import org.apache.metron.rest.RestException; import org.apache.metron.rest.model.PcapResponse; @@ -45,44 +56,79 @@ public class PcapServiceImpl implements PcapService { private Environment environment; private Configuration configuration; - private PcapJob pcapJob; + private JobManager jobManager; @Autowired - public PcapServiceImpl(Environment environment, Configuration configuration, PcapJob pcapJob) { + public PcapServiceImpl(Environment environment, Configuration configuration) { this.environment = environment; this.configuration = configuration; - this.pcapJob = pcapJob; + this.jobManager = new JobManager<>(JobServiceStrategies.HDFS); } @Override - public PcapResponse fixed(FixedPcapRequest fixedPcapRequest) throws RestException { + public JobStatus fixed(FixedPcapRequest fixedPcapRequest) throws RestException { if (fixedPcapRequest.getBasePath() == null) { fixedPcapRequest.setBasePath(environment.getProperty(MetronRestConstants.PCAP_INPUT_PATH_SPRING_PROPERTY)); } if (fixedPcapRequest.getBaseOutputPath() == null) { fixedPcapRequest.setBaseOutputPath(environment.getProperty(MetronRestConstants.PCAP_OUTPUT_PATH_SPRING_PROPERTY)); } + + PrefixStrategy prefixStrategy = clock -> { + String timestamp = new Clock().currentTimeFormatted("yyyyMMddHHmm"); + String uuid = UUID.randomUUID().toString().replaceAll("-", ""); + return String.format("%s-%s", timestamp, uuid); + }; + + PcapMRJobConfig> config = new PcapMRJobConfig(); + try { + config.setBasePath(new Path(fixedPcapRequest.getBasePath())) + .setBaseOutputPath(new Path(fixedPcapRequest.getBaseOutputPath())) + .setBeginNS(TimestampConverters.MILLISECONDS.toNanoseconds(fixedPcapRequest.getStartTime())) + .setEndNS(TimestampConverters.MILLISECONDS.toNanoseconds(fixedPcapRequest.getEndTime())) + .setNumReducers(fixedPcapRequest.getNumReducers()) + .setFields(getFixedFields(fixedPcapRequest)) + .setConf(configuration) + .setFs(getFileSystem()) + .setFilterImpl(new FixedPcapFilter.Configurator()) + .setResultsWriter(new PcapResultsWriter()) + .setFinalOutputPath(new Path("/apps/metron/pcap/final")) + .setNumRecordsPerFile(10) + .setOutputFilePrefix(prefixStrategy.apply(new Clock())); + + Statusable pcapJob = new PcapJob(config); + return jobManager.submit(pcapJob, new HashMap<>(), "metron"); + } catch (IOException | JobException e) { + throw new RestException(e); + } + } + + @Override + public JobStatus getJobStatus(String username, String jobId) throws RestException { + try { + return jobManager.getStatus(username, jobId); + } catch (JobException e) { + throw new RestException("Unable to get job status", e); + } + } + + @Override + public PcapResponse getPage(String username, String jobId, int pageNum) throws RestException { PcapResponse response = new PcapResponse(); - SequenceFileIterable results; + Statusable job = jobManager.getJob(username, jobId); try { - results = pcapJob.query( - new Path(fixedPcapRequest.getBasePath()), - new Path(fixedPcapRequest.getBaseOutputPath()), - TimestampConverters.MILLISECONDS.toNanoseconds(fixedPcapRequest.getStartTime()), - TimestampConverters.MILLISECONDS.toNanoseconds(fixedPcapRequest.getEndTime()), - fixedPcapRequest.getNumReducers(), - getFixedFields(fixedPcapRequest), - configuration, - getFileSystem(), - new FixedPcapFilter.Configurator() - ); - if (results != null) { - List pcaps = new ArrayList<>(); - results.iterator().forEachRemaining(pcaps::add); - response.setPcaps(pcaps); + if (job.isDone()) { + Pageable results = job.getFinalResults(); + Path resultsPage = results.getPage(pageNum); + if (resultsPage != null) { + List pcaps = new ArrayList<>(); + byte[] pcap = HDFSUtils.readBytes(resultsPage.toString()); + pcaps.add(pcap); + response.setPcaps(pcaps); + } } - } catch (IOException | ClassNotFoundException | InterruptedException e) { - throw new RestException(e); + } catch (JobException | IOException e) { + throw new RestException("Unable to get results", e); } return response; } diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java index b6f2ea8cb3..1a11c797be 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java @@ -17,18 +17,12 @@ */ package org.apache.metron.rest.service.impl; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.metron.common.Constants; -import org.apache.metron.common.hadoop.SequenceFileIterable; -import org.apache.metron.pcap.PcapHelper; -import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.mr.PcapJob; import org.apache.metron.rest.MetronRestConstants; -import org.apache.metron.rest.RestException; -import org.apache.metron.rest.model.PcapResponse; -import org.apache.metron.rest.model.pcap.FixedPcapRequest; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -36,19 +30,6 @@ import org.junit.rules.ExpectedException; import org.springframework.core.env.Environment; -import java.io.IOException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.when; - @SuppressWarnings("ALL") public class PcapServiceImplTest { @Rule @@ -68,7 +49,13 @@ public void setUp() throws Exception { when(environment.getProperty(MetronRestConstants.PCAP_OUTPUT_PATH_SPRING_PROPERTY)).thenReturn("/output/path"); } + // TODO + @Test + public void placeholder() { + Assert.assertTrue(true); + } +/* @Test public void fixedShouldProperlyCallPcapJobQuery() throws Exception { FixedPcapRequest fixedPcapRequest = new FixedPcapRequest(); @@ -164,4 +151,5 @@ public void fixedShouldThrowRestException() throws Exception { pcapService.fixed(fixedPcapRequest); } + */ } diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java index ae09edf881..642a5bcee6 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java @@ -29,6 +29,13 @@ public class HDFSUtils { + public static byte[] readBytes(String path) throws IOException { + Path inPath = new Path(path); + FileSystem fs = FileSystem.newInstance(inPath.toUri(), new Configuration()); + FSDataInputStream inputStream = fs.open(inPath); + return IOUtils.toByteArray(inputStream); + } + /** * Reads full file contents into a List of Strings. Reads from local FS if file:/// used as the * scheme. Initializes file system with default configuration. diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java index 02b1ab6f1c..35d9580b84 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java @@ -78,4 +78,11 @@ enum JobType { */ Pageable finalizeJob() throws JobException; + /** + * Gets final results of this job as a Pageable. + * + * @return pageable results + */ + Pageable getFinalResults() throws JobException; + } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index d5a53d2376..e656af1cfa 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -80,6 +80,8 @@ public class PcapJob implements Statusable { private Job job; // store a running MR job reference for async status check private JobStatus jobStatus; private PcapMRJobConfig config; + private boolean finalized; // job results have been finalized + private Pageable jobResults; public static enum PCAP_COUNTER { MALFORMED_PACKET_COUNT @@ -398,18 +400,29 @@ public JobStatus getStatus() throws JobException { @SuppressWarnings("unchecked") @Override public Pageable finalizeJob() throws JobException { - try { - SequenceFileIterable interimResults = readInterimResults(jobStatus.getInterimResultPath(), - config.getConf(), config.getFs()); - return writeFinalResults(interimResults, config.getResultsWriter(), - config.getFinalOutputPath(), - config.getNumRecordsPerFile(), - config.getOutputFilePrefix()); - } catch (IOException e) { - throw new JobException("Unable to read intermediate pcap MapReduce results.", e); + if (this.isDone()) { + try { + SequenceFileIterable interimResults = readInterimResults(jobStatus.getInterimResultPath(), + config.getConf(), config.getFs()); + Pageable jobResults = writeFinalResults(interimResults, config.getResultsWriter(), + config.getFinalOutputPath(), + config.getNumRecordsPerFile(), + config.getOutputFilePrefix()); + finalized = true; + return jobResults; + } catch (IOException e) { + throw new JobException("Unable to read intermediate pcap MapReduce results.", e); + } + } else { + return null; } } + @Override + public Pageable getFinalResults() throws JobException { + return jobResults; + } + public Pageable writeFinalResults(SequenceFileIterable results, ResultsWriter resultsWriter, Path outPath, int recPerFile, String prefix) throws IOException { List outFiles = new ArrayList<>(); From 36456d637d9cd13a13ef34735f74d06d350772cf Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Thu, 12 Jul 2018 05:52:42 -0600 Subject: [PATCH 12/36] Redo the pcap asychronicity to use a Timer for checking status. Pass in config as Map --- .../apache/metron/common/utils/HDFSUtils.java | 6 +- .../JobService.java => Finalizer.java} | 26 +- .../java/org/apache/metron/job/JobStatus.java | 1 + .../java/org/apache/metron/job/Pageable.java | 7 + .../org/apache/metron/job/Statusable.java | 27 +- .../job/manager/InMemoryJobManager.java | 68 ++++ .../apache/metron/job/manager/JobManager.java | 37 +-- .../metron/job/service/HdfsJobService.java | 102 ------ ...rTest.java => InMemoryJobManagerTest.java} | 92 ++++-- .../job/service/HdfsJobServiceTest.java | 11 +- .../org/apache/metron/pcap/query/PcapCli.java | 85 ++--- .../org/apache/metron/pcap/PcapJobTest.java | 148 ++++----- .../PcapTopologyIntegrationTest.java | 13 +- .../apache/metron/pcap/query/PcapCliTest.java | 48 +-- .../org/apache/metron/pcap/PcapFiles.java | 5 + .../pcap/finalizer/PcapCliFinalizer.java} | 18 +- .../metron/pcap/finalizer/PcapFinalizer.java | 149 +++++++++ .../finalizer/PcapFinalizerStrategies.java} | 35 +-- .../pcap/finalizer/PcapRestFinalizer.java} | 24 +- .../org/apache/metron/pcap/mr/PcapJob.java | 293 ++++++++---------- .../metron/pcap/mr/PcapMRJobConfig.java | 181 ----------- .../metron/pcap/writer/PcapResultsWriter.java | 32 +- 22 files changed, 620 insertions(+), 788 deletions(-) rename metron-platform/metron-job/src/main/java/org/apache/metron/job/{service/JobService.java => Finalizer.java} (68%) create mode 100644 metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java delete mode 100644 metron-platform/metron-job/src/main/java/org/apache/metron/job/service/HdfsJobService.java rename metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/{JobManagerTest.java => InMemoryJobManagerTest.java} (52%) rename metron-platform/{metron-job/src/main/java/org/apache/metron/job/writer/PrefixStrategy.java => metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java} (60%) create mode 100644 metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java rename metron-platform/{metron-job/src/main/java/org/apache/metron/job/service/JobServiceStrategies.java => metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizerStrategies.java} (53%) rename metron-platform/{metron-job/src/main/java/org/apache/metron/job/writer/ResultsWriter.java => metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java} (66%) delete mode 100644 metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapMRJobConfig.java diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java index 642a5bcee6..5bdd8d6c56 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java @@ -31,7 +31,7 @@ public class HDFSUtils { public static byte[] readBytes(String path) throws IOException { Path inPath = new Path(path); - FileSystem fs = FileSystem.newInstance(inPath.toUri(), new Configuration()); + FileSystem fs = FileSystem.get(inPath.toUri(), new Configuration()); FSDataInputStream inputStream = fs.open(inPath); return IOUtils.toByteArray(inputStream); } @@ -64,7 +64,7 @@ public static List readFile(String path) throws IOException { */ public static List readFile(Configuration config, String path) throws IOException { Path inPath = new Path(path); - FileSystem fs = FileSystem.newInstance(inPath.toUri(), config); + FileSystem fs = FileSystem.get(inPath.toUri(), config); FSDataInputStream inputStream = fs.open(inPath); return IOUtils.readLines(inputStream, "UTF-8"); } @@ -81,7 +81,7 @@ public static List readFile(Configuration config, String path) throws IO */ public static void write(Configuration config, byte[] bytes, String path) throws IOException { Path outPath = new Path(path); - FileSystem fs = FileSystem.newInstance(outPath.toUri(), config); + FileSystem fs = FileSystem.get(outPath.toUri(), config); fs.mkdirs(outPath.getParent()); try (FSDataOutputStream outputStream = fs.create(outPath)) { outputStream.write(bytes); diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobService.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Finalizer.java similarity index 68% rename from metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobService.java rename to metron-platform/metron-job/src/main/java/org/apache/metron/job/Finalizer.java index 0a1013d34c..5001cfad27 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobService.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Finalizer.java @@ -16,19 +16,23 @@ * limitations under the License. */ -package org.apache.metron.job.service; +package org.apache.metron.job; import java.util.Map; -import org.apache.metron.job.Statusable; -public interface JobService { - - void configure(Map config); - - void add(Statusable job, String username, String jobId); - - boolean jobExists(String username, String jobId); - - Statusable getJob(String username, String jobId); +/** + * Finalize a job. + * + * @param Type for the Pageable. + */ +public interface Finalizer { + + /** + * Run any routines for finalizing a job. + * + * @param config options to be used by the finalization process. + * @return Pageable results. + */ + Pageable finalizeJob(Map config) throws JobException; } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java index 4d61df3d57..b67b904a08 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java @@ -29,6 +29,7 @@ public enum State { NOT_RUNNING, RUNNING, SUCCEEDED, + FINALIZING, FAILED, KILLED } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Pageable.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Pageable.java index 1038ab8e3c..9b92c8c0b0 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Pageable.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Pageable.java @@ -35,4 +35,11 @@ public interface Pageable { */ T getPage(int num); + /** + * Number of pages i this Pageable. + * + * @return number of pages + */ + int getSize(); + } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java index 35d9580b84..694577089a 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java @@ -23,7 +23,7 @@ /** * Abstraction for getting status on running jobs. Also provides options for killing and validating. */ -public interface Statusable { +public interface Statusable { enum JobType { MAP_REDUCE, @@ -31,11 +31,18 @@ enum JobType { } /** - * Submit the job. + * Submit the job asynchronously. * * @return self */ - Statusable submit() throws JobException; + Statusable submit(Finalizer finalizer, Map configuration) throws JobException; + + /** + * Synchronous call. + * + * @return pages of results + */ + Pageable get() throws JobException, InterruptedException; /** * Execution framework type of this job. @@ -71,18 +78,4 @@ enum JobType { */ boolean validate(Map configuration); - /** - * Finalize job results. Any post-processing is done here. - * - * @return Results in a pageable fashion. - */ - Pageable finalizeJob() throws JobException; - - /** - * Gets final results of this job as a Pageable. - * - * @return pageable results - */ - Pageable getFinalResults() throws JobException; - } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java new file mode 100644 index 0000000000..593ee82829 --- /dev/null +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java @@ -0,0 +1,68 @@ +/** + * 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.metron.job.manager; + +import java.lang.invoke.MethodHandles; +import java.util.HashMap; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.metron.job.JobException; +import org.apache.metron.job.JobStatus; +import org.apache.metron.job.Statusable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class InMemoryJobManager implements JobManager { + private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + private Map>> jobs; + + public InMemoryJobManager() { + this.jobs = new HashMap<>(); + } + + @Override + public JobStatus submit(Supplier> jobSupplier, String username) throws JobException { + Statusable job = jobSupplier.get(); + Map> userJobs = new HashMap<>(); + userJobs.put(job.getStatus().getJobId(), job); + jobs.put(username, userJobs); + return job.getStatus(); + } + + @Override + public JobStatus getStatus(String username, String jobId) throws JobException { + return jobs.get(username).get(jobId).getStatus(); + } + + @Override + public boolean done(String username, String jobId) throws JobException { + return false; + } + + @Override + public void killJob(String username, String jobId) throws JobException { + + } + + @Override + public Statusable getJob(String username, String jobId) throws JobException { + return null; + } + +} diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java index 57e9d9f494..55f4d07401 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java @@ -18,44 +18,21 @@ package org.apache.metron.job.manager; -import java.util.Map; +import java.util.function.Supplier; import org.apache.metron.job.JobException; import org.apache.metron.job.JobStatus; import org.apache.metron.job.Statusable; -import org.apache.metron.job.service.JobService; -public class JobManager { +public interface JobManager { - private JobService jobs; + JobStatus submit(Supplier> jobSupplier, String username) throws JobException; - public JobManager(JobService jobs) { - this.jobs = jobs; - } + JobStatus getStatus(String username, String jobId) throws JobException; - public JobStatus submit(Statusable job, Map configuration, String username) - throws JobException { - JobStatus status = new JobStatus(); - if (job.validate(configuration)) { - status = job.submit().getStatus(); - } - jobs.add(job, status.getJobId(), username); - return status; - } + boolean done(String username, String jobId) throws JobException; - public JobStatus getStatus(String username, String jobId) throws JobException { - return jobs.getJob(username, jobId).getStatus(); - } + void killJob(String username, String jobId) throws JobException; - public boolean done(String username, String jobId) throws JobException { - return jobs.getJob(username, jobId).isDone(); - } - - public void killJob(String username, String jobId) throws JobException { - jobs.getJob(username, jobId).kill(); - } - - public Statusable getJob(String username, String jobId) { - return jobs.getJob(username, jobId); - } + Statusable getJob(String username, String jobId) throws JobException; } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/HdfsJobService.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/HdfsJobService.java deleted file mode 100644 index ca29b16246..0000000000 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/HdfsJobService.java +++ /dev/null @@ -1,102 +0,0 @@ -/** - * 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.metron.job.service; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.HashMap; -import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.metron.common.utils.HDFSUtils; -import org.apache.metron.job.Statusable; -import org.apache.metron.job.Statusable.JobType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * HDFS-backed implementation of a job service - */ -public class HdfsJobService implements JobService { - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private Map>> jobs; - private String basePath; - private String jobsPath; - - public HdfsJobService() { - jobs = new HashMap<>(); - basePath = "/apps/metron"; - jobsPath = basePath + "/jobs"; - } - - @Override - public void configure(Map config) { - if (config != null) { - if (config.containsKey("basePath")) { - basePath = (String) config.get("basePath"); - jobsPath = basePath + "/jobs"; - } - } - } - - @Override - public void add(Statusable job, String username, String jobId) { - Map> jobMapping = getJobsByUser(username); - jobMapping.put(jobId, job); - jobs.put(username, jobMapping); - try { - writeToHdfs(jobsPath, job.getJobType(), username, jobId); - } catch (IOException e) { - LOG.error("Unable to save running job information to HDFS: {}, {}, {}", job.getJobType(), - username, jobId, e); - } - } - - private Map> getJobsByUser(String username) { - return jobs.getOrDefault(username, new HashMap<>()); - } - - private void writeToHdfs(String basePath, JobType jobType, String username, String jobId) - throws IOException { - String path = String.format("%s/%s/%s/%s", basePath, username, jobType.toString(), jobId); - HDFSUtils.write(new Configuration(), new byte[]{}, path); - } - - @Override - public boolean jobExists(String username, String jobId) { - if (jobs.containsKey(username)) { - Map> jobsByUser = jobs.get(username); - return jobsByUser.containsKey(jobId); - } else { - return false; - } - } - - @Override - public Statusable getJob(String username, String jobId) { - if (jobs.containsKey(username)) { - Map> jobsByUser = jobs.get(username); - return jobsByUser.get(jobId); - } else { - return null; - } - } - -} diff --git a/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/JobManagerTest.java b/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java similarity index 52% rename from metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/JobManagerTest.java rename to metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java index a4e527c6b4..3311ec3935 100644 --- a/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/JobManagerTest.java +++ b/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java @@ -2,17 +2,21 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.util.HashMap; +import java.util.Map; import org.apache.hadoop.fs.Path; +import org.apache.metron.job.Finalizer; import org.apache.metron.job.JobException; import org.apache.metron.job.JobStatus; import org.apache.metron.job.JobStatus.State; +import org.apache.metron.job.Pageable; import org.apache.metron.job.Statusable; -import org.apache.metron.job.service.JobService; +import org.apache.metron.job.Statusable.JobType; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -38,65 +42,84 @@ * limitations under the License. */ -public class JobManagerTest { +public class InMemoryJobManagerTest { - private JobManager jm; @Mock - private JobService jobService; + private Statusable job1; + @Mock + private Statusable job2; @Mock - private Statusable job; - private String username; - private String jobId; + private Finalizer finalizer; + @Mock + private Pageable results; + private JobManager jm; + private Map config; + private String username1; + private String username2; + private String jobId1; + private String jobId2; @Before - public void setup() { + public void setup() throws JobException { MockitoAnnotations.initMocks(this); - jm = new JobManager(jobService); - username = "userabc"; - jobId = "job_abc"; + jm = new InMemoryJobManager(); + config = new HashMap<>(); + username1 = "user123"; + username2 = "user456"; + jobId1 = "job_abc_123"; + jobId2 = "job_def_456"; + when(job1.getJobType()).thenReturn(JobType.MAP_REDUCE); + when(job2.getJobType()).thenReturn(JobType.MAP_REDUCE); + when(finalizer.finalizeJob(any())).thenReturn(results); } @Test public void submits_job_and_returns_status() throws JobException { - when(job.validate(any())).thenReturn(true); - when(job.submit()).thenReturn(job); - when(job.getStatus()).thenReturn(new JobStatus().withState(State.SUCCEEDED).withJobId(jobId)); - JobStatus status = jm.submit(job, new HashMap<>(), username); - verify(jobService).add(job, jobId, username); + when(job1.validate(any())).thenReturn(true); + when(job1.submit(finalizer, config)).thenReturn(job1); + when(job1.getStatus()).thenReturn(new JobStatus().withState(State.RUNNING).withJobId(jobId1)); + JobStatus status = jm.submit(() -> { + try { + return job1.submit(finalizer, config); + } catch (JobException e) { + throw new RuntimeException("Something went wrong", e); + } + }, username1); + assertThat(status.getState(), equalTo(State.RUNNING)); + assertThat(status.getJobId(), equalTo(jobId1)); + when(job1.getStatus()).thenReturn(new JobStatus().withState(State.SUCCEEDED).withJobId(jobId1)); + status = jm.getStatus(username1, status.getJobId()); assertThat(status.getState(), equalTo(State.SUCCEEDED)); + assertThat(status.getJobId(), equalTo(jobId1)); } @Test public void returns_job_status() throws JobException { - JobStatus expected = new JobStatus().withState(State.SUCCEEDED).withJobId(jobId); - when(job.getStatus()).thenReturn(expected); - when(jobService.getJob(username, jobId)).thenReturn(job); - JobStatus status = jm.getStatus(username, jobId); + JobStatus expected = new JobStatus().withState(State.SUCCEEDED).withJobId(jobId1); + when(job1.getStatus()).thenReturn(expected); + JobStatus status = jm.getStatus(username1, jobId1); assertThat(status, equalTo(expected)); } @Test public void returns_job_is_done() throws JobException { - JobStatus expected = new JobStatus().withState(State.SUCCEEDED).withJobId(jobId); - when(job.isDone()).thenReturn(true); - when(jobService.getJob(username, jobId)).thenReturn(job); - boolean done = jm.done(username, jobId); + JobStatus expected = new JobStatus().withState(State.SUCCEEDED).withJobId(jobId1); + when(job1.isDone()).thenReturn(true); + boolean done = jm.done(username1, jobId1); assertThat(done, equalTo(true)); } @Test public void kills_job() throws JobException { - when(jobService.getJob(username, jobId)).thenReturn(job); - jm.killJob(username, jobId); - verify(job).kill(); + jm.killJob(username1, jobId1); + verify(job1).kill(); } @Test public void returns_statusable_job() throws JobException { - when(job.isDone()).thenReturn(true); - when(jobService.getJob(username, jobId)).thenReturn(job); - Statusable done = jm.getJob(username, jobId); - assertThat(done, equalTo(job)); + when(job1.isDone()).thenReturn(true); + Statusable done = jm.getJob(username1, jobId1); + assertThat(done, equalTo(job1)); } @Rule @@ -105,11 +128,12 @@ public void returns_statusable_job() throws JobException { @Test public void job_submission_exception_returned() throws JobException { JobException expected = new JobException("test exception"); - when(job.validate(any())).thenReturn(true); - when(job.submit()).thenThrow(expected); + when(job1.validate(any())).thenReturn(true); +// when(job.submit()).thenThrow(expected); exception.expect(JobException.class); exception.expect(equalTo(expected)); - jm.submit(job, new HashMap<>(), username); +// jm.submit(job, new HashMap<>(), username); + fail(); } } diff --git a/metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java b/metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java index a0f4df8f32..903c18a09e 100644 --- a/metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java +++ b/metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java @@ -18,18 +18,14 @@ package org.apache.metron.job.service; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; -import java.io.File; import java.util.HashMap; import java.util.Map; import org.apache.metron.job.Statusable; import org.apache.metron.job.Statusable.JobType; import org.junit.Before; import org.junit.Rule; -import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.mockito.Mock; import org.mockito.MockitoAnnotations; @@ -47,7 +43,6 @@ public class HdfsJobServiceTest { private String jobId1; private String jobId2; private String basePath; - private HdfsJobService js; @Before public void setup() { @@ -60,10 +55,8 @@ public void setup() { basePath = tempDir.getRoot().getAbsolutePath(); Map config = new HashMap<>(); config.put("basePath", basePath); - js = new HdfsJobService(); - js.configure(config); } - +/* @Test public void adds_jobs() { js.add(job1, username, jobId1); @@ -113,5 +106,5 @@ public void writes_job_info_to_hdfs() { assertThat("File should exist", jobFile.exists(), equalTo(true)); assertThat("File should be a file", jobFile.isFile(), equalTo(true)); } - +*/ } diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java index 2877235997..df997a4638 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java @@ -20,6 +20,8 @@ import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; import java.util.UUID; import org.apache.commons.cli.ParseException; import org.apache.commons.lang3.tuple.Pair; @@ -27,14 +29,14 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.util.GenericOptionsParser; -import org.apache.metron.common.hadoop.SequenceFileIterable; import org.apache.metron.common.system.Clock; import org.apache.metron.common.utils.timestamp.TimestampConverters; import org.apache.metron.job.JobException; +import org.apache.metron.job.Pageable; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.filter.query.QueryPcapFilter; +import org.apache.metron.pcap.finalizer.PcapFinalizerStrategies; import org.apache.metron.pcap.mr.PcapJob; -import org.apache.metron.pcap.writer.PcapResultsWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,17 +48,15 @@ public class PcapCli { return String.format("%s-%s", timestamp, uuid); }; private final PcapJob jobRunner; - private final PcapResultsWriter resultsWriter; private final CliConfig.PrefixStrategy prefixStrategy; public static void main(String[] args) { - int status = new PcapCli(new PcapJob(), new PcapResultsWriter(), PREFIX_STRATEGY).run(args); + int status = new PcapCli(new PcapJob(), PREFIX_STRATEGY).run(args); System.exit(status); } - public PcapCli(PcapJob jobRunner, PcapResultsWriter resultsWriter, CliConfig.PrefixStrategy prefixStrategy) { + public PcapCli(PcapJob jobRunner, CliConfig.PrefixStrategy prefixStrategy) { this.jobRunner = jobRunner; - this.resultsWriter = resultsWriter; this.prefixStrategy = prefixStrategy; } @@ -66,10 +66,8 @@ public int run(String[] args) { return -1; } String jobType = args[0]; - SequenceFileIterable results = null; String[] commandArgs = Arrays.copyOfRange(args, 1, args.length); Configuration hadoopConf = new Configuration(); - resultsWriter.withConfiguration(hadoopConf); String[] otherArgs = null; try { otherArgs = new GenericOptionsParser(hadoopConf, commandArgs).getRemainingArgs(); @@ -78,6 +76,19 @@ public int run(String[] args) { return -1; } CliConfig commonConfig = null; + Pageable results; + Map jobConfig = new HashMap<>(); + jobConfig.put("hadoopConf", hadoopConf); + try { + jobConfig.put("fileSystem", FileSystem.get(hadoopConf)); + } catch (IOException e) { + LOGGER.error("Failed to execute filter job: {}", e.getMessage(), e); + return -1; + } + // write to local FS in the executing directory + String execDir = System.getProperty("user.dir"); + jobConfig.put("finalOutputPath", new Path("file:///" + execDir)); + if ("fixed".equals(jobType)) { FixedCliParser fixedParser = new FixedCliParser(prefixStrategy); FixedCliConfig config = null; @@ -98,18 +109,19 @@ public int run(String[] args) { long startTime = time.getLeft(); long endTime = time.getRight(); + jobConfig.put("basePath", new Path(config.getBasePath())); + jobConfig.put("interimResultPath", new Path(config.getBaseOutputPath())); + jobConfig.put("beginNS", startTime); + jobConfig.put("endNS", endTime); + jobConfig.put("numReducers", config.getNumReducers()); + jobConfig.put("fields", config.getFixedFields()); + jobConfig.put("filterImpl", new FixedPcapFilter.Configurator()); + jobConfig.put("numRecordsPerFile", commonConfig.getNumRecordsPerFile()); + jobConfig.put("finalFilenamePrefix", commonConfig.getPrefix()); + try { - results = jobRunner.query( - new Path(config.getBasePath()), - new Path(config.getBaseOutputPath()), - startTime, - endTime, - config.getNumReducers(), - config.getFixedFields(), - hadoopConf, - FileSystem.get(hadoopConf), - new FixedPcapFilter.Configurator()); - } catch (IOException | ClassNotFoundException | InterruptedException | JobException e) { + results = jobRunner.submit(PcapFinalizerStrategies.CLI, jobConfig).get(); + } catch (InterruptedException | JobException e) { LOGGER.error("Failed to execute fixed filter job: {}", e.getMessage(), e); return -1; } @@ -132,19 +144,19 @@ public int run(String[] args) { long startTime = time.getLeft(); long endTime = time.getRight(); + jobConfig.put("basePath", new Path(config.getBasePath())); + jobConfig.put("interimResultPath", new Path(config.getBaseOutputPath())); + jobConfig.put("beginNS", startTime); + jobConfig.put("endNS", endTime); + jobConfig.put("numReducers", config.getNumReducers()); + jobConfig.put("fields", config.getQuery()); + jobConfig.put("filterImpl", new QueryPcapFilter.Configurator()); + jobConfig.put("numRecordsPerFile", commonConfig.getNumRecordsPerFile()); + jobConfig.put("finalFilenamePrefix", commonConfig.getPrefix()); try { - results = jobRunner.query( - new Path(config.getBasePath()), - new Path(config.getBaseOutputPath()), - startTime, - endTime, - config.getNumReducers(), - config.getQuery(), - hadoopConf, - FileSystem.get(hadoopConf), - new QueryPcapFilter.Configurator()); - } catch (IOException | ClassNotFoundException | InterruptedException | JobException e) { - LOGGER.error("Failed to execute query filter job: {}", e.getMessage(), e); + results = jobRunner.submit(PcapFinalizerStrategies.CLI, jobConfig).get(); + } catch (InterruptedException | JobException e) { + LOGGER.error("Failed to execute fixed filter job: {}", e.getMessage(), e); return -1; } } else { @@ -152,17 +164,6 @@ public int run(String[] args) { return -1; } - try { - // write to local FS in the executing directory - String execDir = System.getProperty("user.dir"); - jobRunner.writeFinalResults(results, resultsWriter, new Path("file:///" + execDir), - commonConfig.getNumRecordsPerFile(), - commonConfig.getPrefix()); - } catch (IOException e) { - LOGGER.error("Unable to write file", e); - return -1; - } - return 0; } diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java index c11002ffdc..4560bd9bda 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java @@ -19,12 +19,8 @@ package org.apache.metron.pcap; import static java.lang.Long.toUnsignedString; -import static java.lang.String.format; import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.notNullValue; -import static org.hamcrest.CoreMatchers.startsWith; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; +import static org.mockito.Matchers.any; import static org.mockito.Mockito.when; import java.io.IOException; @@ -39,8 +35,10 @@ import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.JobID; import org.apache.metron.common.utils.timestamp.TimestampConverters; +import org.apache.metron.job.Finalizer; import org.apache.metron.job.JobStatus; import org.apache.metron.job.JobStatus.State; +import org.apache.metron.job.Pageable; import org.apache.metron.job.Statusable; import org.apache.metron.pcap.filter.PcapFilterConfigurator; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; @@ -59,28 +57,59 @@ public class PcapJobTest { private org.apache.hadoop.mapreduce.JobStatus mrStatus; @Mock private JobID jobId; + @Mock + private Finalizer finalizer; + @Mock + private Pageable pageableResult; + private Map config; + private Configuration hadoopConfig; + private FileSystem fileSystem; private String jobIdVal = "job_abc_123"; private Path basePath; private Path baseOutPath; private long startTime; private long endTime; private int numReducers; + private int numRecordsPerFile; + private Path finalOutputPath; + private String prefix; private Map fixedFields; - private Configuration hadoopConfig; + private PcapJob> testJob; + @Before - public void setup() { + public void setup() throws IOException { MockitoAnnotations.initMocks(this); basePath = new Path("basepath"); baseOutPath = new Path("outpath"); startTime = 100; endTime = 200; numReducers = 5; + numRecordsPerFile = 5; fixedFields = new HashMap<>(); fixedFields.put("ip_src_addr", "192.168.1.1"); hadoopConfig = new Configuration(); + fileSystem = FileSystem.get(hadoopConfig); + finalOutputPath = new Path("finaloutpath"); + prefix = "someprefix"; when(jobId.toString()).thenReturn(jobIdVal); when(mrStatus.getJobID()).thenReturn(jobId); + config = new HashMap<>(); + config.put("hadoopConf", hadoopConfig); + config.put("fileSystem", FileSystem.get(hadoopConfig)); + config.put("basePath", basePath); + config.put("baseInterimResultPath", baseOutPath); + config.put("beginNS", startTime); + config.put("endNS", endTime); + config.put("numReducers", numReducers); + config.put("fields", fixedFields); + config.put("filterImpl", new FixedPcapFilter.Configurator()); + config.put("numRecordsPerFile", numRecordsPerFile); + config.put("finalOutputPath", finalOutputPath); + config.put("finalFilenamePrefix", prefix); + testJob = new TestJob<>(); + testJob.setStatusInterval(10); + testJob.setCompleteCheckInterval(10); } @Test @@ -98,11 +127,18 @@ public void partition_gives_value_in_range() throws Exception { equalTo(8)); } - private class TestJob extends PcapJob { + private class TestJob extends PcapJob { @Override - public Job createJob(Optional jobName, Path basePath, Path outputPath, long beginNS, long endNS, - int numReducers, T fields, Configuration conf, FileSystem fs, + public Job createJob(Optional jobName, + Path basePath, + Path outputPath, + long beginNS, + long endNS, + int numReducers, + T fields, + Configuration conf, + FileSystem fs, PcapFilterConfigurator filterImpl) throws IOException { return job; } @@ -110,30 +146,18 @@ public Job createJob(Optional jobName, Path basePath, Path outputPat @Test public void job_succeeds_synchronously() throws Exception { + when(pageableResult.getSize()).thenReturn(3); + when(finalizer.finalizeJob(any())).thenReturn(pageableResult); when(job.isComplete()).thenReturn(true); when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.SUCCEEDED); when(job.getStatus()).thenReturn(mrStatus); - TestJob testJob = new TestJob(); - Statusable statusable = testJob.query( - Optional.empty(), - basePath, - baseOutPath, - startTime, - endTime, - numReducers, - fixedFields, - hadoopConfig, - FileSystem.get(hadoopConfig), - new FixedPcapFilter.Configurator(), - true); - verify(job, times(1)).waitForCompletion(true); + Statusable statusable = testJob.submit(finalizer, config); + Pageable results = statusable.get(); + Assert.assertThat(results.getSize(), equalTo(3)); JobStatus status = statusable.getStatus(); Assert.assertThat(status.getState(), equalTo(State.SUCCEEDED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); Assert.assertThat(status.getJobId(), equalTo(jobIdVal)); - String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); - Assert.assertThat(status.getInterimResultPath(), notNullValue()); - Assert.assertThat(status.getInterimResultPath().toString(), startsWith(expectedOutPath)); } @Test @@ -141,25 +165,12 @@ public void job_fails_synchronously() throws Exception { when(job.isComplete()).thenReturn(true); when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.FAILED); when(job.getStatus()).thenReturn(mrStatus); - TestJob testJob = new TestJob(); - Statusable statusable = testJob.query( - Optional.empty(), - basePath, - baseOutPath, - startTime, - endTime, - numReducers, - fixedFields, - hadoopConfig, - FileSystem.get(hadoopConfig), - new FixedPcapFilter.Configurator(), - true); + Statusable statusable = testJob.submit(finalizer, config); + Pageable results = statusable.get(); JobStatus status = statusable.getStatus(); Assert.assertThat(status.getState(), equalTo(State.FAILED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); - String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); - Assert.assertThat(status.getInterimResultPath(), notNullValue()); - Assert.assertThat(status.getInterimResultPath().toString(), startsWith(expectedOutPath)); + Assert.assertThat(results, equalTo(null)); } @Test @@ -167,25 +178,12 @@ public void job_fails_with_killed_status_synchronously() throws Exception { when(job.isComplete()).thenReturn(true); when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.KILLED); when(job.getStatus()).thenReturn(mrStatus); - TestJob testJob = new TestJob(); - Statusable statusable = testJob.query( - Optional.empty(), - basePath, - baseOutPath, - startTime, - endTime, - numReducers, - fixedFields, - hadoopConfig, - FileSystem.get(hadoopConfig), - new FixedPcapFilter.Configurator(), - true); + Statusable statusable = testJob.submit(finalizer, config); + Pageable results = statusable.get(); JobStatus status = statusable.getStatus(); Assert.assertThat(status.getState(), equalTo(State.KILLED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); - String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); - Assert.assertThat(status.getInterimResultPath(), notNullValue()); - Assert.assertThat(status.getInterimResultPath().toString(), startsWith(expectedOutPath)); + Assert.assertThat(results, equalTo(null)); } @Test @@ -193,25 +191,11 @@ public void job_succeeds_asynchronously() throws Exception { when(job.isComplete()).thenReturn(true); when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.SUCCEEDED); when(job.getStatus()).thenReturn(mrStatus); - TestJob testJob = new TestJob(); - Statusable statusable = testJob.query( - Optional.empty(), - basePath, - baseOutPath, - startTime, - endTime, - numReducers, - fixedFields, - hadoopConfig, - FileSystem.get(hadoopConfig), - new FixedPcapFilter.Configurator(), - false); + Statusable statusable = testJob.submit(finalizer, config); + while(!statusable.isDone()) {} JobStatus status = statusable.getStatus(); Assert.assertThat(status.getState(), equalTo(State.SUCCEEDED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); - String expectedOutPath = new Path(baseOutPath, format("%s_%s_%s", startTime, endTime, "192.168.1.1")).toString(); - Assert.assertThat(status.getInterimResultPath(), notNullValue()); - Assert.assertThat(status.getInterimResultPath().toString(), startsWith(expectedOutPath)); } @Test @@ -219,19 +203,7 @@ public void job_reports_percent_complete() throws Exception { when(job.isComplete()).thenReturn(false); when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.RUNNING); when(job.getStatus()).thenReturn(mrStatus); - TestJob testJob = new TestJob(); - Statusable statusable = testJob.query( - Optional.empty(), - basePath, - baseOutPath, - startTime, - endTime, - numReducers, - fixedFields, - hadoopConfig, - FileSystem.get(hadoopConfig), - new FixedPcapFilter.Configurator(), - false); + Statusable statusable = testJob.submit(finalizer, config); when(job.mapProgress()).thenReturn(0.5f); when(job.reduceProgress()).thenReturn(0f); JobStatus status = statusable.getStatus(); diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java index c7292abc08..c1cf6046a5 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java @@ -23,28 +23,24 @@ import com.google.common.collect.Collections2; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import java.io.ByteArrayOutputStream; import java.io.File; import java.io.FilenameFilter; import java.io.IOException; import java.util.AbstractMap; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; import javax.annotation.Nullable; import kafka.consumer.ConsumerIterator; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.SequenceFile; import org.apache.kafka.clients.producer.Producer; -import org.apache.metron.common.Constants; import org.apache.metron.integration.BaseIntegrationTest; import org.apache.metron.integration.ComponentRunner; import org.apache.metron.integration.Processor; @@ -57,9 +53,6 @@ import org.apache.metron.integration.utils.KafkaUtil; import org.apache.metron.pcap.PacketInfo; import org.apache.metron.pcap.PcapHelper; -import org.apache.metron.pcap.PcapMerger; -import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; -import org.apache.metron.pcap.filter.query.QueryPcapFilter; import org.apache.metron.pcap.mr.PcapJob; import org.apache.metron.spout.pcap.Endianness; import org.apache.metron.spout.pcap.deserializer.Deserializers; @@ -234,6 +227,11 @@ public ProcessorResult getResult() { } }); PcapJob job = new PcapJob(); + + // TODO + + /* + { //Ensure that only two pcaps are returned when we look at 4 and 5 Iterable results = @@ -510,6 +508,7 @@ public boolean apply(@Nullable JSONObject input) { Assert.assertTrue(baos.toByteArray().length > 0); } System.out.println("Ended"); + */ } finally { runner.stop(); clearOutDir(outDir); diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java index daf3f457f2..58d1f085d3 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java @@ -19,12 +19,6 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertThat; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.anyObject; -import static org.mockito.Matchers.eq; -import static org.mockito.Matchers.isA; -import static org.mockito.Mockito.doCallRealMethod; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -41,22 +35,16 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.metron.common.Constants; import org.apache.metron.common.hadoop.SequenceFileIterable; import org.apache.metron.common.system.Clock; import org.apache.metron.common.utils.timestamp.TimestampConverters; import org.apache.metron.pcap.PcapHelper; -import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; -import org.apache.metron.pcap.filter.query.QueryPcapFilter; import org.apache.metron.pcap.mr.PcapJob; -import org.apache.metron.pcap.writer.PcapResultsWriter; import org.junit.Before; import org.junit.Test; import org.mockito.Mock; -import org.mockito.Mockito; import org.mockito.MockitoAnnotations; public class PcapCliTest { @@ -64,15 +52,13 @@ public class PcapCliTest { @Mock private PcapJob jobRunner; @Mock - private PcapResultsWriter resultsWriter; - @Mock private Clock clock; private String execDir; @Before public void setup() throws IOException { MockitoAnnotations.initMocks(this); - doCallRealMethod().when(jobRunner).writeFinalResults(anyObject(), anyObject(), anyObject(), anyInt(), anyObject()); +// doCallRealMethod().when(jobRunner).writeFinalResults(anyObject(), anyObject(), anyObject(), anyInt(), anyObject()); execDir = System.getProperty("user.dir"); } @@ -105,11 +91,11 @@ public void runs_fixed_pcap_filter_job_with_default_argument_list() throws Excep put(PcapHelper.PacketFields.PACKET_FILTER.getName(), "`casey`"); }}; - when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); +// when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); - PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); + PcapCli cli = new PcapCli(jobRunner, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); +// Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } @Test @@ -145,11 +131,11 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list_and_default_datef put(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName(), "true"); }}; - when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); +// when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); - PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); + PcapCli cli = new PcapCli(jobRunner, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); +// Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } @Test @@ -188,11 +174,11 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list() throws Exceptio long startAsNanos = asNanos("2016-06-13-18:35.00", "yyyy-MM-dd-HH:mm.ss"); long endAsNanos = asNanos("2016-06-15-18:35.00", "yyyy-MM-dd-HH:mm.ss"); - when(jobRunner.query(eq(base_path), eq(base_output_path), eq(startAsNanos), eq(endAsNanos), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); +// when(jobRunner.query(eq(base_path), eq(base_output_path), eq(startAsNanos), eq(endAsNanos), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); - PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); + PcapCli cli = new PcapCli(jobRunner, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); +// Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } private long asNanos(String inDate, String format) throws ParseException { @@ -221,11 +207,11 @@ public void runs_query_pcap_filter_job_with_default_argument_list() throws Excep Path base_output_path = new Path(CliParser.BASE_OUTPUT_PATH_DEFAULT); String query = "some query string"; - when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(QueryPcapFilter.Configurator.class))).thenReturn(iterable); +// when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(QueryPcapFilter.Configurator.class))).thenReturn(iterable); - PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); + PcapCli cli = new PcapCli(jobRunner, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); +// Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } @Test @@ -249,11 +235,11 @@ public void runs_query_pcap_filter_job_with_full_argument_list() throws Exceptio Path base_output_path = new Path("/base/output/path"); String query = "some query string"; - when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(QueryPcapFilter.Configurator.class))).thenReturn(iterable); +// when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(QueryPcapFilter.Configurator.class))).thenReturn(iterable); - PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); + PcapCli cli = new PcapCli(jobRunner, clock -> "random_prefix"); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); - Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); +// Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); } // INVALID OPTION CHECKS @@ -290,7 +276,7 @@ public void assertCliError(String[] args, String type, String optMsg) { PrintStream errOutStream = new PrintStream(new BufferedOutputStream(ebos)); System.setErr(errOutStream); - PcapCli cli = new PcapCli(jobRunner, resultsWriter, clock -> "random_prefix"); + PcapCli cli = new PcapCli(jobRunner, clock -> "random_prefix"); assertThat("Expect errors on run", cli.run(args), equalTo(-1)); assertThat("Expect missing required option error: " + ebos.toString(), ebos.toString().contains(optMsg), equalTo(true)); assertThat("Expect usage to be printed: " + bos.toString(), bos.toString().contains("usage: " + type + " filter options"), equalTo(true)); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java index 997c5f79d8..de82015ada 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java @@ -39,4 +39,9 @@ public Iterable asIterable() { public Path getPage(int num) { return files.get(num); } + + @Override + public int getSize() { + return files.size(); + } } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/writer/PrefixStrategy.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java similarity index 60% rename from metron-platform/metron-job/src/main/java/org/apache/metron/job/writer/PrefixStrategy.java rename to metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java index af7ce042e4..2e0b1de0be 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/writer/PrefixStrategy.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java @@ -16,11 +16,21 @@ * limitations under the License. */ -package org.apache.metron.job.writer; +package org.apache.metron.pcap.finalizer; -import java.util.function.Function; -import org.apache.metron.common.system.Clock; +import java.util.Map; +import org.apache.hadoop.fs.Path; -public interface PrefixStrategy extends Function { +/** + * Write to local FS. + */ +public class PcapCliFinalizer extends PcapFinalizer { + + @Override + protected String getOutputFileName(Map config, int partition) { + Path finalOutputPath = (Path) PcapFinalizerOptions.FINAL_OUTPUT_PATH.get(config); + String prefix = (String) config.get("finalFilenamePrefix"); + return String.format("%s/pcap-data-%s+%04d.pcap", finalOutputPath, prefix, partition); + } } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java new file mode 100644 index 0000000000..fb8f9695d1 --- /dev/null +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java @@ -0,0 +1,149 @@ +/** + * 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.metron.pcap.finalizer; + +import static org.apache.metron.pcap.finalizer.PcapFinalizer.PcapFinalizerOptions.FILE_SYSTEM; +import static org.apache.metron.pcap.finalizer.PcapFinalizer.PcapFinalizerOptions.FINAL_OUTPUT_PATH; +import static org.apache.metron.pcap.finalizer.PcapFinalizer.PcapFinalizerOptions.HADOOP_CONF; +import static org.apache.metron.pcap.finalizer.PcapFinalizer.PcapFinalizerOptions.INTERIM_RESULTS_PATH; +import static org.apache.metron.pcap.finalizer.PcapFinalizer.PcapFinalizerOptions.NUM_RECORDS_PER_FILE; + +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.metron.common.hadoop.SequenceFileIterable; +import org.apache.metron.job.Finalizer; +import org.apache.metron.job.JobException; +import org.apache.metron.job.Pageable; +import org.apache.metron.pcap.PcapFiles; +import org.apache.metron.pcap.writer.PcapResultsWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class PcapFinalizer implements Finalizer { + + public enum PcapFinalizerOptions { + HADOOP_CONF("hadoopConf"), + NUM_RECORDS_PER_FILE("numRecordsPerFile"), + INTERIM_RESULTS_PATH("interimResultsPath"), + FINAL_OUTPUT_PATH("finalOutputPath"), + FILE_SYSTEM("fileSystem"); + + private String optionName; + + PcapFinalizerOptions(String optionName) { + this.optionName = optionName; + } + + public boolean has(Map config) { + return config.containsKey(optionName); + } + + public Object get(Map config) { + return config.get(optionName); + } + } + + private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + private PcapResultsWriter resultsWriter; + + protected PcapFinalizer() { + this.resultsWriter = new PcapResultsWriter(); + } + + protected PcapResultsWriter getResultsWriter() { + return resultsWriter; + } + + @Override + public Pageable finalizeJob(Map config) throws JobException { + Configuration hadoopConfig = (Configuration) HADOOP_CONF.get(config); + int recPerFile = (int) NUM_RECORDS_PER_FILE.get(config); + Path interimResultPath = (Path) INTERIM_RESULTS_PATH.get(config); + Path finalOutputPath = (Path) FINAL_OUTPUT_PATH.get(config); + FileSystem fs = (FileSystem) FILE_SYSTEM.get(config); + + SequenceFileIterable interimResults = null; + try { + interimResults = readInterimResults(interimResultPath, hadoopConfig, fs); + } catch (IOException e) { + throw new JobException("Unable to read interim job results while finalizing", e); + } + List outFiles = new ArrayList<>(); + try { + Iterable> partitions = Iterables.partition(interimResults, recPerFile); + int part = 1; + if (partitions.iterator().hasNext()) { + for (List data : partitions) { + String outFileName = getOutputFileName(config, part++); + if (data.size() > 0) { + getResultsWriter().write(hadoopConfig, data, outFileName); + outFiles.add(new Path(outFileName)); + } + } + } else { + LOG.info("No results returned."); + } + } catch (IOException e) { + throw new JobException("Failed to finalize results", e); + } finally { + try { + interimResults.cleanup(); + } catch (IOException e) { + LOG.warn("Unable to cleanup files in HDFS", e); + } + } + return new PcapFiles(outFiles); + } + + protected abstract String getOutputFileName(Map config, int partition); + + /** + * Returns a lazily-read Iterable over a set of sequence files. + */ + protected SequenceFileIterable readInterimResults(Path interimResultPath, Configuration config, + FileSystem fs) throws IOException { + List files = new ArrayList<>(); + for (RemoteIterator it = fs.listFiles(interimResultPath, false); + it.hasNext(); ) { + Path p = it.next().getPath(); + if (p.getName().equals("_SUCCESS")) { + fs.delete(p, false); + continue; + } + files.add(p); + } + if (files.size() == 0) { + LOG.info("No files to process with specified date range."); + } else { + LOG.debug("Interim results path={}", interimResultPath); + Collections.sort(files, (o1, o2) -> o1.getName().compareTo(o2.getName())); + } + return new SequenceFileIterable(files, config); + } +} diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobServiceStrategies.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizerStrategies.java similarity index 53% rename from metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobServiceStrategies.java rename to metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizerStrategies.java index d3685371bd..8059036828 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/service/JobServiceStrategies.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizerStrategies.java @@ -16,38 +16,27 @@ * limitations under the License. */ -package org.apache.metron.job.service; +package org.apache.metron.pcap.finalizer; import java.util.Map; import org.apache.hadoop.fs.Path; -import org.apache.metron.job.Statusable; +import org.apache.metron.job.Finalizer; +import org.apache.metron.job.JobException; +import org.apache.metron.job.Pageable; -public enum JobServiceStrategies implements JobService { - HDFS(new HdfsJobService()); +public enum PcapFinalizerStrategies implements Finalizer { + CLI(new PcapCliFinalizer()), + REST(new PcapRestFinalizer()); - private JobService jobService; + private Finalizer finalizer; - JobServiceStrategies(JobService jobService) { - this.jobService = jobService; + PcapFinalizerStrategies(Finalizer finalizer) { + this.finalizer = finalizer; } @Override - public void configure(Map config) { - jobService.configure(config); + public Pageable finalizeJob(Map config) throws JobException { + return finalizer.finalizeJob(config); } - @Override - public void add(Statusable job, String username, String jobId) { - jobService.add(job, username, jobId); - } - - @Override - public boolean jobExists(String username, String jobId) { - return jobService.jobExists(username, jobId); - } - - @Override - public Statusable getJob(String username, String jobId) { - return jobService.getJob(username, jobId); - } } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/writer/ResultsWriter.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java similarity index 66% rename from metron-platform/metron-job/src/main/java/org/apache/metron/job/writer/ResultsWriter.java rename to metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java index 8ab4b18f76..463120aea7 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/writer/ResultsWriter.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java @@ -16,20 +16,20 @@ * limitations under the License. */ -package org.apache.metron.job.writer; +package org.apache.metron.pcap.finalizer; -import java.io.IOException; -import java.util.List; +import java.util.Map; +import org.apache.hadoop.fs.Path; -public interface ResultsWriter { +/** + * Write to HDFS. + */ +public class PcapRestFinalizer extends PcapFinalizer { - /** - * Write out results. - * - * @param data data to write. - * @param outPath where to write the data to. - * @throws IOException I/O issue encountered. - */ - void write(List data, String outPath) throws IOException; + @Override + protected String getOutputFileName(Map config, int partition) { + Path finalOutputPath = (Path) config.get("finalOutputPath"); + return String.format("%s/page-%s", finalOutputPath, partition); + } } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index a4789641cb..fd8c5ed4f6 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -22,17 +22,17 @@ import static org.apache.metron.pcap.PcapHelper.lessThanOrEqualTo; import com.google.common.base.Joiner; -import com.google.common.collect.Iterables; import java.io.IOException; import java.lang.invoke.MethodHandles; import java.text.DateFormat; import java.text.SimpleDateFormat; import java.util.ArrayList; -import java.util.Collections; import java.util.Date; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Timer; +import java.util.TimerTask; import java.util.UUID; import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; @@ -50,21 +50,18 @@ import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; -import org.apache.metron.common.hadoop.SequenceFileIterable; +import org.apache.metron.job.Finalizer; import org.apache.metron.job.JobException; import org.apache.metron.job.JobStatus; import org.apache.metron.job.JobStatus.State; import org.apache.metron.job.Pageable; import org.apache.metron.job.Statusable; -import org.apache.metron.job.writer.ResultsWriter; import org.apache.metron.pcap.PacketInfo; -import org.apache.metron.pcap.PcapFiles; import org.apache.metron.pcap.PcapHelper; import org.apache.metron.pcap.filter.PcapFilter; import org.apache.metron.pcap.filter.PcapFilterConfigurator; import org.apache.metron.pcap.filter.PcapFilters; import org.apache.metron.pcap.utils.FileFilterUtil; -import org.apache.metron.pcap.writer.ResultsWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -72,17 +69,24 @@ * Encompasses MapReduce job and final writing of Pageable results to specified location. * Cleans up MapReduce results from HDFS on completion. */ -public class PcapJob implements Statusable { +public class PcapJob implements Statusable { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); public static final String START_TS_CONF = "start_ts"; public static final String END_TS_CONF = "end_ts"; public static final String WIDTH_CONF = "width"; + public static final long THREE_SECONDS = 3000; + public static final long ONE_SECOND = 1000; private Job job; // store a running MR job reference for async status check private JobStatus jobStatus; - private PcapMRJobConfig config; private boolean finalized; // job results have been finalized private Pageable jobResults; + private Finalizer finalizer; + private Map configuration; + private Pageable results; + private Timer timer; + private long statusInterval; // how often timer thread checks job status. + private long completeCheckInterval; // how long we sleep between isDone checks in get() public static enum PCAP_COUNTER { MALFORMED_PACKET_COUNT @@ -177,67 +181,90 @@ protected void reduce(LongWritable key, Iterable values, Context } public PcapJob() { + jobStatus = new JobStatus(); + statusInterval = THREE_SECONDS; + completeCheckInterval = ONE_SECOND; } - public PcapJob(PcapMRJobConfig config) { - this.config = config; + /** + * Primarily for testing. + * + * @param interval time in millis + */ + public void setStatusInterval(long interval) { + statusInterval = interval; } /** - * Run query synchronously. + * Primarily for testing. + * + * @param interval time in milllis */ - public SequenceFileIterable query(Path basePath - , Path baseOutputPath - , long beginNS - , long endNS - , int numReducers - , T fields - , Configuration conf - , FileSystem fs - , PcapFilterConfigurator filterImpl - ) - throws IOException, ClassNotFoundException, InterruptedException, JobException { - Statusable statusable = query(Optional.empty(), basePath, baseOutputPath, beginNS, endNS, - numReducers, fields, conf, fs, filterImpl, true); - JobStatus jobStatus = statusable.getStatus(); - if (jobStatus.getState() == State.SUCCEEDED) { - Path resultPath = jobStatus.getInterimResultPath(); - return readInterimResults(resultPath, conf, fs); - } else { - throw new RuntimeException( - "Unable to complete query due to errors. Please check logs for full errors."); + public void setCompleteCheckInterval(long interval) { + completeCheckInterval = interval; + } + + @Override + public Statusable submit(Finalizer finalizer, Map configuration) + throws JobException { + this.finalizer = finalizer; + this.configuration = configuration; + Optional jobName = Optional.ofNullable((String) configuration.get("jobName")); + Configuration hadoopConf = (Configuration) configuration.get("hadoopConf"); + FileSystem fileSystem = (FileSystem) configuration.get("fileSystem"); + Path basePath = (Path) configuration.get("basePath"); + Path baseInterimResultPath = (Path) configuration.get("baseInterimResultPath"); + long startTime = (long) configuration.get("beginNS"); + long endTime = (long) configuration.get("endNS"); + int numReducers = (int) configuration.get("numReducers"); + T fields = (T) configuration.get("fields"); + PcapFilterConfigurator filterImpl = (PcapFilterConfigurator) configuration.get("filterImpl"); + int numRecordsPerFile = (int) configuration.get("numRecordsPerFile"); + String finalFilenamePrefix = (String) configuration.get("finalFilenamePrefix"); + + try { + return query(jobName, + basePath, + baseInterimResultPath, + startTime, + endTime, + numReducers, + fields, + hadoopConf, + fileSystem, + filterImpl); + } catch (IOException | InterruptedException | ClassNotFoundException e) { + throw new JobException("Failed to run pcap query.", e); } } /** - * Run query sync OR async based on flag. Async mode allows the client to check the returned - * statusable object for status details. + * Run query asynchronously. */ - public Statusable query(Optional jobName, + public Statusable query(Optional jobName, Path basePath, - Path baseOutputPath, + Path baseInterimResultPath, long beginNS, long endNS, int numReducers, T fields, Configuration conf, FileSystem fs, - PcapFilterConfigurator filterImpl, - boolean sync) + PcapFilterConfigurator filterImpl) throws IOException, ClassNotFoundException, InterruptedException { String outputDirName = Joiner.on("_").join(beginNS, endNS, filterImpl.queryToString(fields), UUID.randomUUID().toString()); if(LOG.isDebugEnabled()) { - DateFormat format = SimpleDateFormat.getDateTimeInstance( SimpleDateFormat.LONG + DateFormat format = SimpleDateFormat.getDateTimeInstance(SimpleDateFormat.LONG , SimpleDateFormat.LONG ); String from = format.format(new Date(Long.divideUnsigned(beginNS, 1000000))); String to = format.format(new Date(Long.divideUnsigned(endNS, 1000000))); LOG.debug("Executing query {} on timerange from {} to {}", filterImpl.queryToString(fields), from, to); } - Path interimOutputPath = new Path(baseOutputPath, outputDirName); + Path interimResultPath = new Path(baseInterimResultPath, outputDirName); job = createJob(jobName , basePath - , interimOutputPath + , interimResultPath , beginNS , endNS , numReducers @@ -246,63 +273,38 @@ public Statusable query(Optional jobName, , fs , filterImpl ); - jobStatus = new JobStatus().withInterimResultPath(interimOutputPath); - if (sync) { - job.waitForCompletion(true); - } else { - job.submit(); - } + job.submit(); + startJobStatusTimerThread(statusInterval); return this; } - /** - * Returns a lazily-read Iterable over a set of sequence files - */ - public SequenceFileIterable readInterimResults(Path outputPath, Configuration config, FileSystem fs) throws IOException { - List files = new ArrayList<>(); - for (RemoteIterator it = fs.listFiles(outputPath, false); it.hasNext(); ) { - Path p = it.next().getPath(); - if (p.getName().equals("_SUCCESS")) { - fs.delete(p, false); - continue; - } - files.add(p); - } - if (files.size() == 0) { - LOG.info("No files to process with specified date range."); - } else { - LOG.debug("Output path={}", outputPath); - Collections.sort(files, (o1, o2) -> o1.getName().compareTo(o2.getName())); - } - return new SequenceFileIterable(files, config); - } + private void startJobStatusTimerThread(long interval) { + timer = new Timer(); + timer.scheduleAtFixedRate(new TimerTask() { + @Override + public void run() { + try { + getStatus(); + if (job.isComplete() && !finalized) { + results = finalizer.finalizeJob(configuration); + finalized = true; + jobStatus.withState(State.SUCCEEDED); + cancel(); // be gone, ye! + } + } catch (IOException | JobException e) { + throw new RuntimeException("Error getting job status.", e); + } - @Override - public Statusable submit() throws JobException { - try { - return query( - config.getJobName(), - config.getBasePath(), - config.getBaseOutputPath(), - config.getBeginNS(), - config.getEndNS(), - config.getNumReducers(), - config.getFields(), - config.getConf(), - config.getFs(), - config.getFilterImpl(), - config.isSynchronous()); - } catch (IOException | ClassNotFoundException | InterruptedException e) { - throw new JobException("Unable to run pcap query.", e); - } + } + }, interval, interval); } /** - * Creates, but does not submit the job. + * Creates, but does not submit the job. This is the core MapReduce job. */ - public Job createJob(Optional jobName + public Job createJob(Optional jobName ,Path basePath - , Path outputPath + , Path jobOutputPath , long beginNS , long endNS , int numReducers @@ -335,7 +337,7 @@ public Job createJob(Optional jobName SequenceFileInputFormat.addInputPaths(job, inputPaths); job.setInputFormatClass(SequenceFileInputFormat.class); job.setOutputFormatClass(SequenceFileOutputFormat.class); - SequenceFileOutputFormat.setOutputPath(job, outputPath); + SequenceFileOutputFormat.setOutputPath(job, jobOutputPath); return job; } @@ -365,31 +367,33 @@ public JobStatus getStatus() throws JobException { } else { try { jobStatus.withJobId(job.getStatus().getJobID().toString()); - if (job.isComplete()) { - jobStatus.withPercentComplete(100); - switch (job.getStatus().getState()) { - case SUCCEEDED: - jobStatus.withState(State.SUCCEEDED).withDescription(State.SUCCEEDED.toString()); - break; - case FAILED: - jobStatus.withState(State.FAILED).withDescription(State.FAILED.toString()); - break; - case KILLED: - jobStatus.withState(State.KILLED).withDescription(State.KILLED.toString()); - break; - default: - throw new IllegalStateException( - "Unknown job state reported as 'complete' by mapreduce framework: " + job - .getStatus().getState()); - + if (!finalized) { + if (job.isComplete()) { + jobStatus.withPercentComplete(100); + switch (job.getStatus().getState()) { + case SUCCEEDED: + jobStatus.withState(State.FINALIZING).withDescription(State.FINALIZING.toString()); + break; + case FAILED: + jobStatus.withState(State.FAILED).withDescription(State.FAILED.toString()); + break; + case KILLED: + jobStatus.withState(State.KILLED).withDescription(State.KILLED.toString()); + break; + default: + throw new IllegalStateException( + "Unknown job state reported as 'complete' by mapreduce framework: " + job + .getStatus().getState()); + } + } else { + float mapProg = job.mapProgress(); + float reduceProg = job.reduceProgress(); + float totalProgress = ((mapProg / 2) + (reduceProg / 2)) * 100; + String description = String + .format("map: %s%%, reduce: %s%%", mapProg * 100, reduceProg * 100); + jobStatus.withPercentComplete(totalProgress).withState(State.RUNNING) + .withDescription(description); } - } else { - float mapProg = job.mapProgress(); - float reduceProg = job.reduceProgress(); - float totalProgress = ((mapProg / 2) + (reduceProg / 2)) * 100; - String description = String.format("map: %s%%, reduce: %s%%", mapProg * 100, reduceProg * 100); - jobStatus.withPercentComplete(totalProgress).withState(State.RUNNING) - .withDescription(description); } } catch (Exception e) { throw new RuntimeException("Error occurred while attempting to retrieve job status.", e); @@ -398,67 +402,24 @@ public JobStatus getStatus() throws JobException { return jobStatus; } - @SuppressWarnings("unchecked") @Override - public Pageable finalizeJob() throws JobException { - if (this.isDone()) { - try { - SequenceFileIterable interimResults = readInterimResults(jobStatus.getInterimResultPath(), - config.getConf(), config.getFs()); - Pageable jobResults = writeFinalResults(interimResults, config.getResultsWriter(), - config.getFinalOutputPath(), - config.getNumRecordsPerFile(), - config.getOutputFilePrefix()); - finalized = true; - return jobResults; - } catch (IOException e) { - throw new JobException("Unable to read intermediate pcap MapReduce results.", e); + public Pageable get() throws JobException, InterruptedException { + for (; ; ) { + getStatus(); + if (jobStatus.getState() == State.SUCCEEDED) { + return results; + } else if (jobStatus.getState() == State.KILLED + || jobStatus.getState() == State.FAILED) { + return null; } - } else { - return null; + Thread.sleep(completeCheckInterval); } } - @Override - public Pageable getFinalResults() throws JobException { - return jobResults; - } - - public Pageable writeFinalResults(SequenceFileIterable results, ResultsWriter resultsWriter, - Path outPath, int recPerFile, String prefix) throws IOException { - List outFiles = new ArrayList<>(); - try { - Iterable> partitions = Iterables.partition(results, recPerFile); - int part = 1; - if (partitions.iterator().hasNext()) { - for (List data : partitions) { - String outFileName = String.format("%s/pcap-data-%s+%04d.pcap", outPath, prefix, part++); - if (data.size() > 0) { - resultsWriter.write(data, outFileName); - outFiles.add(new Path(outFileName)); - } - } - } else { - LOG.info("No results returned."); - } - } finally { - try { - results.cleanup(); - } catch (IOException e) { - LOG.warn("Unable to cleanup files in HDFS", e); - } - } - return new PcapFiles(outFiles); - } - @Override public boolean isDone() throws JobException { getStatus(); - try { - return job.isComplete(); - } catch (IOException e) { - throw new RuntimeException("Error occurred while attempting to retrieve job status.", e); - } + return finalized; } @Override diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapMRJobConfig.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapMRJobConfig.java deleted file mode 100644 index dce7fe78b3..0000000000 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapMRJobConfig.java +++ /dev/null @@ -1,181 +0,0 @@ -/** - * 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.metron.pcap.mr; - -import java.util.Optional; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.metron.job.writer.ResultsWriter; -import org.apache.metron.pcap.filter.PcapFilterConfigurator; - -public class PcapMRJobConfig { - - private Optional jobName; - private Path basePath; - private Path baseOutputPath; - private Path finalOutputPath; - private ResultsWriter resultsWriter; - private int numRecordsPerFile; - private String outputFilePrefix; - private long beginNS; - private long endNS; - private int numReducers; - private T fields; - private Configuration conf; - private FileSystem fs; - private PcapFilterConfigurator filterImpl; - private boolean synchronous; - - public Optional getJobName() { - return jobName; - } - - public PcapMRJobConfig setJobName(Optional jobName) { - this.jobName = jobName; - return this; - } - - public Path getBasePath() { - return basePath; - } - - public PcapMRJobConfig setBasePath(Path basePath) { - this.basePath = basePath; - return this; - } - - public Path getBaseOutputPath() { - return baseOutputPath; - } - - public PcapMRJobConfig setBaseOutputPath(Path baseOutputPath) { - this.baseOutputPath = baseOutputPath; - return this; - } - - public long getBeginNS() { - return beginNS; - } - - public PcapMRJobConfig setBeginNS(long beginNS) { - this.beginNS = beginNS; - return this; - } - - public long getEndNS() { - return endNS; - } - - public PcapMRJobConfig setEndNS(long endNS) { - this.endNS = endNS; - return this; - } - - public int getNumReducers() { - return numReducers; - } - - public PcapMRJobConfig setNumReducers(int numReducers) { - this.numReducers = numReducers; - return this; - } - - public T getFields() { - return fields; - } - - public PcapMRJobConfig setFields(T fields) { - this.fields = fields; - return this; - } - - public Configuration getConf() { - return conf; - } - - public PcapMRJobConfig setConf(Configuration conf) { - this.conf = conf; - return this; - } - - public FileSystem getFs() { - return fs; - } - - public PcapMRJobConfig setFs(FileSystem fs) { - this.fs = fs; - return this; - } - - public PcapFilterConfigurator getFilterImpl() { - return filterImpl; - } - - public PcapMRJobConfig setFilterImpl(PcapFilterConfigurator filterImpl) { - this.filterImpl = filterImpl; - return this; - } - - public boolean isSynchronous() { - return synchronous; - } - - public PcapMRJobConfig setSynchronous(boolean synchronous) { - this.synchronous = synchronous; - return this; - } - - public Path getFinalOutputPath() { - return finalOutputPath; - } - - public PcapMRJobConfig setFinalOutputPath(Path finalOutputPath) { - this.finalOutputPath = finalOutputPath; - return this; - } - - public ResultsWriter getResultsWriter() { - return resultsWriter; - } - - public PcapMRJobConfig setResultsWriter(ResultsWriter resultsWriter) { - this.resultsWriter = resultsWriter; - return this; - } - - public int getNumRecordsPerFile() { - return numRecordsPerFile; - } - - public PcapMRJobConfig setNumRecordsPerFile(int numRecordsPerFile) { - this.numRecordsPerFile = numRecordsPerFile; - return this; - } - - public String getOutputFilePrefix() { - return outputFilePrefix; - } - - public PcapMRJobConfig setOutputFilePrefix(String outputFilePrefix) { - this.outputFilePrefix = outputFilePrefix; - return this; - } - -} diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/writer/PcapResultsWriter.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/writer/PcapResultsWriter.java index 3ca834f4c1..62ac27cfa2 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/writer/PcapResultsWriter.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/writer/PcapResultsWriter.java @@ -22,42 +22,19 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.metron.common.utils.HDFSUtils; -import org.apache.metron.job.writer.ResultsWriter; import org.apache.metron.pcap.PcapMerger; -public class PcapResultsWriter implements ResultsWriter { - - private Configuration config; - - /** - * Creates default hadoop configuration. - */ - public PcapResultsWriter() { - this.config = new Configuration(); - } - - /** - * Pass in hadoop config - * - * @param config Standard hadoop filesystem config. - */ - public PcapResultsWriter(Configuration config) { - this.config = config; - } - - public void withConfiguration(Configuration config) { - this.config = config; - } +public class PcapResultsWriter { /** * Write out pcaps. Configuration offers ability to configure for HDFS or local FS, if desired. * + * @param config Standard hadoop filesystem config. * @param pcaps pcap data to write. Pre-merged format as a list of pcaps as byte arrays. * @param outPath where to write the pcap data to. * @throws IOException I/O issue encountered. */ - @Override - public void write(List pcaps, String outPath) throws IOException { + public void write(Configuration config, List pcaps, String outPath) throws IOException { HDFSUtils.write(config, mergePcaps(pcaps), outPath); } @@ -68,7 +45,7 @@ public void write(List pcaps, String outPath) throws IOException { * @return merged result. * @throws IOException I/O issue encountered. */ - protected byte[] mergePcaps(List pcaps) throws IOException { + public byte[] mergePcaps(List pcaps) throws IOException { if (pcaps == null) { return new byte[]{}; } @@ -79,5 +56,4 @@ protected byte[] mergePcaps(List pcaps) throws IOException { PcapMerger.merge(baos, pcaps); return baos.toByteArray(); } - } From 6e0b64ce1c9b1c3ca1b7a6e2b58ba74c188d702a Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 12 Jul 2018 13:18:09 -0400 Subject: [PATCH 13/36] Refactoring the config options to translate between object and map. --- metron-interface/metron-rest-client/pom.xml | 5 ++ .../metron/rest/model/pcap/PcapRequest.java | 35 +++++----- .../apache/metron/pcap/query/CliConfig.java | 66 ++++++++++--------- .../metron/pcap/query/FixedCliConfig.java | 14 ++-- .../org/apache/metron/pcap/query/PcapCli.java | 52 ++++++--------- .../metron/pcap/query/QueryCliConfig.java | 7 +- .../org/apache/metron/pcap/ConfigOption.java | 24 +++++++ .../org/apache/metron/pcap/ConfigOptions.java | 46 +++++++++++++ .../org/apache/metron/pcap/mr/PcapJob.java | 25 +++---- 9 files changed, 171 insertions(+), 103 deletions(-) create mode 100644 metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java create mode 100644 metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java diff --git a/metron-interface/metron-rest-client/pom.xml b/metron-interface/metron-rest-client/pom.xml index 9ef9ae62db..a2f12882c4 100644 --- a/metron-interface/metron-rest-client/pom.xml +++ b/metron-interface/metron-rest-client/pom.xml @@ -42,6 +42,11 @@ metron-indexing ${project.parent.version} + + org.apache.metron + metron-pcap + ${project.parent.version} + org.eclipse.persistence javax.persistence diff --git a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java index 2648306d0c..623e4328d6 100644 --- a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java +++ b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java @@ -19,52 +19,55 @@ // TODO reconcile with pcapmrjob -public class PcapRequest { +import org.apache.commons.collections4.map.AbstractMapDecorator; +import org.apache.metron.pcap.ConfigOptions; - private String baseOutputPath; - private String basePath; - private Long startTime = 0L; - private Long endTime = System.currentTimeMillis(); - private Integer numReducers = 1; +public class PcapRequest extends AbstractMapDecorator { + + public PcapRequest() { + setStartTime(0L); + setEndTime(System.currentTimeMillis()); + setNumReducers(1); + } public String getBaseOutputPath() { - return baseOutputPath; + return ConfigOptions.INTERRIM_RESULT_PATH.get(this, String.class); } public void setBaseOutputPath(String baseOutputPath) { - this.baseOutputPath = baseOutputPath; + ConfigOptions.INTERRIM_RESULT_PATH.put(this, baseOutputPath); } public String getBasePath() { - return basePath; + return ConfigOptions.BASE_PATH.get(this, String.class); } public void setBasePath(String basePath) { - this.basePath = basePath; + ConfigOptions.BASE_PATH.put(this, basePath); } public Long getStartTime() { - return startTime; + return ConfigOptions.START_TIME.get(this, Long.class); } public void setStartTime(Long startTime) { - this.startTime = startTime; + ConfigOptions.START_TIME.put(this, startTime); } public Long getEndTime() { - return endTime; + return ConfigOptions.END_TIME.get(this, Long.class); } public void setEndTime(Long endTime) { - this.endTime = endTime; + ConfigOptions.END_TIME.put(this, endTime); } public Integer getNumReducers() { - return numReducers; + return ConfigOptions.NUM_REDUCERS.get(this, Integer.class); } public void setNumReducers(Integer numReducers) { - this.numReducers = numReducers; + ConfigOptions.NUM_REDUCERS.put(this, numReducers); } @Override diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliConfig.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliConfig.java index 1d8e3f3f62..21fdd623f1 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliConfig.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliConfig.java @@ -17,49 +17,53 @@ */ package org.apache.metron.pcap.query; +import org.apache.commons.collections4.map.AbstractMapDecorator; import org.apache.commons.lang3.StringUtils; import org.apache.metron.common.system.Clock; +import org.apache.metron.pcap.ConfigOption; +import org.apache.metron.pcap.ConfigOptions; import java.text.DateFormat; import java.text.SimpleDateFormat; -import java.util.UUID; -import java.util.function.Consumer; +import java.util.HashMap; import java.util.function.Function; -public class CliConfig { +public class CliConfig extends AbstractMapDecorator{ public interface PrefixStrategy extends Function{} private boolean showHelp; - private String prefix; - private String basePath; - private String baseOutputPath; - private long startTime; - private long endTime; - private int numReducers; - private int numRecordsPerFile; private DateFormat dateFormat; + public CliConfig() { + super(new HashMap<>()); + } public CliConfig(PrefixStrategy prefixStrategy) { - showHelp = false; - basePath = ""; - baseOutputPath = ""; - startTime = -1L; - endTime = -1L; - numReducers = 0; - prefix = prefixStrategy.apply(new Clock()); + this(); + setShowHelp(false); + setBasePath(""); + setBaseOutputPath(""); + setStartTime(-1L); + setEndTime(-1L); + setNumReducers(0); + setPrefix(prefixStrategy.apply(new Clock())); + } + + public Object getOption(ConfigOption option) { + Object o = get(option.getKey()); + return option.transform().apply(option.getKey(), o); } public String getPrefix() { - return prefix; + return ConfigOptions.PREFIX.get(this, String.class); } public void setPrefix(String prefix) { - this.prefix = prefix; + ConfigOptions.PREFIX.put(this, prefix); } public int getNumReducers() { - return numReducers; + return ConfigOptions.NUM_REDUCERS.get(this, Integer.class); } public boolean showHelp() { @@ -71,35 +75,35 @@ public void setShowHelp(boolean showHelp) { } public String getBasePath() { - return basePath; + return ConfigOptions.BASE_PATH.get(this, String.class); } public String getBaseOutputPath() { - return baseOutputPath; + return ConfigOptions.INTERRIM_RESULT_PATH.get(this, String.class); } public long getStartTime() { - return startTime; + return ConfigOptions.START_TIME.get(this, Long.class); } public long getEndTime() { - return endTime; + return ConfigOptions.END_TIME.get(this, Long.class); } public void setBasePath(String basePath) { - this.basePath = basePath; + ConfigOptions.BASE_PATH.put(this, basePath); } public void setBaseOutputPath(String baseOutputPath) { - this.baseOutputPath = baseOutputPath; + ConfigOptions.INTERRIM_RESULT_PATH.put(this, baseOutputPath); } public void setStartTime(long startTime) { - this.startTime = startTime; + ConfigOptions.START_TIME.put(this, startTime); } public void setEndTime(long endTime) { - this.endTime = endTime; + ConfigOptions.END_TIME.put(this, endTime); } public boolean isNullOrEmpty(String val) { @@ -115,14 +119,14 @@ public DateFormat getDateFormat() { } public void setNumReducers(int numReducers) { - this.numReducers = numReducers; + ConfigOptions.NUM_REDUCERS.put(this, numReducers); } public int getNumRecordsPerFile() { - return numRecordsPerFile; + return ConfigOptions.NUM_RECORDS_PER_FILE.get(this, Integer.class); } public void setNumRecordsPerFile(int numRecordsPerFile) { - this.numRecordsPerFile = numRecordsPerFile; + ConfigOptions.NUM_RECORDS_PER_FILE.put(this, numRecordsPerFile); } } diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/FixedCliConfig.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/FixedCliConfig.java index 03caed7157..d0deb48943 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/FixedCliConfig.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/FixedCliConfig.java @@ -17,33 +17,31 @@ */ package org.apache.metron.pcap.query; -import org.apache.metron.common.Constants; +import org.apache.metron.pcap.ConfigOptions; -import java.util.EnumMap; import java.util.LinkedHashMap; import java.util.Map; public class FixedCliConfig extends CliConfig { - private Map fixedFields; - public FixedCliConfig(PrefixStrategy prefixStrategy) { super(prefixStrategy); - this.fixedFields = new LinkedHashMap<>(); + setFixedFields(new LinkedHashMap<>()); } public Map getFixedFields() { - return fixedFields; + return ConfigOptions.FIELDS.get(this, Map.class); } public void setFixedFields(Map fixedFields) { - this.fixedFields = fixedFields; + ConfigOptions.FIELDS.put(this, fixedFields); } public void putFixedField(String key, String value) { + Map fixedFields = ConfigOptions.FIELDS.get(this, Map.class); String trimmedVal = value != null ? value.trim() : null; if (!isNullOrEmpty(trimmedVal)) { - this.fixedFields.put(key, value); + fixedFields.put(key, value); } } diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java index df997a4638..ad447215a6 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.Arrays; -import java.util.HashMap; import java.util.Map; import java.util.UUID; import org.apache.commons.cli.ParseException; @@ -33,8 +32,8 @@ import org.apache.metron.common.utils.timestamp.TimestampConverters; import org.apache.metron.job.JobException; import org.apache.metron.job.Pageable; +import org.apache.metron.pcap.ConfigOptions; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; -import org.apache.metron.pcap.filter.query.QueryPcapFilter; import org.apache.metron.pcap.finalizer.PcapFinalizerStrategies; import org.apache.metron.pcap.mr.PcapJob; import org.slf4j.Logger; @@ -60,6 +59,7 @@ public PcapCli(PcapJob jobRunner, CliConfig.PrefixStrategy prefixStrategy) { this.prefixStrategy = prefixStrategy; } + public int run(String[] args) { if (args.length < 1) { printBasicHelp(); @@ -77,17 +77,8 @@ public int run(String[] args) { } CliConfig commonConfig = null; Pageable results; - Map jobConfig = new HashMap<>(); - jobConfig.put("hadoopConf", hadoopConf); - try { - jobConfig.put("fileSystem", FileSystem.get(hadoopConf)); - } catch (IOException e) { - LOGGER.error("Failed to execute filter job: {}", e.getMessage(), e); - return -1; - } // write to local FS in the executing directory String execDir = System.getProperty("user.dir"); - jobConfig.put("finalOutputPath", new Path("file:///" + execDir)); if ("fixed".equals(jobType)) { FixedCliParser fixedParser = new FixedCliParser(prefixStrategy); @@ -95,6 +86,7 @@ public int run(String[] args) { try { config = fixedParser.parse(otherArgs); commonConfig = config; + ConfigOptions.FINAL_OUTPUT_PATH.put(commonConfig, new Path("file:///" + execDir)); } catch (ParseException | java.text.ParseException e) { System.err.println(e.getMessage()); System.err.flush(); @@ -109,19 +101,17 @@ public int run(String[] args) { long startTime = time.getLeft(); long endTime = time.getRight(); - jobConfig.put("basePath", new Path(config.getBasePath())); - jobConfig.put("interimResultPath", new Path(config.getBaseOutputPath())); - jobConfig.put("beginNS", startTime); - jobConfig.put("endNS", endTime); - jobConfig.put("numReducers", config.getNumReducers()); - jobConfig.put("fields", config.getFixedFields()); - jobConfig.put("filterImpl", new FixedPcapFilter.Configurator()); - jobConfig.put("numRecordsPerFile", commonConfig.getNumRecordsPerFile()); - jobConfig.put("finalFilenamePrefix", commonConfig.getPrefix()); + //TODO: Figure out how to do jobname here..it's missing. + //TODO: Figure out how to do final prefix filename here, it's missing, I think. + ConfigOptions.START_TIME_NS.put(commonConfig, startTime); + ConfigOptions.END_TIME_NS.put(commonConfig, endTime); + ConfigOptions.FILTER_IMPL.put(commonConfig, new FixedPcapFilter.Configurator()); + ConfigOptions.HADOOP_CONF.put(commonConfig, hadoopConf); try { - results = jobRunner.submit(PcapFinalizerStrategies.CLI, jobConfig).get(); - } catch (InterruptedException | JobException e) { + ConfigOptions.FILESYSTEM.put(commonConfig, FileSystem.get(hadoopConf)); + results = jobRunner.submit(PcapFinalizerStrategies.CLI, commonConfig).get(); + } catch (IOException|InterruptedException | JobException e) { LOGGER.error("Failed to execute fixed filter job: {}", e.getMessage(), e); return -1; } @@ -144,18 +134,14 @@ public int run(String[] args) { long startTime = time.getLeft(); long endTime = time.getRight(); - jobConfig.put("basePath", new Path(config.getBasePath())); - jobConfig.put("interimResultPath", new Path(config.getBaseOutputPath())); - jobConfig.put("beginNS", startTime); - jobConfig.put("endNS", endTime); - jobConfig.put("numReducers", config.getNumReducers()); - jobConfig.put("fields", config.getQuery()); - jobConfig.put("filterImpl", new QueryPcapFilter.Configurator()); - jobConfig.put("numRecordsPerFile", commonConfig.getNumRecordsPerFile()); - jobConfig.put("finalFilenamePrefix", commonConfig.getPrefix()); + ConfigOptions.START_TIME_NS.put(commonConfig, startTime); + ConfigOptions.END_TIME_NS.put(commonConfig, endTime); + ConfigOptions.FILTER_IMPL.put(commonConfig, new FixedPcapFilter.Configurator()); + ConfigOptions.HADOOP_CONF.put(commonConfig, hadoopConf); try { - results = jobRunner.submit(PcapFinalizerStrategies.CLI, jobConfig).get(); - } catch (InterruptedException | JobException e) { + ConfigOptions.FILESYSTEM.put(commonConfig, FileSystem.get(hadoopConf)); + results = jobRunner.submit(PcapFinalizerStrategies.CLI, commonConfig).get(); + } catch (IOException| InterruptedException | JobException e) { LOGGER.error("Failed to execute fixed filter job: {}", e.getMessage(), e); return -1; } diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/QueryCliConfig.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/QueryCliConfig.java index 67f045ff8d..f18de4ccfc 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/QueryCliConfig.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/QueryCliConfig.java @@ -17,18 +17,19 @@ */ package org.apache.metron.pcap.query; +import org.apache.metron.pcap.ConfigOptions; + public class QueryCliConfig extends CliConfig { - private String query; public QueryCliConfig(PrefixStrategy prefixStrategy) { super(prefixStrategy); } public String getQuery() { - return query; + return ConfigOptions.FIELDS.get(this, String.class); } public void setQuery(String query) { - this.query = query; + ConfigOptions.FIELDS.put(this, query); } } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java new file mode 100644 index 0000000000..e11f6eb61e --- /dev/null +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java @@ -0,0 +1,24 @@ +package org.apache.metron.pcap; + +import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Function; + +public interface ConfigOption { + String getKey(); + default BiFunction transform() { + return (s,o) -> o; + } + + default void put(Map map, Object value) { + map.put(getKey(), value); + } + + default T get(Map map, Class clazz) { + return clazz.cast(map.get(getKey())); + } + + default T getTransformed(Map map, Class clazz) { + return clazz.cast(transform().apply(getKey(), map.get(getKey()))); + } +} diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java new file mode 100644 index 0000000000..a5fd0132ed --- /dev/null +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java @@ -0,0 +1,46 @@ +package org.apache.metron.pcap; + +import org.apache.hadoop.fs.Path; + +import java.util.function.BiFunction; + +public enum ConfigOptions implements ConfigOption { + PREFIX("prefix"), + FINAL_FILENAME_PREFIX("finalFilenamePrefix"), + JOB_NAME("jobName"), + FINAL_OUTPUT_PATH("finalOutputPath"), + BASE_PATH("basePath", (s,o) -> o == null?null:new Path(o.toString())), + INTERRIM_RESULT_PATH("interimResultPath", (s,o) -> o == null?null:new Path(o.toString())), + NUM_REDUCERS("numReducers"), + START_TIME("startTime"), + END_TIME("endTime"), + START_TIME_NS("startNs"), + END_TIME_NS("endNs"), + NUM_RECORDS_PER_FILE("numRecordsPerFile"), + FIELDS("fields"), + FILTER_IMPL("filterImpl"), + HADOOP_CONF("hadoopConf"), + FILESYSTEM("fileSystem") + ; + String key; + BiFunction transform = (s, o) -> o; + + ConfigOptions(String key) { + this.key = key; + } + ConfigOptions(String key, BiFunction transform ) { + this.key = key; + this.transform = transform; + } + + + @Override + public String getKey() { + return key; + } + + @Override + public BiFunction transform() { + return transform; + } +} diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index fd8c5ed4f6..9122bcbea7 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -56,6 +56,7 @@ import org.apache.metron.job.JobStatus.State; import org.apache.metron.job.Pageable; import org.apache.metron.job.Statusable; +import org.apache.metron.pcap.ConfigOptions; import org.apache.metron.pcap.PacketInfo; import org.apache.metron.pcap.PcapHelper; import org.apache.metron.pcap.filter.PcapFilter; @@ -209,18 +210,18 @@ public Statusable submit(Finalizer finalizer, Map co throws JobException { this.finalizer = finalizer; this.configuration = configuration; - Optional jobName = Optional.ofNullable((String) configuration.get("jobName")); - Configuration hadoopConf = (Configuration) configuration.get("hadoopConf"); - FileSystem fileSystem = (FileSystem) configuration.get("fileSystem"); - Path basePath = (Path) configuration.get("basePath"); - Path baseInterimResultPath = (Path) configuration.get("baseInterimResultPath"); - long startTime = (long) configuration.get("beginNS"); - long endTime = (long) configuration.get("endNS"); - int numReducers = (int) configuration.get("numReducers"); - T fields = (T) configuration.get("fields"); - PcapFilterConfigurator filterImpl = (PcapFilterConfigurator) configuration.get("filterImpl"); - int numRecordsPerFile = (int) configuration.get("numRecordsPerFile"); - String finalFilenamePrefix = (String) configuration.get("finalFilenamePrefix"); + Optional jobName = Optional.ofNullable(ConfigOptions.JOB_NAME.get(configuration, String.class)); + Configuration hadoopConf = ConfigOptions.HADOOP_CONF.get(configuration, Configuration.class); + FileSystem fileSystem = ConfigOptions.FILESYSTEM.get(configuration, FileSystem.class); + Path basePath = ConfigOptions.BASE_PATH.getTransformed(configuration, Path.class); + Path baseInterimResultPath = ConfigOptions.INTERRIM_RESULT_PATH.getTransformed(configuration, Path.class); + long startTime = ConfigOptions.START_TIME_NS.get(configuration, Long.class); + long endTime = ConfigOptions.END_TIME_NS.get(configuration, Long.class); + int numReducers = ConfigOptions.NUM_REDUCERS.get(configuration, Integer.class); + T fields = (T) ConfigOptions.FIELDS.get(configuration, Object.class); + PcapFilterConfigurator filterImpl = (PcapFilterConfigurator) ConfigOptions.FILTER_IMPL.get(configuration, Object.class); + int numRecordsPerFile = ConfigOptions.NUM_RECORDS_PER_FILE.get(configuration, Integer.class); + String finalFilenamePrefix = ConfigOptions.FINAL_FILENAME_PREFIX.get(configuration, String.class); try { return query(jobName, From 0aa8e6eb967c6163c88b02c42074ced96243a7c3 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 12 Jul 2018 14:10:09 -0400 Subject: [PATCH 14/36] Redoing some stuff --- .../java/org/apache/metron/pcap/query/PcapCli.java | 3 +++ .../apache/metron/pcap/finalizer/PcapFinalizer.java | 12 +++++++----- .../main/java/org/apache/metron/pcap/mr/PcapJob.java | 2 +- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java index ad447215a6..6ed299b4db 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java @@ -134,6 +134,9 @@ public int run(String[] args) { long startTime = time.getLeft(); long endTime = time.getRight(); + //TODO: Figure out how to do jobname here..it's missing. + //TODO: Figure out how to do final prefix filename here, it's missing, I think. + ConfigOptions.START_TIME_NS.put(commonConfig, startTime); ConfigOptions.END_TIME_NS.put(commonConfig, endTime); ConfigOptions.FILTER_IMPL.put(commonConfig, new FixedPcapFilter.Configurator()); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java index fb8f9695d1..e083cbc578 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java @@ -40,6 +40,7 @@ import org.apache.metron.job.Finalizer; import org.apache.metron.job.JobException; import org.apache.metron.job.Pageable; +import org.apache.metron.pcap.ConfigOptions; import org.apache.metron.pcap.PcapFiles; import org.apache.metron.pcap.writer.PcapResultsWriter; import org.slf4j.Logger; @@ -47,6 +48,7 @@ public abstract class PcapFinalizer implements Finalizer { + //TODO: Move this into the config object public enum PcapFinalizerOptions { HADOOP_CONF("hadoopConf"), NUM_RECORDS_PER_FILE("numRecordsPerFile"), @@ -82,11 +84,11 @@ protected PcapResultsWriter getResultsWriter() { @Override public Pageable finalizeJob(Map config) throws JobException { - Configuration hadoopConfig = (Configuration) HADOOP_CONF.get(config); - int recPerFile = (int) NUM_RECORDS_PER_FILE.get(config); - Path interimResultPath = (Path) INTERIM_RESULTS_PATH.get(config); - Path finalOutputPath = (Path) FINAL_OUTPUT_PATH.get(config); - FileSystem fs = (FileSystem) FILE_SYSTEM.get(config); + Configuration hadoopConfig = ConfigOptions.HADOOP_CONF.get(config, Configuration.class); + int recPerFile = ConfigOptions.NUM_RECORDS_PER_FILE.get(config, Integer.class); + Path interimResultPath = ConfigOptions.INTERRIM_RESULT_PATH.getTransformed(config, Path.class); + Path finalOutputPath = ConfigOptions.FINAL_OUTPUT_PATH.getTransformed(config, Path.class); + FileSystem fs = ConfigOptions.FILESYSTEM.get(config, FileSystem.class); SequenceFileIterable interimResults = null; try { diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index 9122bcbea7..90efea9e41 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -219,7 +219,7 @@ public Statusable submit(Finalizer finalizer, Map co long endTime = ConfigOptions.END_TIME_NS.get(configuration, Long.class); int numReducers = ConfigOptions.NUM_REDUCERS.get(configuration, Integer.class); T fields = (T) ConfigOptions.FIELDS.get(configuration, Object.class); - PcapFilterConfigurator filterImpl = (PcapFilterConfigurator) ConfigOptions.FILTER_IMPL.get(configuration, Object.class); + PcapFilterConfigurator filterImpl = ConfigOptions.FILTER_IMPL.get(configuration, PcapFilterConfigurator.class); int numRecordsPerFile = ConfigOptions.NUM_RECORDS_PER_FILE.get(configuration, Integer.class); String finalFilenamePrefix = ConfigOptions.FINAL_FILENAME_PREFIX.get(configuration, String.class); From 9c6eaea48b7dc5b2cf1d6e7b7373258afbcda965 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 12 Jul 2018 14:15:38 -0400 Subject: [PATCH 15/36] Removed some duplication --- .../org/apache/metron/pcap/ConfigOption.java | 5 ++- .../org/apache/metron/pcap/ConfigOptions.java | 2 ++ .../pcap/finalizer/PcapCliFinalizer.java | 5 +-- .../metron/pcap/finalizer/PcapFinalizer.java | 33 ++----------------- .../pcap/finalizer/PcapRestFinalizer.java | 3 +- .../org/apache/metron/pcap/mr/PcapJob.java | 4 +-- 6 files changed, 14 insertions(+), 38 deletions(-) diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java index e11f6eb61e..031252ca7a 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java @@ -2,7 +2,6 @@ import java.util.Map; import java.util.function.BiFunction; -import java.util.function.Function; public interface ConfigOption { String getKey(); @@ -18,6 +17,10 @@ default T get(Map map, Class clazz) { return clazz.cast(map.get(getKey())); } + default T get(Map map, BiFunction transform, Class clazz) { + return clazz.cast(map.get(getKey())); + } + default T getTransformed(Map map, Class clazz) { return clazz.cast(transform().apply(getKey(), map.get(getKey()))); } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java index a5fd0132ed..8f456aeefa 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java @@ -22,6 +22,8 @@ public enum ConfigOptions implements ConfigOption { HADOOP_CONF("hadoopConf"), FILESYSTEM("fileSystem") ; + + public static final BiFunction STRING_TO_PATH = (s,o) -> o == null?null:new Path(o.toString()); String key; BiFunction transform = (s, o) -> o; diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java index 2e0b1de0be..1666618184 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java @@ -20,6 +20,7 @@ import java.util.Map; import org.apache.hadoop.fs.Path; +import org.apache.metron.pcap.ConfigOptions; /** * Write to local FS. @@ -28,8 +29,8 @@ public class PcapCliFinalizer extends PcapFinalizer { @Override protected String getOutputFileName(Map config, int partition) { - Path finalOutputPath = (Path) PcapFinalizerOptions.FINAL_OUTPUT_PATH.get(config); - String prefix = (String) config.get("finalFilenamePrefix"); + Path finalOutputPath = ConfigOptions.FINAL_OUTPUT_PATH.getTransformed(config, Path.class); + String prefix = ConfigOptions.FINAL_FILENAME_PREFIX.get(config, String.class); return String.format("%s/pcap-data-%s+%04d.pcap", finalOutputPath, prefix, partition); } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java index e083cbc578..03bcfa4e3e 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java @@ -18,12 +18,6 @@ package org.apache.metron.pcap.finalizer; -import static org.apache.metron.pcap.finalizer.PcapFinalizer.PcapFinalizerOptions.FILE_SYSTEM; -import static org.apache.metron.pcap.finalizer.PcapFinalizer.PcapFinalizerOptions.FINAL_OUTPUT_PATH; -import static org.apache.metron.pcap.finalizer.PcapFinalizer.PcapFinalizerOptions.HADOOP_CONF; -import static org.apache.metron.pcap.finalizer.PcapFinalizer.PcapFinalizerOptions.INTERIM_RESULTS_PATH; -import static org.apache.metron.pcap.finalizer.PcapFinalizer.PcapFinalizerOptions.NUM_RECORDS_PER_FILE; - import com.google.common.collect.Iterables; import java.io.IOException; import java.lang.invoke.MethodHandles; @@ -48,29 +42,6 @@ public abstract class PcapFinalizer implements Finalizer { - //TODO: Move this into the config object - public enum PcapFinalizerOptions { - HADOOP_CONF("hadoopConf"), - NUM_RECORDS_PER_FILE("numRecordsPerFile"), - INTERIM_RESULTS_PATH("interimResultsPath"), - FINAL_OUTPUT_PATH("finalOutputPath"), - FILE_SYSTEM("fileSystem"); - - private String optionName; - - PcapFinalizerOptions(String optionName) { - this.optionName = optionName; - } - - public boolean has(Map config) { - return config.containsKey(optionName); - } - - public Object get(Map config) { - return config.get(optionName); - } - } - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private PcapResultsWriter resultsWriter; @@ -86,8 +57,8 @@ protected PcapResultsWriter getResultsWriter() { public Pageable finalizeJob(Map config) throws JobException { Configuration hadoopConfig = ConfigOptions.HADOOP_CONF.get(config, Configuration.class); int recPerFile = ConfigOptions.NUM_RECORDS_PER_FILE.get(config, Integer.class); - Path interimResultPath = ConfigOptions.INTERRIM_RESULT_PATH.getTransformed(config, Path.class); - Path finalOutputPath = ConfigOptions.FINAL_OUTPUT_PATH.getTransformed(config, Path.class); + Path interimResultPath = ConfigOptions.INTERRIM_RESULT_PATH.get(config, ConfigOptions.STRING_TO_PATH, Path.class); + Path finalOutputPath = ConfigOptions.FINAL_OUTPUT_PATH.get(config, ConfigOptions.STRING_TO_PATH, Path.class); FileSystem fs = ConfigOptions.FILESYSTEM.get(config, FileSystem.class); SequenceFileIterable interimResults = null; diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java index 463120aea7..fa8511b845 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java @@ -20,6 +20,7 @@ import java.util.Map; import org.apache.hadoop.fs.Path; +import org.apache.metron.pcap.ConfigOptions; /** * Write to HDFS. @@ -28,7 +29,7 @@ public class PcapRestFinalizer extends PcapFinalizer { @Override protected String getOutputFileName(Map config, int partition) { - Path finalOutputPath = (Path) config.get("finalOutputPath"); + Path finalOutputPath = ConfigOptions.FINAL_OUTPUT_PATH.getTransformed(config, Path.class); return String.format("%s/page-%s", finalOutputPath, partition); } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index 90efea9e41..c3fe913506 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -199,7 +199,7 @@ public void setStatusInterval(long interval) { /** * Primarily for testing. * - * @param interval time in milllis + * @param interval time in millis */ public void setCompleteCheckInterval(long interval) { completeCheckInterval = interval; @@ -220,8 +220,6 @@ public Statusable submit(Finalizer finalizer, Map co int numReducers = ConfigOptions.NUM_REDUCERS.get(configuration, Integer.class); T fields = (T) ConfigOptions.FIELDS.get(configuration, Object.class); PcapFilterConfigurator filterImpl = ConfigOptions.FILTER_IMPL.get(configuration, PcapFilterConfigurator.class); - int numRecordsPerFile = ConfigOptions.NUM_RECORDS_PER_FILE.get(configuration, Integer.class); - String finalFilenamePrefix = ConfigOptions.FINAL_FILENAME_PREFIX.get(configuration, String.class); try { return query(jobName, From e52c5f00b9a720d7895180d36bca7285c0d0d335 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 12 Jul 2018 14:17:20 -0400 Subject: [PATCH 16/36] forgot licenses --- .../org/apache/metron/pcap/ConfigOption.java | 18 ++++++++++++++++++ .../org/apache/metron/pcap/ConfigOptions.java | 18 ++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java index 031252ca7a..e87c176719 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java @@ -1,3 +1,21 @@ +/** + * 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.metron.pcap; import java.util.Map; diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java index 8f456aeefa..96f9f7b083 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java @@ -1,3 +1,21 @@ +/** + * 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.metron.pcap; import org.apache.hadoop.fs.Path; From 7971ecba71b811b23187e70d8f41e9ec89c8da18 Mon Sep 17 00:00:00 2001 From: merrimanr Date: Thu, 12 Jul 2018 16:51:24 -0500 Subject: [PATCH 17/36] initial commit --- .../apache/metron/common/utils/HDFSUtils.java | 5 +- .../PcapTopologyIntegrationTest.java | 557 +++++++++++------- .../org/apache/metron/pcap/ConfigOptions.java | 1 + .../org/apache/metron/pcap/mr/PcapJob.java | 6 +- 4 files changed, 338 insertions(+), 231 deletions(-) diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java index 5bdd8d6c56..2e7ab498e8 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java @@ -30,7 +30,10 @@ public class HDFSUtils { public static byte[] readBytes(String path) throws IOException { - Path inPath = new Path(path); + return readBytes(new Path(path)); + } + + public static byte[] readBytes(Path inPath) throws IOException { FileSystem fs = FileSystem.get(inPath.toUri(), new Configuration()); FSDataInputStream inputStream = fs.open(inPath); return IOUtils.toByteArray(inputStream); diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java index c1cf6046a5..ed705b8082 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java @@ -23,24 +23,32 @@ import com.google.common.collect.Collections2; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; + +import java.io.ByteArrayOutputStream; import java.io.File; import java.io.FilenameFilter; import java.io.IOException; import java.util.AbstractMap; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import javax.annotation.Nullable; import kafka.consumer.ConsumerIterator; +import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.SequenceFile; import org.apache.kafka.clients.producer.Producer; +import org.apache.metron.common.Constants; +import org.apache.metron.common.utils.HDFSUtils; import org.apache.metron.integration.BaseIntegrationTest; import org.apache.metron.integration.ComponentRunner; import org.apache.metron.integration.Processor; @@ -51,8 +59,17 @@ import org.apache.metron.integration.components.MRComponent; import org.apache.metron.integration.components.ZKServerComponent; import org.apache.metron.integration.utils.KafkaUtil; +import org.apache.metron.job.Finalizer; +import org.apache.metron.job.JobStatus; +import org.apache.metron.job.Statusable; +import org.apache.metron.pcap.ConfigOptions; import org.apache.metron.pcap.PacketInfo; import org.apache.metron.pcap.PcapHelper; +import org.apache.metron.pcap.PcapMerger; +import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; +import org.apache.metron.pcap.filter.query.QueryPcapFilter; +import org.apache.metron.pcap.finalizer.PcapFinalizer; +import org.apache.metron.pcap.finalizer.PcapFinalizerStrategies; import org.apache.metron.pcap.mr.PcapJob; import org.apache.metron.spout.pcap.Endianness; import org.apache.metron.spout.pcap.deserializer.Deserializers; @@ -66,13 +83,22 @@ public class PcapTopologyIntegrationTest extends BaseIntegrationTest { final static String KAFKA_TOPIC = "pcap"; private static String BASE_DIR = "pcap"; private static String DATA_DIR = BASE_DIR + "/data_dir"; - private static String QUERY_DIR = BASE_DIR + "/query"; + private static String INTERIM_RESULT = BASE_DIR + "/query"; + private static String OUTPUT_DIR = BASE_DIR + "/output"; + private static final int MAX_RETRIES = 30; + private static final int SLEEP_MS = 500; private String topologiesDir = "src/main/flux"; private String targetDir = "target"; - private static void clearOutDir(File outDir) { - for(File f : outDir.listFiles()) { - f.delete(); + private static void clearOutDirs(File... dirs) throws IOException { + for(File dir: dirs) { + for(File f : dir.listFiles()) { + if (f.isDirectory()) { + FileUtils.deleteDirectory(f); + } else { + f.delete(); + } + } } } private static int numFiles(File outDir, Configuration config) { @@ -151,10 +177,10 @@ public void testTopology(Function updatePropertiesCallback topologiesDir = UnitTestHelper.findDir("topologies"); } targetDir = UnitTestHelper.findDir("target"); - final File outDir = getOutDir(targetDir); - final File queryDir = getQueryDir(targetDir); - clearOutDir(outDir); - clearOutDir(queryDir); + final File inputDir = getDir(targetDir, DATA_DIR); + final File interimResultDir = getDir(targetDir, INTERIM_RESULT); + final File outputDir = getDir(targetDir, OUTPUT_DIR); + clearOutDirs(inputDir, interimResultDir, outputDir); File baseDir = new File(new File(targetDir), BASE_DIR); //Assert.assertEquals(0, numFiles(outDir)); @@ -168,7 +194,7 @@ public void testTopology(Function updatePropertiesCallback setProperty("topology.worker.childopts", ""); setProperty("spout.kafka.topic.pcap", KAFKA_TOPIC); setProperty("kafka.pcap.start", "EARLIEST"); - setProperty("kafka.pcap.out", outDir.getAbsolutePath()); + setProperty("kafka.pcap.out", inputDir.getAbsolutePath()); setProperty("kafka.pcap.numPackets", "2"); setProperty("kafka.pcap.maxTimeMS", "200000000"); setProperty("kafka.pcap.ts_granularity", "NANOSECONDS"); @@ -212,7 +238,7 @@ public void testTopology(Function updatePropertiesCallback runner.process(new Processor() { @Override public ReadinessState process(ComponentRunner runner) { - int numFiles = numFiles(outDir, mr.getConfiguration()); + int numFiles = numFiles(inputDir, mr.getConfiguration()); int expectedNumFiles = pcapEntries.size() / 2; if (numFiles == expectedNumFiles) { return ReadinessState.READY; @@ -226,165 +252,223 @@ public ProcessorResult getResult() { return null; } }); - PcapJob job = new PcapJob(); - - // TODO - /* + Map configuration = new HashMap<>(); + Configuration hadoopConf = new Configuration(); + ConfigOptions.JOB_NAME.put(configuration, "jobName"); + ConfigOptions.HADOOP_CONF.put(configuration, hadoopConf); + ConfigOptions.FILESYSTEM.put(configuration, FileSystem.get(hadoopConf)); + ConfigOptions.BASE_PATH.put(configuration, new Path(inputDir.getAbsolutePath())); + ConfigOptions.BASE_INTERRIM_RESULT_PATH.put(configuration, new Path(interimResultDir.getAbsolutePath())); + ConfigOptions.START_TIME_NS.put(configuration, getTimestamp(4, pcapEntries)); + ConfigOptions.END_TIME_NS.put(configuration, getTimestamp(5, pcapEntries)); + ConfigOptions.NUM_REDUCERS.put(configuration, 10); + ConfigOptions.NUM_RECORDS_PER_FILE.put(configuration, 2); + ConfigOptions.FINAL_OUTPUT_PATH.put(configuration, new Path(outputDir.getAbsolutePath())); { //Ensure that only two pcaps are returned when we look at 4 and 5 - Iterable results = - job.query(new Path(outDir.getAbsolutePath()) - , new Path(queryDir.getAbsolutePath()) - , getTimestamp(4, pcapEntries) - , getTimestamp(5, pcapEntries) - , 10 - , new HashMap<>() - , new Configuration() - , FileSystem.get(new Configuration()) - , new FixedPcapFilter.Configurator() - ); - assertInOrder(results); - Assert.assertEquals(Iterables.size(results), 2); + ConfigOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); + ConfigOptions.FIELDS.put(configuration, new HashMap()); + PcapJob> job = new PcapJob<>(); + Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); + Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); + Assert.assertEquals(JobStatus.State.RUNNING, results.getStatus().getState()); + waitForJob(results); + + Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); + Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + try { + return HDFSUtils.readBytes(path); + } catch (IOException e) { + throw new IllegalStateException(e); + } + }); + assertInOrder(bytes); + Assert.assertEquals(results.get().getSize(), 1); } { // Ensure that only two pcaps are returned when we look at 4 and 5 // test with empty query filter - Iterable results = - job.query(new Path(outDir.getAbsolutePath()) - , new Path(queryDir.getAbsolutePath()) - , getTimestamp(4, pcapEntries) - , getTimestamp(5, pcapEntries) - , 10 - , "" - , new Configuration() - , FileSystem.get(new Configuration()) - , new QueryPcapFilter.Configurator() - ); - assertInOrder(results); - Assert.assertEquals(Iterables.size(results), 2); + ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + ConfigOptions.FIELDS.put(configuration, ""); + PcapJob job = new PcapJob<>(); + Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); + Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); + Assert.assertEquals(JobStatus.State.RUNNING, results.getStatus().getState()); + waitForJob(results); + + Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); + Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + try { + return HDFSUtils.readBytes(path); + } catch (IOException e) { + throw new IllegalStateException(e); + } + }); + assertInOrder(bytes); + Assert.assertEquals(results.get().getSize(), 1); } { //ensure that none get returned since that destination IP address isn't in the dataset - Iterable results = - job.query(new Path(outDir.getAbsolutePath()) - , new Path(queryDir.getAbsolutePath()) - , getTimestamp(0, pcapEntries) - , getTimestamp(1, pcapEntries) - , 10 - , new HashMap() {{ - put(Constants.Fields.DST_ADDR.getName(), "207.28.210.1"); - }} - , new Configuration() - , FileSystem.get(new Configuration()) - , new FixedPcapFilter.Configurator() - ); - assertInOrder(results); - Assert.assertEquals(Iterables.size(results), 0); + ConfigOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); + ConfigOptions.FIELDS.put(configuration, new HashMap() {{ + put(Constants.Fields.DST_ADDR.getName(), "207.28.210.1"); + }}); + PcapJob> job = new PcapJob<>(); + Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); + Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); + Assert.assertEquals(JobStatus.State.RUNNING, results.getStatus().getState()); + waitForJob(results); + + Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); + Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + try { + return HDFSUtils.readBytes(path); + } catch (IOException e) { + throw new IllegalStateException(e); + } + }); + assertInOrder(bytes); + Assert.assertEquals(results.get().getSize(), 0); } { // ensure that none get returned since that destination IP address isn't in the dataset // test with query filter - Iterable results = - job.query(new Path(outDir.getAbsolutePath()) - , new Path(queryDir.getAbsolutePath()) - , getTimestamp(0, pcapEntries) - , getTimestamp(1, pcapEntries) - , 10 - , "ip_dst_addr == '207.28.210.1'" - , new Configuration() - , FileSystem.get(new Configuration()) - , new QueryPcapFilter.Configurator() - ); - assertInOrder(results); - Assert.assertEquals(Iterables.size(results), 0); + ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + ConfigOptions.FIELDS.put(configuration, "ip_dst_addr == '207.28.210.1'"); + PcapJob job = new PcapJob<>(); + Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); + Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); + Assert.assertEquals(JobStatus.State.RUNNING, results.getStatus().getState()); + waitForJob(results); + + Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); + Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + try { + return HDFSUtils.readBytes(path); + } catch (IOException e) { + throw new IllegalStateException(e); + } + }); + assertInOrder(bytes); + Assert.assertEquals(results.get().getSize(), 0); } { //same with protocol as before with the destination addr - Iterable results = - job.query(new Path(outDir.getAbsolutePath()) - , new Path(queryDir.getAbsolutePath()) - , getTimestamp(0, pcapEntries) - , getTimestamp(1, pcapEntries) - , 10 - , new HashMap() {{ - put(Constants.Fields.PROTOCOL.getName(), "foo"); - }} - , new Configuration() - , FileSystem.get(new Configuration()) - , new FixedPcapFilter.Configurator() - ); - assertInOrder(results); - Assert.assertEquals(Iterables.size(results), 0); + ConfigOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); + ConfigOptions.FIELDS.put(configuration, new HashMap() {{ + put(Constants.Fields.PROTOCOL.getName(), "foo"); + }}); + PcapJob> job = new PcapJob<>(); + Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); + Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); + Assert.assertEquals(JobStatus.State.RUNNING, results.getStatus().getState()); + waitForJob(results); + + Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); + Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + try { + return HDFSUtils.readBytes(path); + } catch (IOException e) { + throw new IllegalStateException(e); + } + }); + assertInOrder(bytes); + Assert.assertEquals(results.get().getSize(), 0); } { //same with protocol as before with the destination addr //test with query filter - Iterable results = - job.query(new Path(outDir.getAbsolutePath()) - , new Path(queryDir.getAbsolutePath()) - , getTimestamp(0, pcapEntries) - , getTimestamp(1, pcapEntries) - , 10 - , "protocol == 'foo'" - , new Configuration() - , FileSystem.get(new Configuration()) - , new QueryPcapFilter.Configurator() - ); - assertInOrder(results); - Assert.assertEquals(Iterables.size(results), 0); + ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + ConfigOptions.FIELDS.put(configuration, "protocol == 'foo'"); + PcapJob job = new PcapJob<>(); + Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); + Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); + Assert.assertEquals(JobStatus.State.RUNNING, results.getStatus().getState()); + waitForJob(results); + + Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); + Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + try { + return HDFSUtils.readBytes(path); + } catch (IOException e) { + throw new IllegalStateException(e); + } + }); + assertInOrder(bytes); + Assert.assertEquals(results.get().getSize(), 0); } { //make sure I get them all. - Iterable results = - job.query(new Path(outDir.getAbsolutePath()) - , new Path(queryDir.getAbsolutePath()) - , getTimestamp(0, pcapEntries) - , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 - , 10 - , new HashMap<>() - , new Configuration() - , FileSystem.get(new Configuration()) - , new FixedPcapFilter.Configurator() - ); - assertInOrder(results); - Assert.assertEquals(Iterables.size(results), pcapEntries.size()); + ConfigOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); + ConfigOptions.FIELDS.put(configuration, new HashMap<>()); + ConfigOptions.START_TIME_NS.put(configuration, getTimestamp(0, pcapEntries)); + ConfigOptions.END_TIME_NS.put(configuration, getTimestamp(pcapEntries.size()-1, pcapEntries) + 1); + PcapJob> job = new PcapJob<>(); + Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); + Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); + Assert.assertEquals(JobStatus.State.RUNNING, results.getStatus().getState()); + waitForJob(results); + + Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); + Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + try { + return HDFSUtils.readBytes(path); + } catch (IOException e) { + throw new IllegalStateException(e); + } + }); + assertInOrder(bytes); + Assert.assertEquals(10, results.get().getSize()); } { //make sure I get them all. //with query filter - Iterable results = - job.query(new Path(outDir.getAbsolutePath()) - , new Path(queryDir.getAbsolutePath()) - , getTimestamp(0, pcapEntries) - , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 - , 10 - , "" - , new Configuration() - , FileSystem.get(new Configuration()) - , new QueryPcapFilter.Configurator() - ); - assertInOrder(results); - Assert.assertEquals(Iterables.size(results), pcapEntries.size()); + ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + ConfigOptions.FIELDS.put(configuration, ""); + ConfigOptions.START_TIME_NS.put(configuration, getTimestamp(0, pcapEntries)); + ConfigOptions.END_TIME_NS.put(configuration, getTimestamp(pcapEntries.size()-1, pcapEntries) + 1); + PcapJob job = new PcapJob<>(); + Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); + Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); + Assert.assertEquals(JobStatus.State.RUNNING, results.getStatus().getState()); + waitForJob(results); + + Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); + Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + try { + return HDFSUtils.readBytes(path); + } catch (IOException e) { + throw new IllegalStateException(e); + } + }); + assertInOrder(bytes); + Assert.assertEquals(10, results.get().getSize()); } { - Iterable results = - job.query(new Path(outDir.getAbsolutePath()) - , new Path(queryDir.getAbsolutePath()) - , getTimestamp(0, pcapEntries) - , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 - , 10 - , new HashMap() {{ - put(Constants.Fields.DST_PORT.getName(), "22"); - }} - , new Configuration() - , FileSystem.get(new Configuration()) - , new FixedPcapFilter.Configurator() - ); - assertInOrder(results); - Assert.assertTrue(Iterables.size(results) > 0); - Assert.assertEquals(Iterables.size(results) + ConfigOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); + ConfigOptions.FIELDS.put(configuration, new HashMap() {{ + put(Constants.Fields.DST_PORT.getName(), "22"); + }}); + ConfigOptions.NUM_RECORDS_PER_FILE.put(configuration, 1); + PcapJob> job = new PcapJob<>(); + Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); + Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); + Assert.assertEquals(JobStatus.State.RUNNING, results.getStatus().getState()); + waitForJob(results); + + Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); + Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + try { + return HDFSUtils.readBytes(path); + } catch (IOException e) { + throw new IllegalStateException(e); + } + }); + assertInOrder(bytes); + Assert.assertTrue(results.get().getSize() > 0); + Assert.assertEquals(Iterables.size(bytes) , Iterables.size(filterPcaps(pcapEntries, new Predicate() { @Override public boolean apply(@Nullable JSONObject input) { @@ -395,74 +479,63 @@ public boolean apply(@Nullable JSONObject input) { ) ); ByteArrayOutputStream baos = new ByteArrayOutputStream(); - PcapMerger.merge(baos, Iterables.partition(results, 1).iterator().next()); - Assert.assertTrue(baos.toByteArray().length > 0); - } - { - //test with query filter and byte array matching - Iterable results = - job.query(new Path(outDir.getAbsolutePath()) - , new Path(queryDir.getAbsolutePath()) - , getTimestamp(0, pcapEntries) - , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 - , 10 - , "BYTEARRAY_MATCHER('2f56abd814bc56420489ca38e7faf8cec3d4', packet)" - , new Configuration() - , FileSystem.get(new Configuration()) - , new QueryPcapFilter.Configurator() - ); - assertInOrder(results); - Assert.assertEquals(1, Iterables.size(results)); - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - PcapMerger.merge(baos, Iterables.partition(results, 1).iterator().next()); + PcapMerger.merge(baos, HDFSUtils.readBytes(results.get().getPage(0))); Assert.assertTrue(baos.toByteArray().length > 0); } { + //same with protocol as before with the destination addr //test with query filter - Iterable results = - job.query(new Path(outDir.getAbsolutePath()) - , new Path(queryDir.getAbsolutePath()) - , getTimestamp(0, pcapEntries) - , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 - , 10 - , "ip_dst_port == 22" - , new Configuration() - , FileSystem.get(new Configuration()) - , new QueryPcapFilter.Configurator() - ); - assertInOrder(results); - Assert.assertTrue(Iterables.size(results) > 0); - Assert.assertEquals(Iterables.size(results) + ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + ConfigOptions.FIELDS.put(configuration, "ip_dst_port == 22"); + PcapJob job = new PcapJob<>(); + Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); + Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); + Assert.assertEquals(JobStatus.State.RUNNING, results.getStatus().getState()); + waitForJob(results); + + Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); + Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + try { + return HDFSUtils.readBytes(path); + } catch (IOException e) { + throw new IllegalStateException(e); + } + }); + assertInOrder(bytes); + Assert.assertEquals(Iterables.size(bytes) , Iterables.size(filterPcaps(pcapEntries, new Predicate() { @Override public boolean apply(@Nullable JSONObject input) { Object prt = input.get(Constants.Fields.DST_PORT.getName()); - return prt != null && (Long) prt == 22; + return prt != null && prt.toString().equals("22"); } }, withHeaders) ) ); - assertInOrder(results); ByteArrayOutputStream baos = new ByteArrayOutputStream(); - PcapMerger.merge(baos, Iterables.partition(results, 1).iterator().next()); + PcapMerger.merge(baos, HDFSUtils.readBytes(results.get().getPage(0))); Assert.assertTrue(baos.toByteArray().length > 0); } { - //test with query filter - Iterable results = - job.query(new Path(outDir.getAbsolutePath()) - , new Path(queryDir.getAbsolutePath()) - , getTimestamp(0, pcapEntries) - , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 - , 10 - , "ip_dst_port > 20 and ip_dst_port < 55792" - , new Configuration() - , FileSystem.get(new Configuration()) - , new QueryPcapFilter.Configurator() - ); - assertInOrder(results); - Assert.assertTrue(Iterables.size(results) > 0); - Assert.assertEquals(Iterables.size(results) + // test with query filter ip_dst_port > 20 and ip_dst_port < 55792 + ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + ConfigOptions.FIELDS.put(configuration, "ip_dst_port > 20 and ip_dst_port < 55792"); + PcapJob job = new PcapJob<>(); + Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); + Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); + Assert.assertEquals(JobStatus.State.RUNNING, results.getStatus().getState()); + waitForJob(results); + + Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); + Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + try { + return HDFSUtils.readBytes(path); + } catch (IOException e) { + throw new IllegalStateException(e); + } + }); + assertInOrder(bytes); + Assert.assertEquals(Iterables.size(bytes) , Iterables.size(filterPcaps(pcapEntries, new Predicate() { @Override public boolean apply(@Nullable JSONObject input) { @@ -472,64 +545,92 @@ public boolean apply(@Nullable JSONObject input) { }, withHeaders) ) ); - assertInOrder(results); ByteArrayOutputStream baos = new ByteArrayOutputStream(); - PcapMerger.merge(baos, Iterables.partition(results, 1).iterator().next()); + PcapMerger.merge(baos, HDFSUtils.readBytes(results.get().getPage(0))); Assert.assertTrue(baos.toByteArray().length > 0); } { - //test with query filter - Iterable results = - job.query(new Path(outDir.getAbsolutePath()) - , new Path(queryDir.getAbsolutePath()) - , getTimestamp(0, pcapEntries) - , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1 - , 10 - , "ip_dst_port > 55790" - , new Configuration() - , FileSystem.get(new Configuration()) - , new QueryPcapFilter.Configurator() - ); - assertInOrder(results); - Assert.assertTrue(Iterables.size(results) > 0); - Assert.assertEquals(Iterables.size(results) + //test with query filter ip_dst_port > 55790 + ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + ConfigOptions.FIELDS.put(configuration, "ip_dst_port > 55790"); + PcapJob job = new PcapJob<>(); + Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); + Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); + Assert.assertEquals(JobStatus.State.RUNNING, results.getStatus().getState()); + waitForJob(results); + + Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); + Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + try { + return HDFSUtils.readBytes(path); + } catch (IOException e) { + throw new IllegalStateException(e); + } + }); + assertInOrder(bytes); + Assert.assertEquals(Iterables.size(bytes) , Iterables.size(filterPcaps(pcapEntries, new Predicate() { - @Override - public boolean apply(@Nullable JSONObject input) { - Object prt = input.get(Constants.Fields.DST_PORT.getName()); - return prt != null && (Long) prt > 55790; - } - }, withHeaders) + @Override + public boolean apply(@Nullable JSONObject input) { + Object prt = input.get(Constants.Fields.DST_PORT.getName()); + return prt != null && (Long) prt > 55790; + } + }, withHeaders) ) ); - assertInOrder(results); ByteArrayOutputStream baos = new ByteArrayOutputStream(); - PcapMerger.merge(baos, Iterables.partition(results, 1).iterator().next()); + PcapMerger.merge(baos, HDFSUtils.readBytes(results.get().getPage(0))); + Assert.assertTrue(baos.toByteArray().length > 0); + } + { + //test with query filter and byte array matching + ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + ConfigOptions.FIELDS.put(configuration, "BYTEARRAY_MATCHER('2f56abd814bc56420489ca38e7faf8cec3d4', packet)"); + ConfigOptions.START_TIME_NS.put(configuration, getTimestamp(0, pcapEntries)); + ConfigOptions.END_TIME_NS.put(configuration, getTimestamp(pcapEntries.size()-1, pcapEntries) + 1); + PcapJob job = new PcapJob<>(); + Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); + Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); + Assert.assertEquals(JobStatus.State.RUNNING, results.getStatus().getState()); + waitForJob(results); + + Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); + Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + try { + return HDFSUtils.readBytes(path); + } catch (IOException e) { + throw new IllegalStateException(e); + } + }); + assertInOrder(bytes); + Assert.assertEquals(1, results.get().getSize()); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + PcapMerger.merge(baos, HDFSUtils.readBytes(results.get().getPage(0))); Assert.assertTrue(baos.toByteArray().length > 0); } + System.out.println("Ended"); - */ } finally { runner.stop(); - clearOutDir(outDir); - clearOutDir(queryDir); + clearOutDirs(inputDir, interimResultDir, outputDir); } } - private File getOutDir(String targetDir) { - File outDir = new File(new File(targetDir), DATA_DIR); - if (!outDir.exists()) { - outDir.mkdirs(); + private void waitForJob(Statusable statusable) throws Exception { + for (int t = 0; t < MAX_RETRIES; ++t, Thread.sleep(SLEEP_MS)) { + if (statusable.isDone()) { + return; + } } - return outDir; + throw new Exception("Job did not complete within " + (MAX_RETRIES * SLEEP_MS) + " seconds"); } - private File getQueryDir(String targetDir) { - File outDir = new File(new File(targetDir), QUERY_DIR); - if (!outDir.exists()) { - outDir.mkdirs(); + private File getDir(String targetDir, String childDir) { + File directory = new File(new File(targetDir), childDir); + if (!directory.exists()) { + directory.mkdirs(); } - return outDir; + return directory; } private static Iterable> readPcaps(Path pcapFile, boolean withHeaders) throws IOException { diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java index 96f9f7b083..81657e1adf 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java @@ -28,6 +28,7 @@ public enum ConfigOptions implements ConfigOption { JOB_NAME("jobName"), FINAL_OUTPUT_PATH("finalOutputPath"), BASE_PATH("basePath", (s,o) -> o == null?null:new Path(o.toString())), + BASE_INTERRIM_RESULT_PATH("baseInterimResultPath", (s,o) -> o == null?null:new Path(o.toString())), INTERRIM_RESULT_PATH("interimResultPath", (s,o) -> o == null?null:new Path(o.toString())), NUM_REDUCERS("numReducers"), START_TIME("startTime"), diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index c3fe913506..76d858f08f 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -214,7 +214,7 @@ public Statusable submit(Finalizer finalizer, Map co Configuration hadoopConf = ConfigOptions.HADOOP_CONF.get(configuration, Configuration.class); FileSystem fileSystem = ConfigOptions.FILESYSTEM.get(configuration, FileSystem.class); Path basePath = ConfigOptions.BASE_PATH.getTransformed(configuration, Path.class); - Path baseInterimResultPath = ConfigOptions.INTERRIM_RESULT_PATH.getTransformed(configuration, Path.class); + Path baseInterimResultPath = ConfigOptions.BASE_INTERRIM_RESULT_PATH.getTransformed(configuration, Path.class); long startTime = ConfigOptions.START_TIME_NS.get(configuration, Long.class); long endTime = ConfigOptions.END_TIME_NS.get(configuration, Long.class); int numReducers = ConfigOptions.NUM_REDUCERS.get(configuration, Integer.class); @@ -229,7 +229,8 @@ public Statusable submit(Finalizer finalizer, Map co endTime, numReducers, fields, - hadoopConf, + // create a new copy for each job, bad things happen when hadoop config is reused + new Configuration(hadoopConf), fileSystem, filterImpl); } catch (IOException | InterruptedException | ClassNotFoundException e) { @@ -261,6 +262,7 @@ public Statusable query(Optional jobName, LOG.debug("Executing query {} on timerange from {} to {}", filterImpl.queryToString(fields), from, to); } Path interimResultPath = new Path(baseInterimResultPath, outputDirName); + ConfigOptions.INTERRIM_RESULT_PATH.put(configuration, interimResultPath); job = createJob(jobName , basePath , interimResultPath From 720dab608ffcf3cb40cabc79ec69e98541b709ee Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Fri, 13 Jul 2018 13:10:49 -0600 Subject: [PATCH 18/36] Move pcap config to metron-pcap module. Remove metron-api module completely --- .../metron/rest/model/pcap/PcapRequest.java | 22 +- metron-platform/metron-api/README.md | 65 -- metron-platform/metron-api/pom.xml | 269 -------- .../metron/api/ConfigurationManager.java | 135 ---- .../api/helper/service/PcapServiceCli.java | 169 ----- .../metron/pcapservice/ConfigurationUtil.java | 64 -- .../pcapservice/PcapReceiverImplRestEasy.java | 299 --------- .../metron/pcapservice/PcapsResponse.java | 118 ---- .../metron/pcapservice/RestTestingUtil.java | 329 ---------- .../pcapservice/rest/JettyServiceRunner.java | 44 -- .../metron/pcapservice/rest/PcapService.java | 56 -- .../src/main/resources/META-INF/LICENSE | 605 ------------------ .../src/main/resources/META-INF/NOTICE | 39 -- .../pcapservice/ConfigurationUtilTest.java | 43 -- .../PcapReceiverImplRestEasyTest.java | 255 -------- .../src/test/resources/log4j.properties | 24 - .../src/test/resources/test-tcp-packet.pcap | Bin 144 -> 0 bytes .../common/configuration}/ConfigOption.java | 2 +- .../java/org/apache/metron/job/JobStatus.java | 12 +- .../org/apache/metron/job/Statusable.java | 2 +- .../apache/metron/job/manager/JobManager.java | 3 + .../job/manager/InMemoryJobManagerTest.java | 28 + .../apache/metron/pcap/query/CliParser.java | 19 +- .../metron/pcap/query/FixedCliParser.java | 10 +- .../org/apache/metron/pcap/query/PcapCli.java | 53 +- .../metron/pcap/query/QueryCliParser.java | 10 +- .../org/apache/metron/pcap/PcapJobTest.java | 31 +- .../org/apache/metron/pcap/PcapFiles.java | 2 +- .../{ConfigOptions.java => PcapOptions.java} | 39 +- .../metron/pcap/config/FixedPcapConfig.java} | 14 +- .../metron/pcap/config/PcapConfig.java} | 64 +- .../metron/pcap/config/QueryPcapConfig.java} | 12 +- .../pcap/finalizer/PcapCliFinalizer.java | 6 +- .../metron/pcap/finalizer/PcapFinalizer.java | 12 +- .../pcap/finalizer/PcapRestFinalizer.java | 4 +- .../org/apache/metron/pcap/mr/PcapJob.java | 163 +++-- metron-platform/pom.xml | 1 - 37 files changed, 280 insertions(+), 2743 deletions(-) delete mode 100644 metron-platform/metron-api/README.md delete mode 100644 metron-platform/metron-api/pom.xml delete mode 100644 metron-platform/metron-api/src/main/java/org/apache/metron/api/ConfigurationManager.java delete mode 100644 metron-platform/metron-api/src/main/java/org/apache/metron/api/helper/service/PcapServiceCli.java delete mode 100644 metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/ConfigurationUtil.java delete mode 100644 metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/PcapReceiverImplRestEasy.java delete mode 100644 metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/PcapsResponse.java delete mode 100644 metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/RestTestingUtil.java delete mode 100644 metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/rest/JettyServiceRunner.java delete mode 100644 metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/rest/PcapService.java delete mode 100644 metron-platform/metron-api/src/main/resources/META-INF/LICENSE delete mode 100644 metron-platform/metron-api/src/main/resources/META-INF/NOTICE delete mode 100644 metron-platform/metron-api/src/test/java/org/apache/metron/pcapservice/ConfigurationUtilTest.java delete mode 100644 metron-platform/metron-api/src/test/java/org/apache/metron/pcapservice/PcapReceiverImplRestEasyTest.java delete mode 100644 metron-platform/metron-api/src/test/resources/log4j.properties delete mode 100644 metron-platform/metron-api/src/test/resources/test-tcp-packet.pcap rename metron-platform/{metron-pcap/src/main/java/org/apache/metron/pcap => metron-common/src/main/java/org/apache/metron/common/configuration}/ConfigOption.java (96%) rename metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/{ConfigOptions.java => PcapOptions.java} (64%) rename metron-platform/{metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/FixedCliConfig.java => metron-pcap/src/main/java/org/apache/metron/pcap/config/FixedPcapConfig.java} (77%) rename metron-platform/{metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliConfig.java => metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java} (57%) rename metron-platform/{metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/QueryCliConfig.java => metron-pcap/src/main/java/org/apache/metron/pcap/config/QueryPcapConfig.java} (76%) diff --git a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java index 623e4328d6..b65bb1b91f 100644 --- a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java +++ b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java @@ -20,7 +20,7 @@ // TODO reconcile with pcapmrjob import org.apache.commons.collections4.map.AbstractMapDecorator; -import org.apache.metron.pcap.ConfigOptions; +import org.apache.metron.pcap.PcapOptions; public class PcapRequest extends AbstractMapDecorator { @@ -31,43 +31,43 @@ public PcapRequest() { } public String getBaseOutputPath() { - return ConfigOptions.INTERRIM_RESULT_PATH.get(this, String.class); + return PcapOptions.INTERIM_RESULT_PATH.get(this, String.class); } public void setBaseOutputPath(String baseOutputPath) { - ConfigOptions.INTERRIM_RESULT_PATH.put(this, baseOutputPath); + PcapOptions.INTERIM_RESULT_PATH.put(this, baseOutputPath); } public String getBasePath() { - return ConfigOptions.BASE_PATH.get(this, String.class); + return PcapOptions.BASE_PATH.get(this, String.class); } public void setBasePath(String basePath) { - ConfigOptions.BASE_PATH.put(this, basePath); + PcapOptions.BASE_PATH.put(this, basePath); } public Long getStartTime() { - return ConfigOptions.START_TIME.get(this, Long.class); + return PcapOptions.START_TIME_MS.get(this, Long.class); } public void setStartTime(Long startTime) { - ConfigOptions.START_TIME.put(this, startTime); + PcapOptions.START_TIME_MS.put(this, startTime); } public Long getEndTime() { - return ConfigOptions.END_TIME.get(this, Long.class); + return PcapOptions.END_TIME_MS.get(this, Long.class); } public void setEndTime(Long endTime) { - ConfigOptions.END_TIME.put(this, endTime); + PcapOptions.END_TIME_MS.put(this, endTime); } public Integer getNumReducers() { - return ConfigOptions.NUM_REDUCERS.get(this, Integer.class); + return PcapOptions.NUM_REDUCERS.get(this, Integer.class); } public void setNumReducers(Integer numReducers) { - ConfigOptions.NUM_REDUCERS.put(this, numReducers); + PcapOptions.NUM_REDUCERS.put(this, numReducers); } @Override diff --git a/metron-platform/metron-api/README.md b/metron-platform/metron-api/README.md deleted file mode 100644 index 4a0dde8298..0000000000 --- a/metron-platform/metron-api/README.md +++ /dev/null @@ -1,65 +0,0 @@ - -# Metron PCAP Service - -The purpose of the Metron PCAP service is to provide a middle tier to -negotiate retrieving packet capture data which flows into Metron. This -packet data is of a form which `libpcap` based tools can read. - -## Starting the Service - -You can start the service either via the init.d script installed, -`/etc/init.d/pcapservice` or directly via the `yarn jar` command: -`yarn jar $METRON_HOME/lib/metron-api-$METRON_VERSION.jar org.apache.metron.pcapservice.rest.PcapService -port $SERVICE_PORT -query_hdfs_path $QUERY_PATH -pcap_hdfs_path $PCAP_PATH` - -where -* `METRON_HOME` is the location of the metron installation -* `METRON_VERSION` is the version of the metron installation -* `SERVICE_PORT` is the port to bind the REST service to. -* `QUERY_PATH` is the temporary location to store query results. They are deleted after the service reads them. -* `PCAP_PATH` is the path to the packet data on HDFS - -## The `/pcapGetter/getPcapsByIdentifiers` endpoint - -This endpoint takes the following query parameters and returns the subset of -packets matching this query: -* `srcIp` : The source IP to match on -* `srcPort` : The source port to match on -* `dstIp` : The destination IP to match on -* `dstPort` : The destination port to match on -* `startTime` : The start time in milliseconds -* `endTime` : The end time in milliseconds -* `numReducers` : Specify the number of reducers to use when executing the mapreduce job -* `includeReverseTraffic` : Indicates if filter should check swapped src/dest addresses and IPs - -## The `/pcapGetter/getPcapsByQuery` endpoint - -This endpoint takes the following query parameters and returns the subset of -packets matching this query. This endpoint exposes Stellar querying capabilities: -* `query` : The Stellar query to execute -* `startTime` : The start time in milliseconds -* `endTime` : The end time in milliseconds -* `numReducers` : Specify the number of reducers to use when executing the mapreduce job - -Example: -`curl -XGET "http://node1:8081/pcapGetter/getPcapsByQuery?query=ip_src_addr+==+'192.168.66.121'+and+ip_src_port+==+'60500'&startTime=1476936000000"` - -All of these parameters are optional. In the case of a missing -parameter, it is treated as a wildcard. - -Unlike the CLI tool, there is no paging mechanism. The REST API will stream back data as a single file. diff --git a/metron-platform/metron-api/pom.xml b/metron-platform/metron-api/pom.xml deleted file mode 100644 index 22b953c29a..0000000000 --- a/metron-platform/metron-api/pom.xml +++ /dev/null @@ -1,269 +0,0 @@ - - - 4.0.0 - - org.apache.metron - metron-platform - 0.5.1 - - metron-api - metron-api - Metron API - https://metron.apache.org/ - - UTF-8 - ${global_flume_version} - ${global_hadoop_version} - ${global_slf4j_version} - 3.4.5.2.0.6.0-76 - 1.2.15 - - 3.0.0.RELEASE - 3.2.6.RELEASE - 1.2.2 - 2.4 - 1.10 - 2.6 - 3.2.1 - 1.8.3 - 2.1.1 - ${global_junit_version} - 1.3 - 1.9.5 - 1.3.0 - - - - org.jboss.resteasy - jaxrs-api - 3.0.4.Final - - - org.apache.metron - metron-common - ${project.parent.version} - - - org.apache.metron - metron-pcap - ${project.parent.version} - - - org.slf4j - slf4j-log4j12 - - - - - - commons-beanutils - commons-beanutils - ${commons-beanutils.version} - - - org.apache.commons - commons-jexl - ${commons-jexl.version} - - - - commons-configuration - commons-configuration - ${commons-configuration.version} - - - org.slf4j - slf4j-api - - - - - junit - junit - ${junit.version} - test - - - org.powermock - powermock-api-mockito - 1.5 - test - - - org.powermock - powermock-core - 1.5 - test - - - org.powermock - powermock-module-junit4 - 1.5 - test - - - joda-time - joda-time - 2.3 - - - com.google.guava - guava - ${global_hbase_guava_version} - - - org.apache.hadoop - hadoop-common - ${global_hadoop_version} - - - org.slf4j - slf4j-log4j12 - - - - - org.apache.hadoop - hadoop-hdfs - ${global_hadoop_version} - - - org.apache.hadoop - hadoop-client - ${global_hadoop_version} - - - org.slf4j - slf4j-log4j12 - - - - - org.springframework.integration - spring-integration-http - ${spring.integration.version} - - - org.springframework - spring-webmvc - ${spring.version} - - - org.jboss.resteasy - resteasy-jaxrs - 3.0.1.Final - - - org.slf4j - slf4j-simple - - - - - org.jboss.resteasy - resteasy-jaxb-provider - 3.0.1.Final - compile - - - org.jboss.resteasy - async-http-servlet-3.0 - 3.0.1.Final - compile - - - org.eclipse.jetty - jetty-server - 9.3.0.M0 - - - org.eclipse.jetty - jetty-servlet - 9.3.0.M0 - - - - - - - org.apache.maven.plugins - maven-shade-plugin - ${global_shade_version} - - true - - - *slf4j* - - - - - - package - - shade - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - com.google.common - org.apache.metron.guava - - - org.apache.commons.configuration - org.apache.metron.commons.configuration - - - - - - .yaml - LICENSE.txt - ASL2.0 - NOTICE.txt - - - - - - - - - - - - - - - - - diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/api/ConfigurationManager.java b/metron-platform/metron-api/src/main/java/org/apache/metron/api/ConfigurationManager.java deleted file mode 100644 index 7297b7eca9..0000000000 --- a/metron-platform/metron-api/src/main/java/org/apache/metron/api/ConfigurationManager.java +++ /dev/null @@ -1,135 +0,0 @@ -/** - * 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.metron.api; - - -import java.io.File; -import java.lang.invoke.MethodHandles; -import java.util.HashMap; -import java.util.Map; -import org.apache.commons.configuration.CombinedConfiguration; -import org.apache.commons.configuration.Configuration; -import org.apache.commons.configuration.ConfigurationException; -import org.apache.commons.configuration.ConfigurationRuntimeException; -import org.apache.commons.configuration.DefaultConfigurationBuilder; -import org.apache.commons.lang.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Configuration manager class which loads all 'config-definition.xml' files and - * creates a Configuration object which holds all properties from the underlying - * configuration resource - */ -public class ConfigurationManager { - private static final Logger LOGGER = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - /** configuration definition file name. */ - private static String DEFAULT_CONFIG_DEFINITION_FILE_NAME = "config-definition.xml"; - - /** Stores a map with the configuration for each path specified. */ - private static Map configurationsCache = new HashMap(); - - - /** - * Common method to load content of all configuration resources defined in - * 'config-definition.xml'. - * - * @param configDefFilePath - * the config def file path - * @return Configuration - */ - public static Configuration getConfiguration(String configDefFilePath) { - if (configurationsCache.containsKey(configDefFilePath)) { - return configurationsCache.get(configDefFilePath); - } - CombinedConfiguration configuration = null; - synchronized (configurationsCache) { - if (configurationsCache.containsKey(configDefFilePath)) { - return configurationsCache.get(configDefFilePath); - } - DefaultConfigurationBuilder builder = new DefaultConfigurationBuilder(); - String filePath = getConfigDefFilePath(configDefFilePath); - LOGGER.info("loading from 'configDefFilePath' : {}", filePath); - builder.setFile(new File(filePath)); - try { - configuration = builder.getConfiguration(true); - configurationsCache.put(filePath, configuration); - } catch (ConfigurationException|ConfigurationRuntimeException e) { - LOGGER.info("Exception in loading property files.", e); - } - } - return configuration; - } - - /** - * Removes the configuration created from a config definition file located at - * 'configDefFilePath'. - * - * @param configDefFilePath - * path to the config definition file - */ - public static void clearConfiguration(String configDefFilePath) { - configurationsCache.remove(configDefFilePath); - } - - /** - * Gets the configuration. - * - * @return the configuration - */ - public static Configuration getConfiguration() { - return getConfiguration(null); - } - - /** - * Returns the 'config-definition.xml' file path. 1. If the param - * 'configDefFilePath' has a valid value, returns configDefFilePath 2. If the - * system property key 'configDefFilePath' has a valid value, returns the - * value 3. By default, it returns the file name 'config-definition.xml' - * - * @param configDefFilePath - * given input path to the config definition file - * @return the config def file path - */ - private static String getConfigDefFilePath(String configDefFilePath) { - if (StringUtils.isNotEmpty(configDefFilePath)) { - return configDefFilePath; - } - return DEFAULT_CONFIG_DEFINITION_FILE_NAME; - } - - /** - * The main method. - * - * @param args - * the args - * @throws InterruptedException - * the interrupted exception - */ - public static void main(String[] args) throws InterruptedException { - Configuration config = ConfigurationManager - .getConfiguration("/Users/Sayi/Documents/config/config-definition-dpi.xml"); - System.out.println("elastic.search.cluster =" - + config.getString("elastic.search.cluster")); - Thread.sleep(10000); - System.out.println("storm.topology.dpi.bolt.es-index.index.name =" - + config.getString("storm.topology.dpi.bolt.es-index.index.name")); - } - -} diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/api/helper/service/PcapServiceCli.java b/metron-platform/metron-api/src/main/java/org/apache/metron/api/helper/service/PcapServiceCli.java deleted file mode 100644 index 09b8c7ad28..0000000000 --- a/metron-platform/metron-api/src/main/java/org/apache/metron/api/helper/service/PcapServiceCli.java +++ /dev/null @@ -1,169 +0,0 @@ -/** - * 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.metron.api.helper.service; - -import org.apache.commons.cli.BasicParser; -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.CommandLineParser; -import org.apache.commons.cli.HelpFormatter; -import org.apache.commons.cli.Option; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.ParseException; -import org.apache.commons.configuration.ConfigurationUtils; -import org.apache.log4j.PropertyConfigurator; - -public class PcapServiceCli { - - private String[] args = null; - private Options options = new Options(); - - int port = 8081; - String uri = "/pcapGetter"; - String pcapHdfsPath= "/apps/metron/pcap"; - String queryHdfsPath = "/apps/metron/pcap_query"; - public int getPort() { - return port; - } - - public void setPort(int port) { - this.port = port; - } - - public String getUri() { - return uri; - } - - public void setUri(String uri) { - this.uri = uri; - } - - public String getPcapHdfsPath() { - return pcapHdfsPath; - } - - public String getQueryHdfsPath() { - return queryHdfsPath; - } - public PcapServiceCli(String[] args) { - - this.args = args; - - Option help = new Option("h", "Display help menu"); - options.addOption(help); - options.addOption( - "port", - true, - "OPTIONAL ARGUMENT [portnumber] If this argument sets the port for starting the service. If this argument is not set the port will start on defaut port 8081"); - options.addOption( - "endpoint_uri", - true, - "OPTIONAL ARGUMENT [/uri/to/service] This sets the URI for the service to be hosted. The default URI is /pcapGetter"); - options.addOption( - "query_hdfs_path", - true, - "[query_hdfs_loc] The location in HDFS to temporarily store query results. They will be cleaned up after the query is returned." - ); - options.addOption( - "pcap_hdfs_path", - true, - "[pcap_hdfs_path] The location in HDFS where PCAP raw data is stored in sequence files." - ); - options.addOption( - "log4j", - true, - "OPTIONAL ARGUMENT [log4j] The log4j properties." - ); - } - - public void parse() { - CommandLineParser parser = new BasicParser(); - - CommandLine cmd = null; - - try { - cmd = parser.parse(options, args); - } catch (ParseException e1) { - - e1.printStackTrace(); - } - - if (cmd.hasOption("h")) { - help(); - } - - if(cmd.hasOption("log4j")) { - PropertyConfigurator.configure(cmd.getOptionValue("log4j")); - } - - if (cmd.hasOption("port")) { - - try { - port = Integer.parseInt(cmd.getOptionValue("port").trim()); - } catch (Exception e) { - - System.out.println("[Metron] Invalid value for port entered"); - help(); - } - } - if(cmd.hasOption("pcap_hdfs_path")) { - pcapHdfsPath = cmd.getOptionValue("pcap_hdfs_path"); - } - else { - throw new IllegalStateException("You must specify the pcap hdfs path"); - } - if(cmd.hasOption("query_hdfs_path")) { - queryHdfsPath = cmd.getOptionValue("query_hdfs_path"); - } - else { - throw new IllegalStateException("You must specify the query temp hdfs path"); - } - if (cmd.hasOption("endpoint_uri")) { - - try { - - if (uri == null || uri.equals("")) - throw new Exception("invalid uri"); - - uri = cmd.getOptionValue("uri").trim(); - - if (uri.charAt(0) != '/') - uri = "/" + uri; - - if (uri.charAt(uri.length()) == '/') - uri = uri.substring(0, uri.length() - 1); - - } catch (Exception e) { - System.out.println("[Metron] Invalid URI entered"); - help(); - } - } - - } - - private void help() { - // This prints out some help - HelpFormatter formater = new HelpFormatter(); - - formater.printHelp("Topology Options:", options); - - // System.out - // .println("[Metron] Example usage: \n storm jar Metron-Topologies-0.3BETA-SNAPSHOT.jar org.apache.metron.topology.Bro -local_mode true -config_path Metron_Configs/ -generator_spout true"); - - System.exit(0); - } -} diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/ConfigurationUtil.java b/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/ConfigurationUtil.java deleted file mode 100644 index 0ad0a43b63..0000000000 --- a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/ConfigurationUtil.java +++ /dev/null @@ -1,64 +0,0 @@ -/** - * 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.metron.pcapservice; - -import org.apache.commons.configuration.BaseConfiguration; -import org.apache.commons.configuration.Configuration; - - - - -/** - * utility class for this module which loads commons configuration to fetch - * properties from underlying resources to communicate with HDFS. - * - */ -public class ConfigurationUtil { - - private static Configuration propConfiguration = null; - - - /** - * Loads configuration resources - * @return Configuration - */ - public synchronized static Configuration getConfiguration() { - if(propConfiguration == null){ - propConfiguration = new BaseConfiguration(); - } - return propConfiguration; - } - - public static String getPcapOutputPath() { - return getConfiguration().getString("pcap.output.path"); - } - - public static void setPcapOutputPath(String path) { - getConfiguration().setProperty("pcap.output.path", path); - } - - public static String getTempQueryOutputPath() { - return getConfiguration().getString("temp.query.output.path"); - } - public static void setTempQueryOutputPath(String path) { - getConfiguration().setProperty("temp.query.output.path", path); - } - - - -} diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/PcapReceiverImplRestEasy.java b/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/PcapReceiverImplRestEasy.java deleted file mode 100644 index 56ec7b1d5d..0000000000 --- a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/PcapReceiverImplRestEasy.java +++ /dev/null @@ -1,299 +0,0 @@ -/** - * 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.metron.pcapservice; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Joiner; -import com.google.common.collect.Lists; -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import javax.servlet.http.HttpServletResponse; -import javax.ws.rs.DefaultValue; -import javax.ws.rs.GET; -import javax.ws.rs.Path; -import javax.ws.rs.QueryParam; -import javax.ws.rs.WebApplicationException; -import javax.ws.rs.core.Context; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.metron.common.Constants; -import org.apache.metron.common.hadoop.SequenceFileIterable; -import org.apache.metron.common.utils.timestamp.TimestampConverters; -import org.apache.metron.pcap.PcapHelper; -import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; -import org.apache.metron.pcap.filter.query.QueryPcapFilter; -import org.apache.metron.pcap.mr.PcapJob; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Path("/") -public class PcapReceiverImplRestEasy { - private static final Logger LOGGER = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - /** The Constant HEADER_CONTENT_DISPOSITION_NAME. */ - private static final String HEADER_CONTENT_DISPOSITION_NAME = "Content-Disposition"; - - /** The Constant HEADER_CONTENT_DISPOSITION_VALUE. */ - private static final String HEADER_CONTENT_DISPOSITION_VALUE = "attachment; filename=\"managed-threat.pcap\""; - - /** partial response key header name. */ - private static final String HEADER_PARTIAL_RESPONE_KEY = "lastRowKey"; - private static ThreadLocal CONFIGURATION = new ThreadLocal() { - /** - * Returns the current thread's "initial value" for this - * thread-local variable. This method will be invoked the first - * time a thread accesses the variable with the {@link #get} - * method, unless the thread previously invoked the {@link #set} - * method, in which case the {@code initialValue} method will not - * be invoked for the thread. Normally, this method is invoked at - * most once per thread, but it may be invoked again in case of - * subsequent invocations of {@link #remove} followed by {@link #get}. - *

- *

This implementation simply returns {@code null}; if the - * programmer desires thread-local variables to have an initial - * value other than {@code null}, {@code ThreadLocal} must be - * subclassed, and this method overridden. Typically, an - * anonymous inner class will be used. - * - * @return the initial value for this thread-local - */ - @Override - protected Configuration initialValue() { - return new Configuration(); - } - }; - PcapJob queryUtil = new PcapJob(); - - protected PcapJob getQueryUtil() { - return queryUtil; - } - - private static boolean isValidPort(String port) { - if( port != null && !port.equals("") ) { - try { - Integer.parseInt(port); - return true; - } - catch(Exception e) { - return false; - } - } - return false; - } - - /** - * Enable filtering PCAP results by query filter string and start/end packet TS - * - * @param query Filter results based on this query - * @param startTime Only return packets originating after this start time - * @param endTime Only return packets originating before this end time - * @param numReducers Number of reducers to use - * @param servlet_response - * @return REST response - * @throws IOException - */ - @GET - @Path("/pcapGetter/getPcapsByQuery") - public Response getPcapsByIdentifiers( - @QueryParam ("query") String query, - @DefaultValue("-1") @QueryParam ("startTime")long startTime, - @DefaultValue("-1") @QueryParam ("endTime")long endTime, - @DefaultValue("10") @QueryParam ("numReducers")int numReducers, - @Context HttpServletResponse servlet_response) - - throws IOException { - PcapsResponse response = new PcapsResponse(); - SequenceFileIterable results = null; - try { - if (startTime < 0) { - startTime = 0L; - } - if (endTime < 0) { - endTime = System.currentTimeMillis(); - } - if(query == null) { - return Response.serverError().status(Response.Status.NO_CONTENT) - .entity("Query is null").build(); - } - //convert to nanoseconds since the epoch - startTime = TimestampConverters.MILLISECONDS.toNanoseconds(startTime); - endTime = TimestampConverters.MILLISECONDS.toNanoseconds(endTime); - LOGGER.debug("Query received: {}", query); - results = getQueryUtil().query(new org.apache.hadoop.fs.Path(ConfigurationUtil.getPcapOutputPath()) - , new org.apache.hadoop.fs.Path(ConfigurationUtil.getTempQueryOutputPath()) - , startTime - , endTime - , numReducers - , query - , CONFIGURATION.get() - , FileSystem.get(CONFIGURATION.get()) - , new QueryPcapFilter.Configurator() - ); - - response.setPcaps(results != null ? Lists.newArrayList(results) : null); - } catch (Exception e) { - LOGGER.error("Exception occurred while fetching Pcaps by identifiers :", e); - throw new WebApplicationException("Unable to fetch Pcaps via MR job", e); - } finally { - if (null != results) { - results.cleanup(); - } - } - - // return http status '200 OK' along with the complete pcaps response file, - // and headers - return Response - .ok(response.getPcaps(), MediaType.APPLICATION_OCTET_STREAM) - .status(200).build(); - } - - /** - * Enable filtering PCAP results by fixed properties and start/end packet TS - * - * @param srcIp filter value - * @param dstIp filter value - * @param protocol filter value - * @param srcPort filter value - * @param dstPort filter value - * @param startTime filter value - * @param endTime filter value - * @param numReducers Specify the number of reducers to use when executing the mapreduce job - * @param includeReverseTraffic Indicates if filter should check swapped src/dest addresses and IPs - * @param servlet_response - * @return REST response - * @throws IOException - */ - @GET - @Path("/pcapGetter/getPcapsByIdentifiers") - public Response getPcapsByIdentifiers( - @QueryParam ("srcIp") String srcIp, - @QueryParam ("dstIp") String dstIp, - @QueryParam ("protocol") String protocol, - @QueryParam ("srcPort") String srcPort, - @QueryParam ("dstPort") String dstPort, - @DefaultValue("-1") @QueryParam ("startTime")long startTime, - @DefaultValue("-1") @QueryParam ("endTime")long endTime, - @DefaultValue("10") @QueryParam ("numReducers")int numReducers, - @DefaultValue("false") @QueryParam ("includeReverseTraffic") boolean includeReverseTraffic, - @DefaultValue("") @QueryParam ("packetFilter") String packetFilter, - @Context HttpServletResponse servlet_response) - - throws IOException { - - if (!isValidPort(srcPort)) { - return Response.serverError().status(Response.Status.NO_CONTENT) - .entity("'srcPort' must not be null, empty or a non-integer").build(); - } - - if (!isValidPort(dstPort)) { - return Response.serverError().status(Response.Status.NO_CONTENT) - .entity("'dstPort' must not be null, empty or a non-integer").build(); - } - - final boolean includeReverseTrafficF = includeReverseTraffic; - PcapsResponse response = new PcapsResponse(); - SequenceFileIterable results = null; - try { - if(startTime < 0) { - startTime = 0L; - } - if(endTime < 0) { - endTime = System.currentTimeMillis(); - } - - //convert to nanoseconds since the epoch - startTime = TimestampConverters.MILLISECONDS.toNanoseconds(startTime); - endTime = TimestampConverters.MILLISECONDS.toNanoseconds(endTime); - Map query = new HashMap() {{ - if(srcIp != null) { - put(Constants.Fields.SRC_ADDR.getName(), srcIp); - } - if(dstIp != null) { - put(Constants.Fields.DST_ADDR.getName(), dstIp); - } - if(srcPort != null) { - put(Constants.Fields.SRC_PORT.getName(), srcPort); - } - if(dstPort != null) { - put(Constants.Fields.DST_PORT.getName(), dstPort); - } - if(protocol != null) { - put(Constants.Fields.PROTOCOL.getName(), protocol); - } - put(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName(), "" + includeReverseTrafficF); - if(!org.apache.commons.lang3.StringUtils.isEmpty(packetFilter)) { - put(PcapHelper.PacketFields.PACKET_FILTER.getName(), packetFilter); - } - }}; - if(LOGGER.isDebugEnabled()) { - LOGGER.debug("Query received: {}", Joiner.on(",").join(query.entrySet())); - } - results = getQueryUtil().query(new org.apache.hadoop.fs.Path(ConfigurationUtil.getPcapOutputPath()) - , new org.apache.hadoop.fs.Path(ConfigurationUtil.getTempQueryOutputPath()) - , startTime - , endTime - , numReducers - , query - , CONFIGURATION.get() - , FileSystem.get(CONFIGURATION.get()) - , new FixedPcapFilter.Configurator() - ); - response.setPcaps(results != null ? Lists.newArrayList(results) : null); - - } catch (Exception e) { - LOGGER.error("Exception occurred while fetching Pcaps by identifiers :", e); - throw new WebApplicationException("Unable to fetch Pcaps via MR job", e); - } finally { - if (null != results) { - results.cleanup(); - } - } - - // return http status '200 OK' along with the complete pcaps response file, - // and headers - return Response - .ok(response.getPcaps(), MediaType.APPLICATION_OCTET_STREAM) - .status(200).build(); - } - /** - * This method parses the each value in the List using delimiter ',' and - * builds a new List;. - * - * @param keys - * list of keys to be parsed - * @return list of keys - */ - @VisibleForTesting - List parseKeys(List keys) { - // Assert.notEmpty(keys); - List parsedKeys = new ArrayList(); - for (String key : keys) { - parsedKeys.addAll(Arrays.asList(StringUtils.split( - StringUtils.trim(key), ","))); - } - return parsedKeys; - } -} diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/PcapsResponse.java b/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/PcapsResponse.java deleted file mode 100644 index a5f825df2e..0000000000 --- a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/PcapsResponse.java +++ /dev/null @@ -1,118 +0,0 @@ -/** - * 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.metron.pcapservice; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -import org.apache.metron.pcap.PcapMerger; - - - -/** - * Holds pcaps data, status and the partial response key. - * - * @author Sayi - */ -public class PcapsResponse { - - /** - * The Enum Status. - */ - public enum Status { - - /** The partial. */ - PARTIAL, - /** The complete. */ - COMPLETE - }; - - /** response of the processed keys. */ - private List pcaps = new ArrayList();; - - /** partial response key. */ - private String lastRowKey; - - /** The status. */ - private Status status = Status.COMPLETE; - - /** - * Sets the pcaps. - * - * @param pcaps - * the new pcaps - */ - public void setPcaps(List pcaps) { - this.pcaps = pcaps; - } - - /** - * Adds the pcaps. - * - * @param pcaps - * the pcaps - */ - public void addPcaps(byte[] pcaps) { - this.pcaps.add(pcaps); - } - - - /** - * Gets the response size. - * - * @return the response size - */ - public long getResponseSize() { - long responseSize = 0; - for (byte[] pcap : this.pcaps) { - responseSize = responseSize + pcap.length; - } - return responseSize; - } - - /** - * Gets the pcaps. - * - * @return the pcaps - * @throws IOException - * Signals that an I/O exception has occurred. - */ - public byte[] getPcaps() throws IOException { - if(pcaps == null) { - return new byte[] {}; - } - if (pcaps.size() == 1) { - return pcaps.get(0); - } - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - PcapMerger.merge(baos, pcaps); - return baos.toByteArray(); - } - - /* (non-Javadoc) - * @see java.lang.Object#toString() - */ - @Override - public String toString() { - return "PcapsResponse [lastRowKey=" + lastRowKey - + ", status=" + status + ", pcapsSize=" - + String.valueOf(getResponseSize()) + "]"; - } -} diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/RestTestingUtil.java b/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/RestTestingUtil.java deleted file mode 100644 index ef31a2a6bb..0000000000 --- a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/RestTestingUtil.java +++ /dev/null @@ -1,329 +0,0 @@ -/** - * 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.metron.pcapservice; - -import java.util.HashMap; -import java.util.Map; - -import org.springframework.http.HttpEntity; -import org.springframework.http.HttpHeaders; -import org.springframework.http.HttpMethod; -import org.springframework.http.MediaType; -import org.springframework.http.ResponseEntity; -import org.springframework.web.client.RestTemplate; - -/** - * The Class RestTestingUtil. - */ -public class RestTestingUtil { - - /** The host name. */ - public static String hostName = null; - - /** - * Gets the pcaps by keys. - * - * @param keys - * the keys - * @return the pcaps by keys - */ - @SuppressWarnings("unchecked") - private static void getPcapsByKeys(String keys) { - System.out - .println("**********************getPcapsByKeys ******************************************************************************************"); - // 1. - String url = "http://" + hostName - + "/cisco-rest/pcapGetter/getPcapsByKeys?keys={keys}" - + "&includeReverseTraffic={includeReverseTraffic}" - + "&startTime={startTime}" + "&endTime={endTime}" - + "&maxResponseSize={maxResponseSize}"; - // default values - String startTime = "-1"; - String endTime = "-1"; - String maxResponseSize = "6"; - String includeReverseTraffic = "false"; - - @SuppressWarnings("rawtypes") - Map map = new HashMap(); - map.put("keys", keys); - map.put("includeReverseTraffic", includeReverseTraffic); - map.put("startTime", startTime); - map.put("endTime", endTime); - map.put("maxResponseSize", maxResponseSize); - - RestTemplate template = new RestTemplate(); - - // set headers and entity to send - HttpHeaders headers = new HttpHeaders(); - headers.set("Accept", MediaType.APPLICATION_OCTET_STREAM_VALUE); - HttpEntity requestEntity = new HttpEntity(headers); - - // 1. - ResponseEntity response1 = template.exchange(url, HttpMethod.GET, - requestEntity, byte[].class, map); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out - .format( - "getPcapsByKeys : request= \n response= %s \n", - keys, includeReverseTraffic, startTime, endTime, maxResponseSize, - response1); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out.println(); - - // 2. with reverse traffic - includeReverseTraffic = "true"; - map.put("includeReverseTraffic", includeReverseTraffic); - ResponseEntity response2 = template.exchange(url, HttpMethod.GET, - requestEntity, byte[].class, map); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out - .format( - "getPcapsByKeys : request= \n response= %s \n", - keys, includeReverseTraffic, startTime, endTime, maxResponseSize, - response2); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out.println(); - - // 3.with time range - startTime = System.getProperty("startTime", "-1"); - endTime = System.getProperty("endTime", "-1"); - map.put("startTime", startTime); - map.put("endTime", endTime); - ResponseEntity response3 = template.exchange(url, HttpMethod.GET, - requestEntity, byte[].class, map); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out - .format( - "getPcapsByKeys : request= \n response= %s \n", - keys, includeReverseTraffic, startTime, endTime, maxResponseSize, - response3); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out.println(); - - // 4.with maxResponseSize - maxResponseSize = System.getProperty("maxResponseSize", "6"); - map.put("maxResponseSize", maxResponseSize); - ResponseEntity response4 = template.exchange(url, HttpMethod.GET, - requestEntity, byte[].class, map); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out - .format( - "getPcapsByKeys : request= \n response= %s \n", - keys, includeReverseTraffic, startTime, endTime, maxResponseSize, - response4); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out.println(); - - } - - /** - * Gets the pcaps by keys range. - * - * @param startKey - * the start key - * @param endKey - * the end key - * @return the pcaps by keys range - */ - @SuppressWarnings("unchecked") - private static void getPcapsByKeysRange(String startKey, String endKey) { - System.out - .println("**********************getPcapsByKeysRange ******************************************************************************************"); - // 1. - String url = "http://" + hostName - + "/cisco-rest/pcapGetter/getPcapsByKeyRange?startKey={startKey}" - + "&endKey={endKey}" + "&startTime={startTime}" + "&endTime={endTime}" - + "&maxResponseSize={maxResponseSize}"; - // default values - String startTime = "-1"; - String endTime = "-1"; - String maxResponseSize = "6"; - @SuppressWarnings("rawtypes") - Map map = new HashMap(); - map.put("startKey", startKey); - map.put("endKey", "endKey"); - map.put("startTime", startTime); - map.put("endTime", endTime); - map.put("maxResponseSize", maxResponseSize); - - RestTemplate template = new RestTemplate(); - - // set headers and entity to send - HttpHeaders headers = new HttpHeaders(); - headers.set("Accept", MediaType.APPLICATION_OCTET_STREAM_VALUE); - HttpEntity requestEntity = new HttpEntity(headers); - - // 1. - ResponseEntity response1 = template.exchange(url, HttpMethod.GET, - requestEntity, byte[].class, map); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out - .format( - "getPcapsByKeysRange : request= \n response= %s \n", - startKey, endKey, startTime, endTime, maxResponseSize, response1); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out.println(); - - // 2. with time range - startTime = System.getProperty("startTime", "-1"); - endTime = System.getProperty("endTime", "-1"); - map.put("startTime", startTime); - map.put("endTime", endTime); - ResponseEntity response2 = template.exchange(url, HttpMethod.GET, - requestEntity, byte[].class, map); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out - .format( - "getPcapsByKeysRange : request= \n response= %s \n", - startKey, endKey, startTime, endTime, maxResponseSize, response2); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out.println(); - - // 3. with maxResponseSize - maxResponseSize = System.getProperty("maxResponseSize", "6"); - map.put("maxResponseSize", maxResponseSize); - ResponseEntity response3 = template.exchange(url, HttpMethod.GET, - requestEntity, byte[].class, map); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out - .format( - "getPcapsByKeysRange : request= \n response= %s \n", - startKey, endKey, startTime, endTime, maxResponseSize, response3); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out.println(); - - } - - private static void getPcapsByIdentifiers(String srcIp, String dstIp, String protocol, String srcPort, String dstPort) { - System.out - .println("**********************getPcapsByKeysRange ******************************************************************************************"); - // 1. - String url = "http://" + hostName - + "/pcapGetter/getPcapsByIdentifiers?srcIp={srcIp}" - + "&dstIp={dstIp}" + "&protocol={protocol}" + "&srcPort={srcPort}" - + "&dstPort={dstPort}"; - // default values - String startTime = "-1"; - String endTime = "-1"; - String maxResponseSize = "6"; - @SuppressWarnings("rawtypes") - Map map = new HashMap(); - map.put("srcIp", srcIp); - map.put("dstIp", dstIp); - map.put("protocol", protocol); - map.put("srcPort", srcPort); - map.put("dstPort", dstPort); - - RestTemplate template = new RestTemplate(); - - // set headers and entity to send - HttpHeaders headers = new HttpHeaders(); - headers.set("Accept", MediaType.APPLICATION_OCTET_STREAM_VALUE); - HttpEntity requestEntity = new HttpEntity(headers); - - // 1. - ResponseEntity response1 = template.exchange(url, HttpMethod.GET, - requestEntity, byte[].class, map); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out - .format( - "getPcapsByIdentifiers : request= \n response= %s \n", - srcIp, dstIp, protocol, endTime, srcPort, dstPort, response1); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out.println(); - - // 2. with time range - startTime = System.getProperty("startTime", "-1"); - endTime = System.getProperty("endTime", "-1"); - map.put("startTime", startTime); - map.put("endTime", endTime); - ResponseEntity response2 = template.exchange(url, HttpMethod.GET, - requestEntity, byte[].class, map); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out - .format( - "getPcapsByIdentifiers : request= \n response= %s \n", - srcIp, dstIp, protocol, endTime, srcPort, dstPort, response2); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out.println(); - - // 3. with maxResponseSize - maxResponseSize = System.getProperty("maxResponseSize", "6"); - map.put("maxResponseSize", maxResponseSize); - ResponseEntity response3 = template.exchange(url, HttpMethod.GET, - requestEntity, byte[].class, map); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out - .format( - "getPcapsByIdentifiers : request= \n response= %s \n", - srcIp, dstIp, protocol, endTime, srcPort, dstPort, response3); - System.out - .println("----------------------------------------------------------------------------------------------------"); - System.out.println(); - - } - - /** - * The main method. - * - * @param args - * the arguments - */ - public static void main(String[] args) { - - /* - * Run this program with system properties - * - * -DhostName=mon.hw.com:8090 - * -Dkeys=18800006-1800000b-06-0019-b39d,18800006- - * 1800000b-06-0050-5af6-64840-40785 - * -DstartKey=18000002-18800002-06-0436-0019-2440-34545 - * -DendKey=18000002-18800002-06-b773-0019-2840-34585 - */ - - hostName = System.getProperty("hostName"); - - String keys = System.getProperty("keys"); - - String statyKey = System.getProperty("startKey"); - String endKey = System.getProperty("endKey"); - - getPcapsByKeys(keys); - getPcapsByKeysRange(statyKey, endKey); - - } -} diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/rest/JettyServiceRunner.java b/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/rest/JettyServiceRunner.java deleted file mode 100644 index 2a930b8639..0000000000 --- a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/rest/JettyServiceRunner.java +++ /dev/null @@ -1,44 +0,0 @@ -/** - * 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.metron.pcapservice.rest; - -import org.apache.metron.pcapservice.PcapReceiverImplRestEasy; - -import java.util.HashSet; -import java.util.Set; - -import javax.ws.rs.core.Application; - - -public class JettyServiceRunner extends Application { - - - private static Set services = new HashSet(); - - public JettyServiceRunner() { - // initialize restful services - services.add(new PcapReceiverImplRestEasy()); - } - @Override - public Set getSingletons() { - return services; - } - public static Set getServices() { - return services; - } -} diff --git a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/rest/PcapService.java b/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/rest/PcapService.java deleted file mode 100644 index d183d5d955..0000000000 --- a/metron-platform/metron-api/src/main/java/org/apache/metron/pcapservice/rest/PcapService.java +++ /dev/null @@ -1,56 +0,0 @@ -/** - * 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.metron.pcapservice.rest; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import org.apache.metron.api.helper.service.PcapServiceCli; -import org.apache.metron.pcapservice.ConfigurationUtil; -import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -import org.jboss.resteasy.plugins.server.servlet.HttpServletDispatcher; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -public class PcapService { - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - public static void main(String[] args) throws IOException { - PcapServiceCli cli = new PcapServiceCli(args); - cli.parse(); - ConfigurationUtil.setPcapOutputPath(cli.getPcapHdfsPath()); - LOG.info("Pcap location set to {}", cli.getPcapHdfsPath()); - ConfigurationUtil.setTempQueryOutputPath(cli.getQueryHdfsPath()); - LOG.info("Query temp location set to {}", cli.getQueryHdfsPath()); - Server server = new Server(cli.getPort()); - ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS); - context.setContextPath("/"); - ServletHolder h = new ServletHolder(new HttpServletDispatcher()); - h.setInitParameter("javax.ws.rs.Application", "org.apache.metron.pcapservice.rest.JettyServiceRunner"); - context.addServlet(h, "/*"); - server.setHandler(context); - try { - server.start(); - server.join(); - } catch (Exception e) { - e.printStackTrace(); - } - } -} diff --git a/metron-platform/metron-api/src/main/resources/META-INF/LICENSE b/metron-platform/metron-api/src/main/resources/META-INF/LICENSE deleted file mode 100644 index 0541bd4177..0000000000 --- a/metron-platform/metron-api/src/main/resources/META-INF/LICENSE +++ /dev/null @@ -1,605 +0,0 @@ -Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "{}" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright {yyyy} {name of copyright owner} - - Licensed 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. - ------------------------------------------------------------------------------------- - ------------------------------------------------------------------------------------- - BSD ------------------------------------------------------------------------------------- -This product bundles asm 3.1, which is available under a "BSD Software License" license. For details, see http://asm.ow2.org/ -This product bundles protobuf-java 2.5.0, which is available under a "BSD Software License" license. For details, see http://code.google.com/p/protobuf -This product bundles jsch 0.1.42, which is available under a "BSD Software License" license. For details, see http://www.jcraft.com/jsch/ -This product bundles paranamer 2.3, which is available under a "BSD Software License" license. For details, see https://github.com/paul-hammant/paranamer -This product bundles leveldbjni-all 1.8, which is available under a "BSD Software License" license. For details, see https://github.com/fusesource/leveldbjni -This product bundles xmlenc 0.52, which is available under a "BSD Software License" license. For details, see http://xmlenc.sourceforge.net - -This product contains a modified version of 'JZlib', a re-implementation of -zlib in pure Java, which can be obtained at: - - * LICENSE: - * license/LICENSE.jzlib.txt (BSD Style License) - * HOMEPAGE: - * http://www.jcraft.com/jzlib/ - - - -Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. - -2. Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. - -3. Neither the name of the copyright holder nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - ------------------------------------------------------------------------------------- - CDDLv1.1 ------------------------------------------------------------------------------------- - -This product bundles jersey-guice 1.9, which is available under a "Common Development and Distribution License v1.1" license. For details, see https://jersey.java.net/ -This product bundles jersey-client 1.9, which is available under a "Common Development and Distribution License v1.1" license. For details, see https://jersey.java.net/ -This product bundles jersey-core 1.9, which is available under a "Common Development and Distribution License v1.1" license. For details, see https://jersey.java.net/ -This product bundles jersey-json 1.9, which is available under a "Common Development and Distribution License v1.1" license. For details, see https://jersey.java.net/ -This product bundles jersey-server 1.9, which is available under a "Common Development and Distribution License v1.1" license. For details, see https://jersey.java.net/ -This product bundles jaxb-impl 2.2.5-2, which is available under a "Common Development and Distribution License" license. For details, see http://jaxb.java.net/ -This product bundles activation 1.1, which is available under a "Common Development and Distribution License v1.0" license. For details, see http://java.sun.com/products/javabeans/jaf/index.jsp -This product bundles jsr250-api 1.0, which is available under a "Common Development and Distribution License v1.0" license. For details, see http://jcp.org/aboutJava/communityprocess/final/jsr250/index.html -This product bundles mail 1.4, which is available under a "Common Development and Distribution License v1.0" license. For details, see https://glassfish.dev.java.net/javaee5/mail/ -This product bundles javax.servlet-api 3.1.0, which is available under a "Common Development and Distribution License" license. For details, see http://servlet-spec.java.net -This product bundles servlet-api 2.5, which is available under a "Common Development and Distribution License" license. For details, see http://oracle.com -This product bundles jaxb-api 2.2.2, which is available under a "Common Development and Distribution License" license. For details, see https://jaxb.dev.java.net/ -This product bundles stax-api 1.0-2, which is available under a "Common Development and Distribution License v1.0" license. For details, see https://docs.oracle.com/javase/7/docs/api/javax/xml/stream/package-summary.html - -OMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0 - 1. Definitions. - 1.1. "Contributor" means each individual or entity that - creates or contributes to the creation of Modifications. - 1.2. "Contributor Version" means the combination of the - Original Software, prior Modifications used by a - Contributor (if any), and the Modifications made by that - particular Contributor. - 1.3. "Covered Software" means (a) the Original Software, or - (b) Modifications, or (c) the combination of files - containing Original Software with files containing - Modifications, in each case including portions thereof. - 1.4. "Executable" means the Covered Software in any form - other than Source Code. - 1.5. "Initial Developer" means the individual or entity - that first makes Original Software available under this - License. - 1.6. "Larger Work" means a work which combines Covered - Software or portions thereof with code not governed by the - terms of this License. - 1.7. "License" means this document. - 1.8. "Licensable" means having the right to grant, to the - maximum extent possible, whether at the time of the initial - grant or subsequently acquired, any and all of the rights - conveyed herein. - 1.9. "Modifications" means the Source Code and Executable - form of any of the following: - A. Any file that results from an addition to, - deletion from or modification of the contents of a - file containing Original Software or previous - Modifications; - B. Any new file that contains any part of the - Original Software or previous Modification; or - C. Any new file that is contributed or otherwise made - available under the terms of this License. - 1.10. "Original Software" means the Source Code and - Executable form of computer software code that is - originally released under this License. - 1.11. "Patent Claims" means any patent claim(s), now owned - or hereafter acquired, including without limitation, - method, process, and apparatus claims, in any patent - Licensable by grantor. - 1.12. "Source Code" means (a) the common form of computer - software code in which modifications are made and (b) - associated documentation included in or with such code. - 1.13. "You" (or "Your") means an individual or a legal - entity exercising rights under, and complying with all of - the terms of, this License. For legal entities, "You" - includes any entity which controls, is controlled by, or is - under common control with You. For purposes of this - definition, "control" means (a) the power, direct or - indirect, to cause the direction or management of such - entity, whether by contract or otherwise, or (b) ownership - of more than fifty percent (50%) of the outstanding shares - or beneficial ownership of such entity. - 2. License Grants. - 2.1. The Initial Developer Grant. - Conditioned upon Your compliance with Section 3.1 below and - subject to third party intellectual property claims, the - Initial Developer hereby grants You a world-wide, - royalty-free, non-exclusive license: - (a) under intellectual property rights (other than - patent or trademark) Licensable by Initial Developer, - to use, reproduce, modify, display, perform, - sublicense and distribute the Original Software (or - portions thereof), with or without Modifications, - and/or as part of a Larger Work; and - (b) under Patent Claims infringed by the making, - using or selling of Original Software, to make, have - made, use, practice, sell, and offer for sale, and/or - otherwise dispose of the Original Software (or - portions thereof). - (c) The licenses granted in Sections 2.1(a) and (b) - are effective on the date Initial Developer first - distributes or otherwise makes the Original Software - available to a third party under the terms of this - License. - (d) Notwithstanding Section 2.1(b) above, no patent - license is granted: (1) for code that You delete from - the Original Software, or (2) for infringements - caused by: (i) the modification of the Original - Software, or (ii) the combination of the Original - Software with other software or devices. - 2.2. Contributor Grant. - Conditioned upon Your compliance with Section 3.1 below and - subject to third party intellectual property claims, each - Contributor hereby grants You a world-wide, royalty-free, - non-exclusive license: - (a) under intellectual property rights (other than - patent or trademark) Licensable by Contributor to - use, reproduce, modify, display, perform, sublicense - and distribute the Modifications created by such - Contributor (or portions thereof), either on an - unmodified basis, with other Modifications, as - Covered Software and/or as part of a Larger Work; and - (b) under Patent Claims infringed by the making, - using, or selling of Modifications made by that - Contributor either alone and/or in combination with - its Contributor Version (or portions of such - combination), to make, use, sell, offer for sale, - have made, and/or otherwise dispose of: (1) - Modifications made by that Contributor (or portions - thereof); and (2) the combination of Modifications - made by that Contributor with its Contributor Version - (or portions of such combination). - (c) The licenses granted in Sections 2.2(a) and - 2.2(b) are effective on the date Contributor first - distributes or otherwise makes the Modifications - available to a third party. - (d) Notwithstanding Section 2.2(b) above, no patent - license is granted: (1) for any code that Contributor - has deleted from the Contributor Version; (2) for - infringements caused by: (i) third party - modifications of Contributor Version, or (ii) the - combination of Modifications made by that Contributor - with other software (except as part of the - Contributor Version) or other devices; or (3) under - Patent Claims infringed by Covered Software in the - absence of Modifications made by that Contributor. - 3. Distribution Obligations. - 3.1. Availability of Source Code. - Any Covered Software that You distribute or otherwise make - available in Executable form must also be made available in - Source Code form and that Source Code form must be - distributed only under the terms of this License. You must - include a copy of this License with every copy of the - Source Code form of the Covered Software You distribute or - otherwise make available. You must inform recipients of any - such Covered Software in Executable form as to how they can - obtain such Covered Software in Source Code form in a - reasonable manner on or through a medium customarily used - for software exchange. - 3.2. Modifications. - The Modifications that You create or to which You - contribute are governed by the terms of this License. You - represent that You believe Your Modifications are Your - original creation(s) and/or You have sufficient rights to - grant the rights conveyed by this License. - 3.3. Required Notices. - You must include a notice in each of Your Modifications - that identifies You as the Contributor of the Modification. - You may not remove or alter any copyright, patent or - trademark notices contained within the Covered Software, or - any notices of licensing or any descriptive text giving - attribution to any Contributor or the Initial Developer. - 3.4. Application of Additional Terms. - You may not offer or impose any terms on any Covered - Software in Source Code form that alters or restricts the - applicable version of this License or the recipients' - rights hereunder. You may choose to offer, and to charge a - fee for, warranty, support, indemnity or liability - obligations to one or more recipients of Covered Software. - However, you may do so only on Your own behalf, and not on - behalf of the Initial Developer or any Contributor. You - must make it absolutely clear that any such warranty, - support, indemnity or liability obligation is offered by - You alone, and You hereby agree to indemnify the Initial - Developer and every Contributor for any liability incurred - by the Initial Developer or such Contributor as a result of - warranty, support, indemnity or liability terms You offer. - 3.5. Distribution of Executable Versions. - You may distribute the Executable form of the Covered - Software under the terms of this License or under the terms - of a license of Your choice, which may contain terms - different from this License, provided that You are in - compliance with the terms of this License and that the - license for the Executable form does not attempt to limit - or alter the recipient's rights in the Source Code form - from the rights set forth in this License. If You - distribute the Covered Software in Executable form under a - different license, You must make it absolutely clear that - any terms which differ from this License are offered by You - alone, not by the Initial Developer or Contributor. You - hereby agree to indemnify the Initial Developer and every - Contributor for any liability incurred by the Initial - Developer or such Contributor as a result of any such terms - You offer. - 3.6. Larger Works. - You may create a Larger Work by combining Covered Software - with other code not governed by the terms of this License - and distribute the Larger Work as a single product. In such - a case, You must make sure the requirements of this License - are fulfilled for the Covered Software. - 4. Versions of the License. - 4.1. New Versions. - Sun Microsystems, Inc. is the initial license steward and - may publish revised and/or new versions of this License - from time to time. Each version will be given a - distinguishing version number. Except as provided in - Section 4.3, no one other than the license steward has the - right to modify this License. - 4.2. Effect of New Versions. - You may always continue to use, distribute or otherwise - make the Covered Software available under the terms of the - version of the License under which You originally received - the Covered Software. If the Initial Developer includes a - notice in the Original Software prohibiting it from being - distributed or otherwise made available under any - subsequent version of the License, You must distribute and - make the Covered Software available under the terms of the - version of the License under which You originally received - the Covered Software. Otherwise, You may also choose to - use, distribute or otherwise make the Covered Software - available under the terms of any subsequent version of the - License published by the license steward. - 4.3. Modified Versions. - When You are an Initial Developer and You want to create a - new license for Your Original Software, You may create and - use a modified version of this License if You: (a) rename - the license and remove any references to the name of the - license steward (except to note that the license differs - from this License); and (b) otherwise make it clear that - the license contains terms which differ from this License. - 5. DISCLAIMER OF WARRANTY. - COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN "AS IS" - BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, - INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED - SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR - PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND - PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY - COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE - INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF - ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF - WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF - ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS - DISCLAIMER. - 6. TERMINATION. - 6.1. This License and the rights granted hereunder will - terminate automatically if You fail to comply with terms - herein and fail to cure such breach within 30 days of - becoming aware of the breach. Provisions which, by their - nature, must remain in effect beyond the termination of - this License shall survive. - 6.2. If You assert a patent infringement claim (excluding - declaratory judgment actions) against Initial Developer or - a Contributor (the Initial Developer or Contributor against - whom You assert such claim is referred to as "Participant") - alleging that the Participant Software (meaning the - Contributor Version where the Participant is a Contributor - or the Original Software where the Participant is the - Initial Developer) directly or indirectly infringes any - patent, then any and all rights granted directly or - indirectly to You by such Participant, the Initial - Developer (if the Initial Developer is not the Participant) - and all Contributors under Sections 2.1 and/or 2.2 of this - License shall, upon 60 days notice from Participant - terminate prospectively and automatically at the expiration - of such 60 day notice period, unless if within such 60 day - period You withdraw Your claim with respect to the - Participant Software against such Participant either - unilaterally or pursuant to a written agreement with - Participant. - 6.3. In the event of termination under Sections 6.1 or 6.2 - above, all end user licenses that have been validly granted - by You or any distributor hereunder prior to termination - (excluding licenses granted to You by any distributor) - shall survive termination. - 7. LIMITATION OF LIABILITY. - UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT - (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE - INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF - COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE - LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR - CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT - LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF GOODWILL, WORK - STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER - COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN - INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF - LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL - INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT - APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO - NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR - CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT - APPLY TO YOU. - 8. U.S. GOVERNMENT END USERS. - The Covered Software is a "commercial item," as that term is - defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of "commercial - computer software" (as that term is defined at 48 C.F.R. ¤ - 252.227-7014(a)(1)) and "commercial computer software - documentation" as such terms are used in 48 C.F.R. 12.212 (Sept. - 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 - through 227.7202-4 (June 1995), all U.S. Government End Users - acquire Covered Software with only those rights set forth herein. - This U.S. Government Rights clause is in lieu of, and supersedes, - any other FAR, DFAR, or other clause or provision that addresses - Government rights in computer software under this License. - 9. MISCELLANEOUS. - This License represents the complete agreement concerning subject - matter hereof. If any provision of this License is held to be - unenforceable, such provision shall be reformed only to the - extent necessary to make it enforceable. This License shall be - governed by the law of the jurisdiction specified in a notice - contained within the Original Software (except to the extent - applicable law, if any, provides otherwise), excluding such - jurisdiction's conflict-of-law provisions. Any litigation - relating to this License shall be subject to the jurisdiction of - the courts located in the jurisdiction and venue specified in a - notice contained within the Original Software, with the losing - party responsible for costs, including, without limitation, court - costs and reasonable attorneys' fees and expenses. The - application of the United Nations Convention on Contracts for the - International Sale of Goods is expressly excluded. Any law or - regulation which provides that the language of a contract shall - be construed against the drafter shall not apply to this License. - You agree that You alone are responsible for compliance with the - United States export administration regulations (and the export - control laws and regulation of any other countries) when You use, - distribute or otherwise make available any Covered Software. - 10. RESPONSIBILITY FOR CLAIMS. - As between Initial Developer and the Contributors, each party is - responsible for claims and damages arising, directly or - indirectly, out of its utilization of rights under this License - and You agree to work with Initial Developer and Contributors to - distribute such responsibility on an equitable basis. Nothing - herein is intended or shall be deemed to constitute any admission - of liability. - ------------------------------------------------------------------------------------- - MIT ------------------------------------------------------------------------------------- - -This product bundles slf4j-api 1.7.7, which is available under a "MIT Software License" license. For details, see http://www.slf4j.org -This product bundles slf4j-log4j12 1.7.7, which is available under a "MIT Software License" license. For details, see http://www.slf4j.org -This product bundles slf4j-simple 1.7.7, which is available under a "MIT Software License" license. For details, see http://www.slf4j.org - -Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: - -The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. - ------------------------------------------------------------------------------------- - Public Domain ------------------------------------------------------------------------------------- - -This product contains the extensions to Java Collections Framework which has -been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: - - * LICENSE: - * license/LICENSE.jsr166y.txt (Public Domain) - * HOMEPAGE: - * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ - * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ - -This product contains a modified version of Robert Harder's Public Domain -Base64 Encoder and Decoder, which can be obtained at: - - * LICENSE: - * license/LICENSE.base64.txt (Public Domain) - * HOMEPAGE: - * http://iharder.sourceforge.net/current/java/base64/ - - diff --git a/metron-platform/metron-api/src/main/resources/META-INF/NOTICE b/metron-platform/metron-api/src/main/resources/META-INF/NOTICE deleted file mode 100644 index 63bf87abe5..0000000000 --- a/metron-platform/metron-api/src/main/resources/META-INF/NOTICE +++ /dev/null @@ -1,39 +0,0 @@ - -metron-api -Copyright 2006-2016 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -This product includes software developed for Orekit by -CS Systèmes d'Information (http://www.c-s.fr/) -Copyright 2010-2012 CS Systèmes d'Information - - The Netty Project - ================= - -Please visit the Netty web site for more information: - - * http://netty.io/ - -Copyright 2011 The Netty Project - -This product includes software developed by -Joda.org (http://www.joda.org/). - -Spring Framework 3.2.6.RELEASE -Copyright (c) 2002-2013 Pivotal, Inc. - -Google Guice - Core Library -Copyright 2006-2011 Google, Inc. - -Google Guice - Extensions - Servlet -Copyright 2006-2011 Google, Inc. - - Portions of this software were originally based on the following: - - software copyright (c) 1999, IBM Corporation., http://www.ibm.com. - - software copyright (c) 1999, Sun Microsystems., http://www.sun.com. - - voluntary contributions made by Paul Eng on behalf of the - Apache Software Foundation that were originally developed at iClick, Inc., - software copyright (c) 1999. - diff --git a/metron-platform/metron-api/src/test/java/org/apache/metron/pcapservice/ConfigurationUtilTest.java b/metron-platform/metron-api/src/test/java/org/apache/metron/pcapservice/ConfigurationUtilTest.java deleted file mode 100644 index 4ecb3a48b6..0000000000 --- a/metron-platform/metron-api/src/test/java/org/apache/metron/pcapservice/ConfigurationUtilTest.java +++ /dev/null @@ -1,43 +0,0 @@ -/** - * 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.metron.pcapservice; - -import org.junit.Assert; -import org.junit.Test; - - - -public class ConfigurationUtilTest { - - - @Test - public void test_getPcapOutputPath() { - ConfigurationUtil.setPcapOutputPath("/foo"); - Assert.assertEquals(ConfigurationUtil.getPcapOutputPath(), "/foo"); - } - - /** - * Test_get max allowable results size unit. - */ - @Test - public void test_getTempQueryDir() { - ConfigurationUtil.setTempQueryOutputPath("/tmp"); - Assert.assertEquals(ConfigurationUtil.getTempQueryOutputPath(), "/tmp"); - } - -} diff --git a/metron-platform/metron-api/src/test/java/org/apache/metron/pcapservice/PcapReceiverImplRestEasyTest.java b/metron-platform/metron-api/src/test/java/org/apache/metron/pcapservice/PcapReceiverImplRestEasyTest.java deleted file mode 100644 index ea6db7080c..0000000000 --- a/metron-platform/metron-api/src/test/java/org/apache/metron/pcapservice/PcapReceiverImplRestEasyTest.java +++ /dev/null @@ -1,255 +0,0 @@ -/** - * 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.metron.pcapservice; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.metron.common.Constants; -import org.apache.metron.common.hadoop.SequenceFileIterable; -import org.apache.metron.common.utils.timestamp.TimestampConverters; -import org.apache.metron.pcap.PcapHelper; -import org.apache.metron.pcap.filter.PcapFilterConfigurator; -import org.apache.metron.pcap.mr.PcapJob; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; -import java.util.EnumMap; -import java.util.Map; - -public class PcapReceiverImplRestEasyTest { - - public static class MockQueryHandler extends PcapJob { - Path basePath; - Path baseOutputPath; - long beginNS; - long endNS; - R fields; - PcapFilterConfigurator filterImpl; - - @Override - public SequenceFileIterable query(Path basePath - , Path baseOutputPath - , long beginNS - , long endNS - , int numReducers - , T fields - , Configuration conf - , FileSystem fs - , PcapFilterConfigurator filterImpl - ) throws IOException, ClassNotFoundException, InterruptedException - { - this.basePath = basePath; - this.baseOutputPath = baseOutputPath; - this.beginNS = beginNS; - this.endNS = endNS; - this.fields = (R) fields; - this.filterImpl = (PcapFilterConfigurator) filterImpl; - return null; - } - } - - final MockQueryHandler> fixedQueryHandler = new MockQueryHandler<>(); - final MockQueryHandler queryQueryHandler = new MockQueryHandler(); - PcapReceiverImplRestEasy fixedRestEndpoint = new PcapReceiverImplRestEasy() {{ - this.queryUtil = fixedQueryHandler; - }}; - PcapReceiverImplRestEasy queryRestEndpoint = new PcapReceiverImplRestEasy() {{ - this.queryUtil = queryQueryHandler; - }}; - - @Before - public void setup() throws Exception { - ConfigurationUtil.setPcapOutputPath("/output"); - ConfigurationUtil.setTempQueryOutputPath("/tmp"); - } - - @Test - public void testNormalFixedPath() throws Exception { - String srcIp = "srcIp"; - String dstIp = "dstIp"; - String protocol = "protocol"; - String srcPort = "80"; - String dstPort = "100"; - long startTime = 100; - long endTime = 1000; - String query = "`blah`"; - - { - boolean includeReverseTraffic = false; - fixedRestEndpoint.getPcapsByIdentifiers(srcIp, dstIp, protocol, srcPort, dstPort, startTime, endTime, 10, includeReverseTraffic, query, null); - Assert.assertEquals(new Path(ConfigurationUtil.getPcapOutputPath()), fixedQueryHandler.basePath); - Assert.assertEquals(new Path(ConfigurationUtil.getTempQueryOutputPath()), fixedQueryHandler.baseOutputPath); - Assert.assertEquals(srcIp, fixedQueryHandler.fields.get(Constants.Fields.SRC_ADDR.getName())); - Assert.assertEquals(dstIp, fixedQueryHandler.fields.get(Constants.Fields.DST_ADDR.getName())); - Assert.assertEquals(srcPort, fixedQueryHandler.fields.get(Constants.Fields.SRC_PORT.getName())); - Assert.assertEquals(dstPort, fixedQueryHandler.fields.get(Constants.Fields.DST_PORT.getName())); - Assert.assertEquals(TimestampConverters.MILLISECONDS.toNanoseconds(startTime), fixedQueryHandler.beginNS); - Assert.assertEquals(TimestampConverters.MILLISECONDS.toNanoseconds(endTime), fixedQueryHandler.endNS); - Assert.assertEquals(includeReverseTraffic, Boolean.parseBoolean(fixedQueryHandler.fields.get(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName()))); - Assert.assertEquals(query, fixedQueryHandler.fields.get(PcapHelper.PacketFields.PACKET_FILTER.getName())); - } - { - boolean includeReverseTraffic = true; - fixedRestEndpoint.getPcapsByIdentifiers(srcIp, dstIp, protocol, srcPort, dstPort, startTime, endTime, 10, includeReverseTraffic, query, null); - Assert.assertEquals(new Path(ConfigurationUtil.getPcapOutputPath()), fixedQueryHandler.basePath); - Assert.assertEquals(new Path(ConfigurationUtil.getTempQueryOutputPath()), fixedQueryHandler.baseOutputPath); - Assert.assertEquals(srcIp, fixedQueryHandler.fields.get(Constants.Fields.SRC_ADDR.getName())); - Assert.assertEquals(dstIp, fixedQueryHandler.fields.get(Constants.Fields.DST_ADDR.getName())); - Assert.assertEquals(srcPort, fixedQueryHandler.fields.get(Constants.Fields.SRC_PORT.getName())); - Assert.assertEquals(dstPort, fixedQueryHandler.fields.get(Constants.Fields.DST_PORT.getName())); - Assert.assertEquals(TimestampConverters.MILLISECONDS.toNanoseconds(startTime), fixedQueryHandler.beginNS); - Assert.assertEquals(TimestampConverters.MILLISECONDS.toNanoseconds(endTime), fixedQueryHandler.endNS); - Assert.assertEquals(includeReverseTraffic, Boolean.parseBoolean(fixedQueryHandler.fields.get(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName()))); - Assert.assertEquals(query, fixedQueryHandler.fields.get(PcapHelper.PacketFields.PACKET_FILTER.getName())); - } - } - - @Test - public void testNormalQueryPath() throws Exception { - long startTime = 100; - long endTime = 1000; - String query = "ip_src_addr == 'srcIp' and ip_src_port == '80' and ip_dst_addr == 'dstIp' and ip_dst_port == '100' and protocol == 'protocol'"; - queryRestEndpoint.getPcapsByIdentifiers(query, startTime, endTime, 10, null); - Assert.assertEquals(new Path(ConfigurationUtil.getPcapOutputPath()), queryQueryHandler.basePath); - Assert.assertEquals(new Path(ConfigurationUtil.getTempQueryOutputPath()), queryQueryHandler.baseOutputPath); - Assert.assertEquals(query, queryQueryHandler.fields); - Assert.assertEquals(TimestampConverters.MILLISECONDS.toNanoseconds(startTime), queryQueryHandler.beginNS); - Assert.assertEquals(TimestampConverters.MILLISECONDS.toNanoseconds(endTime), queryQueryHandler.endNS); - } - - @Test - public void testNullSrcIp() throws Exception { - String srcIp = null; - String dstIp = "dstIp"; - String protocol = "protocol"; - String srcPort = "80"; - String dstPort = "100"; - long startTime = 100; - long endTime = 1000; - boolean includeReverseTraffic = false; - String query = "`metron`"; - fixedRestEndpoint.getPcapsByIdentifiers(srcIp, dstIp, protocol, srcPort, dstPort, startTime, endTime, 10, includeReverseTraffic, query, null); - Assert.assertEquals(new Path(ConfigurationUtil.getPcapOutputPath()), fixedQueryHandler.basePath); - Assert.assertEquals(new Path(ConfigurationUtil.getTempQueryOutputPath()), fixedQueryHandler.baseOutputPath); - Assert.assertEquals(srcIp, fixedQueryHandler.fields.get(Constants.Fields.SRC_ADDR.getName())); - Assert.assertEquals(dstIp, fixedQueryHandler.fields.get(Constants.Fields.DST_ADDR.getName())); - Assert.assertEquals(srcPort, fixedQueryHandler.fields.get(Constants.Fields.SRC_PORT.getName())); - Assert.assertEquals(dstPort, fixedQueryHandler.fields.get(Constants.Fields.DST_PORT.getName())); - Assert.assertEquals(TimestampConverters.MILLISECONDS.toNanoseconds(startTime), fixedQueryHandler.beginNS); - Assert.assertEquals(TimestampConverters.MILLISECONDS.toNanoseconds(endTime), fixedQueryHandler.endNS); - Assert.assertEquals(includeReverseTraffic, Boolean.parseBoolean(fixedQueryHandler.fields.get(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName()))); - Assert.assertEquals(query, fixedQueryHandler.fields.get(PcapHelper.PacketFields.PACKET_FILTER.getName())); - } - - @Test - public void testNullDstIp() throws Exception { - String srcIp = "srcIp"; - String dstIp = null; - String protocol = "protocol"; - String srcPort = "80"; - String dstPort = "100"; - long startTime = 100; - long endTime = 1000; - String query = null; - boolean includeReverseTraffic = false; - fixedRestEndpoint.getPcapsByIdentifiers(srcIp, dstIp, protocol, srcPort, dstPort, startTime, endTime, 10, includeReverseTraffic, query, null); - Assert.assertEquals(new Path(ConfigurationUtil.getPcapOutputPath()), fixedQueryHandler.basePath); - Assert.assertEquals(new Path(ConfigurationUtil.getTempQueryOutputPath()), fixedQueryHandler.baseOutputPath); - Assert.assertEquals(srcIp, fixedQueryHandler.fields.get(Constants.Fields.SRC_ADDR.getName())); - Assert.assertEquals(dstIp, fixedQueryHandler.fields.get(Constants.Fields.DST_ADDR.getName())); - Assert.assertEquals(srcPort, fixedQueryHandler.fields.get(Constants.Fields.SRC_PORT.getName())); - Assert.assertEquals(dstPort, fixedQueryHandler.fields.get(Constants.Fields.DST_PORT.getName())); - Assert.assertEquals(TimestampConverters.MILLISECONDS.toNanoseconds(startTime), fixedQueryHandler.beginNS); - Assert.assertEquals(TimestampConverters.MILLISECONDS.toNanoseconds(endTime), fixedQueryHandler.endNS); - Assert.assertEquals(includeReverseTraffic, Boolean.parseBoolean(fixedQueryHandler.fields.get(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName()))); - Assert.assertEquals(query, fixedQueryHandler.fields.get(PcapHelper.PacketFields.PACKET_FILTER.getName())); - } - - @Test - public void testEmptyStartTime() throws Exception { - String srcIp = "srcIp"; - String dstIp = "dstIp"; - String protocol = "protocol"; - String srcPort = "80"; - String dstPort = "100"; - long startTime = -1; - long endTime = 1000; - { - String query = ""; - boolean includeReverseTraffic = false; - fixedRestEndpoint.getPcapsByIdentifiers(srcIp, dstIp, protocol, srcPort, dstPort, startTime, endTime, 10, includeReverseTraffic, query, null); - Assert.assertEquals(new Path(ConfigurationUtil.getPcapOutputPath()), fixedQueryHandler.basePath); - Assert.assertEquals(new Path(ConfigurationUtil.getTempQueryOutputPath()), fixedQueryHandler.baseOutputPath); - Assert.assertEquals(srcIp, fixedQueryHandler.fields.get(Constants.Fields.SRC_ADDR.getName())); - Assert.assertEquals(dstIp, fixedQueryHandler.fields.get(Constants.Fields.DST_ADDR.getName())); - Assert.assertEquals(srcPort, fixedQueryHandler.fields.get(Constants.Fields.SRC_PORT.getName() )); - Assert.assertEquals(dstPort, fixedQueryHandler.fields.get(Constants.Fields.DST_PORT.getName())); - Assert.assertEquals(0, fixedQueryHandler.beginNS); - Assert.assertEquals(TimestampConverters.MILLISECONDS.toNanoseconds(endTime), fixedQueryHandler.endNS); - Assert.assertEquals(includeReverseTraffic, Boolean.parseBoolean(fixedQueryHandler.fields.get(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName()))); - Assert.assertEquals(null, fixedQueryHandler.fields.get(PcapHelper.PacketFields.PACKET_FILTER.getName())); - } - { - String query = "ip_src_addr == 'srcIp' and ip_src_port == '80' and ip_dst_addr == 'dstIp' and ip_dst_port == '100' and protocol == 'protocol'"; - queryRestEndpoint.getPcapsByIdentifiers(query, startTime, endTime, 10, null); - Assert.assertEquals(new Path(ConfigurationUtil.getPcapOutputPath()), queryQueryHandler.basePath); - Assert.assertEquals(new Path(ConfigurationUtil.getTempQueryOutputPath()), queryQueryHandler.baseOutputPath); - Assert.assertEquals(query, queryQueryHandler.fields); - Assert.assertEquals(0, queryQueryHandler.beginNS); - Assert.assertEquals(TimestampConverters.MILLISECONDS.toNanoseconds(endTime), queryQueryHandler.endNS); - } - } - - @Test - public void testEmptyEndTime() throws Exception { - String srcIp = "srcIp"; - String dstIp = "dstIp"; - String protocol = "protocol"; - String srcPort = "80"; - String dstPort = "100"; - long startTime = -1; - long endTime = -1; - { - boolean includeReverseTraffic = false; - fixedRestEndpoint.getPcapsByIdentifiers(srcIp, dstIp, protocol, srcPort, dstPort, startTime, endTime, 10, includeReverseTraffic, null, null); - Assert.assertEquals(new Path(ConfigurationUtil.getPcapOutputPath()), fixedQueryHandler.basePath); - Assert.assertEquals(new Path(ConfigurationUtil.getTempQueryOutputPath()), fixedQueryHandler.baseOutputPath); - Assert.assertEquals(srcIp, fixedQueryHandler.fields.get(Constants.Fields.SRC_ADDR.getName())); - Assert.assertEquals(dstIp, fixedQueryHandler.fields.get(Constants.Fields.DST_ADDR.getName())); - Assert.assertEquals(srcPort, fixedQueryHandler.fields.get(Constants.Fields.SRC_PORT.getName())); - Assert.assertEquals(dstPort, fixedQueryHandler.fields.get(Constants.Fields.DST_PORT.getName())); - Assert.assertEquals(0, fixedQueryHandler.beginNS); - Assert.assertTrue(fixedQueryHandler.endNS > 0); - Assert.assertEquals(includeReverseTraffic, Boolean.parseBoolean(fixedQueryHandler.fields.get(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName()))); - } - { - String query = "ip_src_addr == 'srcIp' and ip_src_port == '80' and ip_dst_addr == 'dstIp' and ip_dst_port == '100' and protocol == 'protocol'"; - queryRestEndpoint.getPcapsByIdentifiers(query, startTime, endTime, 10, null); - Assert.assertEquals(new Path(ConfigurationUtil.getPcapOutputPath()), queryQueryHandler.basePath); - Assert.assertEquals(new Path(ConfigurationUtil.getTempQueryOutputPath()), queryQueryHandler.baseOutputPath); - Assert.assertEquals(query, queryQueryHandler.fields); - Assert.assertEquals(0, queryQueryHandler.beginNS); - Assert.assertTrue(queryQueryHandler.endNS > 0); - } - } - -} diff --git a/metron-platform/metron-api/src/test/resources/log4j.properties b/metron-platform/metron-api/src/test/resources/log4j.properties deleted file mode 100644 index 0d50388fff..0000000000 --- a/metron-platform/metron-api/src/test/resources/log4j.properties +++ /dev/null @@ -1,24 +0,0 @@ -# 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. - -# Root logger option -log4j.rootLogger=ERROR, stdout - -# Direct log messages to stdout -log4j.appender.stdout=org.apache.log4j.ConsoleAppender -log4j.appender.stdout.Target=System.out -log4j.appender.stdout.layout=org.apache.log4j.PatternLayout -log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n diff --git a/metron-platform/metron-api/src/test/resources/test-tcp-packet.pcap b/metron-platform/metron-api/src/test/resources/test-tcp-packet.pcap deleted file mode 100644 index 25d47daa309e6f971f3d46a089c9c0faa7c96363..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 144 zcmca|c+)~A1{Q`5JOBS@00I!n2&C5-8V75|GcaTT*&xg$b(EEX2}m+BFmW)rGBAXl z|K`A;z!s$;0hE+z0E$L3NdD+#IA<2M+`mCW#v?->Xc8j_*Xw3)ju$=mPP=;e_$wsm hWTxho=o(rWfPk@HYDHphK~Ab(W?l&|mzvwWNdUYjCv5-# diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigOption.java similarity index 96% rename from metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java rename to metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigOption.java index e87c176719..473664c6e6 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOption.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigOption.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.metron.pcap; +package org.apache.metron.common.configuration; import java.util.Map; import java.util.function.BiFunction; diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java index b67b904a08..5a2f4856c9 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobStatus.java @@ -18,8 +18,6 @@ package org.apache.metron.job; -import org.apache.hadoop.fs.Path; - /** * Capture metadata about a batch job. */ @@ -38,7 +36,7 @@ public enum State { private State state = State.NOT_RUNNING; private double percentComplete = 0.0; private String description; - private Path interimResultPath; + private long completionTime; public JobStatus withJobId(String jobId) { this.jobId = jobId; @@ -60,8 +58,8 @@ public JobStatus withDescription(String description) { return this; } - public JobStatus withInterimResultPath(Path interimResultPath) { - this.interimResultPath = interimResultPath; + public JobStatus withCompletionTime(long completionTime) { + this.completionTime = completionTime; return this; } @@ -81,8 +79,8 @@ public String getDescription() { return description; } - public Path getInterimResultPath() { - return interimResultPath; + public long getCompletionTime() { + return completionTime; } } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java index 694577089a..4b28e5d26f 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java @@ -63,7 +63,7 @@ enum JobType { * * @return true if job is completed, whether KILLED, FAILED, SUCCEEDED. False otherwise. */ - boolean isDone() throws JobException; + boolean isDone(); /** * Kill job. diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java index 55f4d07401..70ee0acdf9 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java @@ -18,6 +18,7 @@ package org.apache.metron.job.manager; +import java.util.List; import java.util.function.Supplier; import org.apache.metron.job.JobException; import org.apache.metron.job.JobStatus; @@ -35,4 +36,6 @@ public interface JobManager { Statusable getJob(String username, String jobId) throws JobException; + List> getJobs(String username, String jobId) throws JobException; + } diff --git a/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java b/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java index 3311ec3935..09721c61c9 100644 --- a/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java +++ b/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java @@ -21,6 +21,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; import org.mockito.Mock; import org.mockito.MockitoAnnotations; @@ -44,6 +45,8 @@ public class InMemoryJobManagerTest { + @Rule + public TemporaryFolder tempDir = new TemporaryFolder(); @Mock private Statusable job1; @Mock @@ -58,6 +61,7 @@ public class InMemoryJobManagerTest { private String username2; private String jobId1; private String jobId2; + private String basePath; @Before public void setup() throws JobException { @@ -68,6 +72,7 @@ public void setup() throws JobException { username2 = "user456"; jobId1 = "job_abc_123"; jobId2 = "job_def_456"; + basePath = tempDir.getRoot().getAbsolutePath(); when(job1.getJobType()).thenReturn(JobType.MAP_REDUCE); when(job2.getJobType()).thenReturn(JobType.MAP_REDUCE); when(finalizer.finalizeJob(any())).thenReturn(results); @@ -97,6 +102,13 @@ public void submits_job_and_returns_status() throws JobException { public void returns_job_status() throws JobException { JobStatus expected = new JobStatus().withState(State.SUCCEEDED).withJobId(jobId1); when(job1.getStatus()).thenReturn(expected); + jm.submit(() -> { + try { + return job1.submit(finalizer, config); + } catch (JobException e) { + throw new RuntimeException("Something went wrong", e); + } + }, username1); JobStatus status = jm.getStatus(username1, jobId1); assertThat(status, equalTo(expected)); } @@ -104,13 +116,29 @@ public void returns_job_status() throws JobException { @Test public void returns_job_is_done() throws JobException { JobStatus expected = new JobStatus().withState(State.SUCCEEDED).withJobId(jobId1); + when(job1.getStatus()).thenReturn(expected); when(job1.isDone()).thenReturn(true); + jm.submit(() -> { + try { + return job1.submit(finalizer, config); + } catch (JobException e) { + throw new RuntimeException("Something went wrong", e); + } + }, username1); boolean done = jm.done(username1, jobId1); assertThat(done, equalTo(true)); } @Test public void kills_job() throws JobException { + JobStatus expected = new JobStatus().withState(State.SUCCEEDED).withJobId(jobId1); + jm.submit(() -> { + try { + return job1.submit(finalizer, config); + } catch (JobException e) { + throw new RuntimeException("Something went wrong", e); + } + }, username1); jm.killJob(username1, jobId1); verify(job1).kill(); } diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliParser.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliParser.java index d5976ae7c9..07a1c3f79e 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliParser.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliParser.java @@ -19,6 +19,7 @@ package org.apache.metron.pcap.query; import org.apache.commons.cli.*; +import org.apache.metron.pcap.config.PcapConfig; /** * Provides commmon required fields for the PCAP filter jobs @@ -29,9 +30,9 @@ public class CliParser { public static final int NUM_REDUCERS_DEFAULT = 10; public static final int NUM_RECORDS_PER_FILE_DEFAULT = 10000; private CommandLineParser parser; - protected CliConfig.PrefixStrategy prefixStrategy; + protected PcapConfig.PrefixStrategy prefixStrategy; - public CliParser(CliConfig.PrefixStrategy prefixStrategy) { + public CliParser(PcapConfig.PrefixStrategy prefixStrategy) { this.prefixStrategy = prefixStrategy; parser = new PosixParser(); } @@ -59,7 +60,7 @@ protected Option newOption(String opt, String longOpt, boolean hasArg, String de return option; } - public void parse(CommandLine commandLine, CliConfig config) throws java.text.ParseException { + public void parse(CommandLine commandLine, PcapConfig config) throws java.text.ParseException { if (commandLine.hasOption("help")) { config.setShowHelp(true); } @@ -72,18 +73,18 @@ public void parse(CommandLine commandLine, CliConfig config) throws java.text.Pa config.setBasePath(BASE_PATH_DEFAULT); } if (commandLine.hasOption("base_output_path")) { - config.setBaseOutputPath(commandLine.getOptionValue("base_output_path")); + config.setInterimResultPath(commandLine.getOptionValue("base_output_path")); } else { - config.setBaseOutputPath(BASE_OUTPUT_PATH_DEFAULT); + config.setInterimResultPath(BASE_OUTPUT_PATH_DEFAULT); } if (commandLine.hasOption("start_time")) { try { if (commandLine.hasOption("date_format")) { long startTime = config.getDateFormat().parse(commandLine.getOptionValue("start_time")).getTime(); - config.setStartTime(startTime); + config.setStartTimeMs(startTime); } else { long startTime = Long.parseLong(commandLine.getOptionValue("start_time")); - config.setStartTime(startTime); + config.setStartTimeMs(startTime); } } catch (NumberFormatException nfe) { //no-op @@ -107,10 +108,10 @@ public void parse(CommandLine commandLine, CliConfig config) throws java.text.Pa try { if (commandLine.hasOption("date_format")) { long endTime = config.getDateFormat().parse(commandLine.getOptionValue("end_time")).getTime(); - config.setEndTime(endTime); + config.setEndTimeMs(endTime); } else { long endTime = Long.parseLong(commandLine.getOptionValue("end_time")); - config.setEndTime(endTime); + config.setEndTimeMs(endTime); } } catch (NumberFormatException nfe) { //no-op diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/FixedCliParser.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/FixedCliParser.java index 4e1bfcfe39..19d351c048 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/FixedCliParser.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/FixedCliParser.java @@ -22,11 +22,13 @@ import org.apache.commons.cli.ParseException; import org.apache.metron.common.Constants; import org.apache.metron.pcap.PcapHelper; +import org.apache.metron.pcap.config.FixedPcapConfig; +import org.apache.metron.pcap.config.PcapConfig; public class FixedCliParser extends CliParser { private Options fixedOptions; - public FixedCliParser(CliConfig.PrefixStrategy prefixStrategy) { + public FixedCliParser(PcapConfig.PrefixStrategy prefixStrategy) { super(prefixStrategy); fixedOptions = buildFixedOptions(); } @@ -51,9 +53,9 @@ private Options buildFixedOptions() { * @return Configuration tailored to fixed pcap queries * @throws ParseException */ - public FixedCliConfig parse(String[] args) throws ParseException, java.text.ParseException { + public FixedPcapConfig parse(String[] args) throws ParseException, java.text.ParseException { CommandLine commandLine = getParser().parse(fixedOptions, args); - FixedCliConfig config = new FixedCliConfig(prefixStrategy); + FixedPcapConfig config = new FixedPcapConfig(prefixStrategy); super.parse(commandLine, config); config.putFixedField(Constants.Fields.SRC_ADDR.getName(), commandLine.getOptionValue("ip_src_addr")); config.putFixedField(Constants.Fields.DST_ADDR.getName(), commandLine.getOptionValue("ip_dst_addr")); @@ -63,7 +65,7 @@ public FixedCliConfig parse(String[] args) throws ParseException, java.text.Pars config.putFixedField(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName(), Boolean.toString(commandLine.hasOption("include_reverse"))); config.putFixedField(PcapHelper.PacketFields.PACKET_FILTER.getName(), commandLine.getOptionValue("packet_filter")); if(commandLine.hasOption("prefix")) { - config.setPrefix(commandLine.getOptionValue("prefix")); + config.setFinalFilenamePrefix(commandLine.getOptionValue("prefix")); } return config; } diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java index 6ed299b4db..7b0b0a109a 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.Arrays; -import java.util.Map; import java.util.UUID; import org.apache.commons.cli.ParseException; import org.apache.commons.lang3.tuple.Pair; @@ -28,11 +27,13 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.util.GenericOptionsParser; -import org.apache.metron.common.system.Clock; import org.apache.metron.common.utils.timestamp.TimestampConverters; import org.apache.metron.job.JobException; import org.apache.metron.job.Pageable; -import org.apache.metron.pcap.ConfigOptions; +import org.apache.metron.pcap.PcapOptions; +import org.apache.metron.pcap.config.FixedPcapConfig; +import org.apache.metron.pcap.config.PcapConfig; +import org.apache.metron.pcap.config.QueryPcapConfig; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.finalizer.PcapFinalizerStrategies; import org.apache.metron.pcap.mr.PcapJob; @@ -41,20 +42,20 @@ public class PcapCli { private static final Logger LOGGER = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - public static final CliConfig.PrefixStrategy PREFIX_STRATEGY = clock -> { - String timestamp = new Clock().currentTimeFormatted("yyyyMMddHHmm"); + public static final PcapConfig.PrefixStrategy PREFIX_STRATEGY = clock -> { + String timestamp = clock.currentTimeFormatted("yyyyMMddHHmm"); String uuid = UUID.randomUUID().toString().replaceAll("-", ""); return String.format("%s-%s", timestamp, uuid); }; private final PcapJob jobRunner; - private final CliConfig.PrefixStrategy prefixStrategy; + private final PcapConfig.PrefixStrategy prefixStrategy; public static void main(String[] args) { int status = new PcapCli(new PcapJob(), PREFIX_STRATEGY).run(args); System.exit(status); } - public PcapCli(PcapJob jobRunner, CliConfig.PrefixStrategy prefixStrategy) { + public PcapCli(PcapJob jobRunner, PcapConfig.PrefixStrategy prefixStrategy) { this.jobRunner = jobRunner; this.prefixStrategy = prefixStrategy; } @@ -75,18 +76,18 @@ public int run(String[] args) { LOGGER.error("Failed to configure hadoop with provided options: {}", e.getMessage(), e); return -1; } - CliConfig commonConfig = null; + PcapConfig commonConfig = null; Pageable results; // write to local FS in the executing directory String execDir = System.getProperty("user.dir"); if ("fixed".equals(jobType)) { FixedCliParser fixedParser = new FixedCliParser(prefixStrategy); - FixedCliConfig config = null; + FixedPcapConfig config = null; try { config = fixedParser.parse(otherArgs); commonConfig = config; - ConfigOptions.FINAL_OUTPUT_PATH.put(commonConfig, new Path("file:///" + execDir)); + PcapOptions.FINAL_OUTPUT_PATH.put(commonConfig, new Path("file:///" + execDir)); } catch (ParseException | java.text.ParseException e) { System.err.println(e.getMessage()); System.err.flush(); @@ -97,19 +98,16 @@ public int run(String[] args) { fixedParser.printHelp(); return 0; } - Pair time = timeAsNanosecondsSinceEpoch(config.getStartTime(), config.getEndTime()); + Pair time = timeAsNanosecondsSinceEpoch(config.getStartTimeMs(), config.getEndTimeMs()); long startTime = time.getLeft(); long endTime = time.getRight(); - //TODO: Figure out how to do jobname here..it's missing. - //TODO: Figure out how to do final prefix filename here, it's missing, I think. - - ConfigOptions.START_TIME_NS.put(commonConfig, startTime); - ConfigOptions.END_TIME_NS.put(commonConfig, endTime); - ConfigOptions.FILTER_IMPL.put(commonConfig, new FixedPcapFilter.Configurator()); - ConfigOptions.HADOOP_CONF.put(commonConfig, hadoopConf); + PcapOptions.START_TIME_NS.put(commonConfig, startTime); + PcapOptions.END_TIME_NS.put(commonConfig, endTime); + PcapOptions.FILTER_IMPL.put(commonConfig, new FixedPcapFilter.Configurator()); + PcapOptions.HADOOP_CONF.put(commonConfig, hadoopConf); try { - ConfigOptions.FILESYSTEM.put(commonConfig, FileSystem.get(hadoopConf)); + PcapOptions.FILESYSTEM.put(commonConfig, FileSystem.get(hadoopConf)); results = jobRunner.submit(PcapFinalizerStrategies.CLI, commonConfig).get(); } catch (IOException|InterruptedException | JobException e) { LOGGER.error("Failed to execute fixed filter job: {}", e.getMessage(), e); @@ -117,7 +115,7 @@ public int run(String[] args) { } } else if ("query".equals(jobType)) { QueryCliParser queryParser = new QueryCliParser(prefixStrategy); - QueryCliConfig config = null; + QueryPcapConfig config = null; try { config = queryParser.parse(otherArgs); commonConfig = config; @@ -130,19 +128,16 @@ public int run(String[] args) { queryParser.printHelp(); return 0; } - Pair time = timeAsNanosecondsSinceEpoch(config.getStartTime(), config.getEndTime()); + Pair time = timeAsNanosecondsSinceEpoch(config.getStartTimeMs(), config.getEndTimeMs()); long startTime = time.getLeft(); long endTime = time.getRight(); - //TODO: Figure out how to do jobname here..it's missing. - //TODO: Figure out how to do final prefix filename here, it's missing, I think. - - ConfigOptions.START_TIME_NS.put(commonConfig, startTime); - ConfigOptions.END_TIME_NS.put(commonConfig, endTime); - ConfigOptions.FILTER_IMPL.put(commonConfig, new FixedPcapFilter.Configurator()); - ConfigOptions.HADOOP_CONF.put(commonConfig, hadoopConf); + PcapOptions.START_TIME_NS.put(commonConfig, startTime); + PcapOptions.END_TIME_NS.put(commonConfig, endTime); + PcapOptions.FILTER_IMPL.put(commonConfig, new FixedPcapFilter.Configurator()); + PcapOptions.HADOOP_CONF.put(commonConfig, hadoopConf); try { - ConfigOptions.FILESYSTEM.put(commonConfig, FileSystem.get(hadoopConf)); + PcapOptions.FILESYSTEM.put(commonConfig, FileSystem.get(hadoopConf)); results = jobRunner.submit(PcapFinalizerStrategies.CLI, commonConfig).get(); } catch (IOException| InterruptedException | JobException e) { LOGGER.error("Failed to execute fixed filter job: {}", e.getMessage(), e); diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/QueryCliParser.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/QueryCliParser.java index d6e5cd19ea..b4375d1b22 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/QueryCliParser.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/QueryCliParser.java @@ -20,11 +20,13 @@ import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; +import org.apache.metron.pcap.config.PcapConfig; +import org.apache.metron.pcap.config.QueryPcapConfig; public class QueryCliParser extends CliParser { private Options queryOptions; - public QueryCliParser(CliConfig.PrefixStrategy prefixStrategy) { + public QueryCliParser(PcapConfig.PrefixStrategy prefixStrategy) { super(prefixStrategy); queryOptions = setupOptions(); } @@ -43,15 +45,15 @@ private Options setupOptions() { * @return Configuration tailored to query pcap queries * @throws ParseException */ - public QueryCliConfig parse(String[] args) throws ParseException, java.text.ParseException { + public QueryPcapConfig parse(String[] args) throws ParseException, java.text.ParseException { CommandLine commandLine = getParser().parse(queryOptions, args); - QueryCliConfig config = new QueryCliConfig(prefixStrategy); + QueryPcapConfig config = new QueryPcapConfig(prefixStrategy); super.parse(commandLine, config); if (commandLine.hasOption("query")) { config.setQuery(commandLine.getOptionValue("query")); } if(commandLine.hasOption("prefix")) { - config.setPrefix(commandLine.getOptionValue("prefix")); + config.setFinalFilenamePrefix(commandLine.getOptionValue("prefix")); } return config; } diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java index 4560bd9bda..385887218e 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java @@ -43,6 +43,7 @@ import org.apache.metron.pcap.filter.PcapFilterConfigurator; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.mr.PcapJob; +import org.apache.metron.pcap.config.FixedPcapConfig; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -61,7 +62,7 @@ public class PcapJobTest { private Finalizer finalizer; @Mock private Pageable pageableResult; - private Map config; + private FixedPcapConfig config; private Configuration hadoopConfig; private FileSystem fileSystem; private String jobIdVal = "job_abc_123"; @@ -72,7 +73,6 @@ public class PcapJobTest { private int numReducers; private int numRecordsPerFile; private Path finalOutputPath; - private String prefix; private Map fixedFields; private PcapJob> testJob; @@ -91,22 +91,21 @@ public void setup() throws IOException { hadoopConfig = new Configuration(); fileSystem = FileSystem.get(hadoopConfig); finalOutputPath = new Path("finaloutpath"); - prefix = "someprefix"; when(jobId.toString()).thenReturn(jobIdVal); when(mrStatus.getJobID()).thenReturn(jobId); - config = new HashMap<>(); - config.put("hadoopConf", hadoopConfig); - config.put("fileSystem", FileSystem.get(hadoopConfig)); - config.put("basePath", basePath); - config.put("baseInterimResultPath", baseOutPath); - config.put("beginNS", startTime); - config.put("endNS", endTime); - config.put("numReducers", numReducers); - config.put("fields", fixedFields); - config.put("filterImpl", new FixedPcapFilter.Configurator()); - config.put("numRecordsPerFile", numRecordsPerFile); - config.put("finalOutputPath", finalOutputPath); - config.put("finalFilenamePrefix", prefix); + // handles setting the file name prefix under the hood + config = new FixedPcapConfig(clock -> "clockprefix"); + PcapOptions.HADOOP_CONF.put(config, hadoopConfig); + PcapOptions.FILESYSTEM.put(config, FileSystem.get(hadoopConfig)); + PcapOptions.BASE_PATH.put(config, basePath); + PcapOptions.INTERIM_RESULT_PATH.put(config, baseOutPath); + PcapOptions.START_TIME_NS.put(config, startTime); + PcapOptions.END_TIME_NS.put(config, endTime); + PcapOptions.NUM_REDUCERS.put(config, numReducers); + PcapOptions.FIELDS.put(config, fixedFields); + PcapOptions.FILTER_IMPL.put(config, new FixedPcapFilter.Configurator()); + PcapOptions.NUM_RECORDS_PER_FILE.put(config, numRecordsPerFile); + PcapOptions.FINAL_OUTPUT_PATH.put(config, finalOutputPath); testJob = new TestJob<>(); testJob.setStatusInterval(10); testJob.setCompleteCheckInterval(10); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java index de82015ada..c2547daf53 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java @@ -24,7 +24,7 @@ public class PcapFiles implements Pageable { - private List files; + private final List files; public PcapFiles(List files) { this.files = files; diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapOptions.java similarity index 64% rename from metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java rename to metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapOptions.java index 96f9f7b083..bf6b46cecd 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/ConfigOptions.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapOptions.java @@ -18,42 +18,41 @@ package org.apache.metron.pcap; -import org.apache.hadoop.fs.Path; - import java.util.function.BiFunction; +import org.apache.hadoop.fs.Path; +import org.apache.metron.common.configuration.ConfigOption; -public enum ConfigOptions implements ConfigOption { - PREFIX("prefix"), - FINAL_FILENAME_PREFIX("finalFilenamePrefix"), +public enum PcapOptions implements ConfigOption { JOB_NAME("jobName"), - FINAL_OUTPUT_PATH("finalOutputPath"), - BASE_PATH("basePath", (s,o) -> o == null?null:new Path(o.toString())), - INTERRIM_RESULT_PATH("interimResultPath", (s,o) -> o == null?null:new Path(o.toString())), + FINAL_FILENAME_PREFIX("finalFilenamePrefix"), + BASE_PATH("basePath", (s, o) -> o == null ? null : new Path(o.toString())), + INTERIM_RESULT_PATH("interimResultPath", (s, o) -> o == null ? null : new Path(o.toString())), + FINAL_OUTPUT_PATH("finalOutputPath", (s, o) -> o == null ? null : new Path(o.toString())), NUM_REDUCERS("numReducers"), - START_TIME("startTime"), - END_TIME("endTime"), - START_TIME_NS("startNs"), - END_TIME_NS("endNs"), + START_TIME_MS("startTimeMs"), + END_TIME_MS("endTimeMs"), + START_TIME_NS("startTimeNs"), + END_TIME_NS("endTimeNs"), NUM_RECORDS_PER_FILE("numRecordsPerFile"), FIELDS("fields"), FILTER_IMPL("filterImpl"), HADOOP_CONF("hadoopConf"), - FILESYSTEM("fileSystem") - ; + FILESYSTEM("fileSystem"); - public static final BiFunction STRING_TO_PATH = (s,o) -> o == null?null:new Path(o.toString()); - String key; - BiFunction transform = (s, o) -> o; + public static final BiFunction STRING_TO_PATH = + (s, o) -> o == null ? null : new Path(o.toString()); + private String key; + private BiFunction transform = (s, o) -> o; - ConfigOptions(String key) { + PcapOptions(String key) { this.key = key; } - ConfigOptions(String key, BiFunction transform ) { + + PcapOptions(String key, BiFunction transform) { this.key = key; this.transform = transform; } - @Override public String getKey() { return key; diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/FixedCliConfig.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/FixedPcapConfig.java similarity index 77% rename from metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/FixedCliConfig.java rename to metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/FixedPcapConfig.java index d0deb48943..2bd93f41ce 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/FixedCliConfig.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/FixedPcapConfig.java @@ -15,30 +15,30 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.metron.pcap.query; +package org.apache.metron.pcap.config; -import org.apache.metron.pcap.ConfigOptions; +import org.apache.metron.pcap.PcapOptions; import java.util.LinkedHashMap; import java.util.Map; -public class FixedCliConfig extends CliConfig { +public class FixedPcapConfig extends PcapConfig { - public FixedCliConfig(PrefixStrategy prefixStrategy) { + public FixedPcapConfig(PrefixStrategy prefixStrategy) { super(prefixStrategy); setFixedFields(new LinkedHashMap<>()); } public Map getFixedFields() { - return ConfigOptions.FIELDS.get(this, Map.class); + return PcapOptions.FIELDS.get(this, Map.class); } public void setFixedFields(Map fixedFields) { - ConfigOptions.FIELDS.put(this, fixedFields); + PcapOptions.FIELDS.put(this, fixedFields); } public void putFixedField(String key, String value) { - Map fixedFields = ConfigOptions.FIELDS.get(this, Map.class); + Map fixedFields = PcapOptions.FIELDS.get(this, Map.class); String trimmedVal = value != null ? value.trim() : null; if (!isNullOrEmpty(trimmedVal)) { fixedFields.put(key, value); diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliConfig.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java similarity index 57% rename from metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliConfig.java rename to metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java index 21fdd623f1..5013e8e980 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliConfig.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java @@ -15,38 +15,38 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.metron.pcap.query; +package org.apache.metron.pcap.config; import org.apache.commons.collections4.map.AbstractMapDecorator; import org.apache.commons.lang3.StringUtils; import org.apache.metron.common.system.Clock; -import org.apache.metron.pcap.ConfigOption; -import org.apache.metron.pcap.ConfigOptions; +import org.apache.metron.common.configuration.ConfigOption; +import org.apache.metron.pcap.PcapOptions; import java.text.DateFormat; import java.text.SimpleDateFormat; import java.util.HashMap; import java.util.function.Function; -public class CliConfig extends AbstractMapDecorator{ +public class PcapConfig extends AbstractMapDecorator{ public interface PrefixStrategy extends Function{} private boolean showHelp; private DateFormat dateFormat; - public CliConfig() { + public PcapConfig() { super(new HashMap<>()); } - public CliConfig(PrefixStrategy prefixStrategy) { + public PcapConfig(PrefixStrategy prefixStrategy) { this(); setShowHelp(false); setBasePath(""); - setBaseOutputPath(""); - setStartTime(-1L); - setEndTime(-1L); + setInterimResultPath(""); + setStartTimeMs(-1L); + setEndTimeMs(-1L); setNumReducers(0); - setPrefix(prefixStrategy.apply(new Clock())); + setFinalFilenamePrefix(prefixStrategy.apply(new Clock())); } public Object getOption(ConfigOption option) { @@ -54,16 +54,16 @@ public Object getOption(ConfigOption option) { return option.transform().apply(option.getKey(), o); } - public String getPrefix() { - return ConfigOptions.PREFIX.get(this, String.class); + public String getFinalFilenamePrefix() { + return PcapOptions.FINAL_FILENAME_PREFIX.get(this, String.class); } - public void setPrefix(String prefix) { - ConfigOptions.PREFIX.put(this, prefix); + public void setFinalFilenamePrefix(String prefix) { + PcapOptions.FINAL_FILENAME_PREFIX.put(this, prefix); } public int getNumReducers() { - return ConfigOptions.NUM_REDUCERS.get(this, Integer.class); + return PcapOptions.NUM_REDUCERS.get(this, Integer.class); } public boolean showHelp() { @@ -75,35 +75,35 @@ public void setShowHelp(boolean showHelp) { } public String getBasePath() { - return ConfigOptions.BASE_PATH.get(this, String.class); + return PcapOptions.BASE_PATH.get(this, String.class); } - public String getBaseOutputPath() { - return ConfigOptions.INTERRIM_RESULT_PATH.get(this, String.class); + public String getInterimResultPath() { + return PcapOptions.INTERIM_RESULT_PATH.get(this, String.class); } - public long getStartTime() { - return ConfigOptions.START_TIME.get(this, Long.class); + public long getStartTimeMs() { + return PcapOptions.START_TIME_MS.get(this, Long.class); } - public long getEndTime() { - return ConfigOptions.END_TIME.get(this, Long.class); + public long getEndTimeMs() { + return PcapOptions.END_TIME_MS.get(this, Long.class); } public void setBasePath(String basePath) { - ConfigOptions.BASE_PATH.put(this, basePath); + PcapOptions.BASE_PATH.put(this, basePath); } - public void setBaseOutputPath(String baseOutputPath) { - ConfigOptions.INTERRIM_RESULT_PATH.put(this, baseOutputPath); + public void setInterimResultPath(String baseOutputPath) { + PcapOptions.INTERIM_RESULT_PATH.put(this, baseOutputPath); } - public void setStartTime(long startTime) { - ConfigOptions.START_TIME.put(this, startTime); + public void setStartTimeMs(long startTime) { + PcapOptions.START_TIME_MS.put(this, startTime); } - public void setEndTime(long endTime) { - ConfigOptions.END_TIME.put(this, endTime); + public void setEndTimeMs(long endTime) { + PcapOptions.END_TIME_MS.put(this, endTime); } public boolean isNullOrEmpty(String val) { @@ -119,14 +119,14 @@ public DateFormat getDateFormat() { } public void setNumReducers(int numReducers) { - ConfigOptions.NUM_REDUCERS.put(this, numReducers); + PcapOptions.NUM_REDUCERS.put(this, numReducers); } public int getNumRecordsPerFile() { - return ConfigOptions.NUM_RECORDS_PER_FILE.get(this, Integer.class); + return PcapOptions.NUM_RECORDS_PER_FILE.get(this, Integer.class); } public void setNumRecordsPerFile(int numRecordsPerFile) { - ConfigOptions.NUM_RECORDS_PER_FILE.put(this, numRecordsPerFile); + PcapOptions.NUM_RECORDS_PER_FILE.put(this, numRecordsPerFile); } } diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/QueryCliConfig.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/QueryPcapConfig.java similarity index 76% rename from metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/QueryCliConfig.java rename to metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/QueryPcapConfig.java index f18de4ccfc..6158509c32 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/QueryCliConfig.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/QueryPcapConfig.java @@ -15,21 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.metron.pcap.query; +package org.apache.metron.pcap.config; -import org.apache.metron.pcap.ConfigOptions; +import org.apache.metron.pcap.PcapOptions; -public class QueryCliConfig extends CliConfig { +public class QueryPcapConfig extends PcapConfig { - public QueryCliConfig(PrefixStrategy prefixStrategy) { + public QueryPcapConfig(PrefixStrategy prefixStrategy) { super(prefixStrategy); } public String getQuery() { - return ConfigOptions.FIELDS.get(this, String.class); + return PcapOptions.FIELDS.get(this, String.class); } public void setQuery(String query) { - ConfigOptions.FIELDS.put(this, query); + PcapOptions.FIELDS.put(this, query); } } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java index 1666618184..f506b21b1c 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java @@ -20,7 +20,7 @@ import java.util.Map; import org.apache.hadoop.fs.Path; -import org.apache.metron.pcap.ConfigOptions; +import org.apache.metron.pcap.PcapOptions; /** * Write to local FS. @@ -29,8 +29,8 @@ public class PcapCliFinalizer extends PcapFinalizer { @Override protected String getOutputFileName(Map config, int partition) { - Path finalOutputPath = ConfigOptions.FINAL_OUTPUT_PATH.getTransformed(config, Path.class); - String prefix = ConfigOptions.FINAL_FILENAME_PREFIX.get(config, String.class); + Path finalOutputPath = PcapOptions.FINAL_OUTPUT_PATH.get(config, PcapOptions.STRING_TO_PATH, Path.class); + String prefix = PcapOptions.FINAL_FILENAME_PREFIX.get(config, String.class); return String.format("%s/pcap-data-%s+%04d.pcap", finalOutputPath, prefix, partition); } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java index 03bcfa4e3e..d7e01af64e 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java @@ -34,8 +34,8 @@ import org.apache.metron.job.Finalizer; import org.apache.metron.job.JobException; import org.apache.metron.job.Pageable; -import org.apache.metron.pcap.ConfigOptions; import org.apache.metron.pcap.PcapFiles; +import org.apache.metron.pcap.PcapOptions; import org.apache.metron.pcap.writer.PcapResultsWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,11 +55,11 @@ protected PcapResultsWriter getResultsWriter() { @Override public Pageable finalizeJob(Map config) throws JobException { - Configuration hadoopConfig = ConfigOptions.HADOOP_CONF.get(config, Configuration.class); - int recPerFile = ConfigOptions.NUM_RECORDS_PER_FILE.get(config, Integer.class); - Path interimResultPath = ConfigOptions.INTERRIM_RESULT_PATH.get(config, ConfigOptions.STRING_TO_PATH, Path.class); - Path finalOutputPath = ConfigOptions.FINAL_OUTPUT_PATH.get(config, ConfigOptions.STRING_TO_PATH, Path.class); - FileSystem fs = ConfigOptions.FILESYSTEM.get(config, FileSystem.class); + Configuration hadoopConfig = PcapOptions.HADOOP_CONF.get(config, Configuration.class); + int recPerFile = PcapOptions.NUM_RECORDS_PER_FILE.get(config, Integer.class); + Path interimResultPath = PcapOptions.INTERIM_RESULT_PATH + .get(config, PcapOptions.STRING_TO_PATH, Path.class); + FileSystem fs = PcapOptions.FILESYSTEM.get(config, FileSystem.class); SequenceFileIterable interimResults = null; try { diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java index fa8511b845..de7e78016b 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java @@ -20,7 +20,7 @@ import java.util.Map; import org.apache.hadoop.fs.Path; -import org.apache.metron.pcap.ConfigOptions; +import org.apache.metron.pcap.PcapOptions; /** * Write to HDFS. @@ -29,7 +29,7 @@ public class PcapRestFinalizer extends PcapFinalizer { @Override protected String getOutputFileName(Map config, int partition) { - Path finalOutputPath = ConfigOptions.FINAL_OUTPUT_PATH.getTransformed(config, Path.class); + Path finalOutputPath = PcapOptions.FINAL_OUTPUT_PATH.getTransformed(config, Path.class); return String.format("%s/page-%s", finalOutputPath, partition); } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index c3fe913506..e2fd561efc 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -56,9 +56,9 @@ import org.apache.metron.job.JobStatus.State; import org.apache.metron.job.Pageable; import org.apache.metron.job.Statusable; -import org.apache.metron.pcap.ConfigOptions; import org.apache.metron.pcap.PacketInfo; import org.apache.metron.pcap.PcapHelper; +import org.apache.metron.pcap.PcapOptions; import org.apache.metron.pcap.filter.PcapFilter; import org.apache.metron.pcap.filter.PcapFilterConfigurator; import org.apache.metron.pcap.filter.PcapFilters; @@ -76,15 +76,13 @@ public class PcapJob implements Statusable { public static final String START_TS_CONF = "start_ts"; public static final String END_TS_CONF = "end_ts"; public static final String WIDTH_CONF = "width"; - public static final long THREE_SECONDS = 3000; - public static final long ONE_SECOND = 1000; - private Job job; // store a running MR job reference for async status check - private JobStatus jobStatus; - private boolean finalized; // job results have been finalized - private Pageable jobResults; + private static final long THREE_SECONDS = 3000; + private static final long ONE_SECOND = 1000; + private Job mrJob; // store a running MR job reference for async status check + private volatile State jobState; // overall job state, including finalization step private Finalizer finalizer; private Map configuration; - private Pageable results; + private Pageable finalResults; private Timer timer; private long statusInterval; // how often timer thread checks job status. private long completeCheckInterval; // how long we sleep between isDone checks in get() @@ -182,7 +180,6 @@ protected void reduce(LongWritable key, Iterable values, Context } public PcapJob() { - jobStatus = new JobStatus(); statusInterval = THREE_SECONDS; completeCheckInterval = ONE_SECOND; } @@ -210,16 +207,16 @@ public Statusable submit(Finalizer finalizer, Map co throws JobException { this.finalizer = finalizer; this.configuration = configuration; - Optional jobName = Optional.ofNullable(ConfigOptions.JOB_NAME.get(configuration, String.class)); - Configuration hadoopConf = ConfigOptions.HADOOP_CONF.get(configuration, Configuration.class); - FileSystem fileSystem = ConfigOptions.FILESYSTEM.get(configuration, FileSystem.class); - Path basePath = ConfigOptions.BASE_PATH.getTransformed(configuration, Path.class); - Path baseInterimResultPath = ConfigOptions.INTERRIM_RESULT_PATH.getTransformed(configuration, Path.class); - long startTime = ConfigOptions.START_TIME_NS.get(configuration, Long.class); - long endTime = ConfigOptions.END_TIME_NS.get(configuration, Long.class); - int numReducers = ConfigOptions.NUM_REDUCERS.get(configuration, Integer.class); - T fields = (T) ConfigOptions.FIELDS.get(configuration, Object.class); - PcapFilterConfigurator filterImpl = ConfigOptions.FILTER_IMPL.get(configuration, PcapFilterConfigurator.class); + Optional jobName = Optional.ofNullable(PcapOptions.JOB_NAME.get(configuration, String.class)); + Configuration hadoopConf = PcapOptions.HADOOP_CONF.get(configuration, Configuration.class); + FileSystem fileSystem = PcapOptions.FILESYSTEM.get(configuration, FileSystem.class); + Path basePath = PcapOptions.BASE_PATH.getTransformed(configuration, Path.class); + Path baseInterimResultPath = PcapOptions.INTERIM_RESULT_PATH.getTransformed(configuration, Path.class); + long startTime = PcapOptions.START_TIME_NS.get(configuration, Long.class); + long endTime = PcapOptions.END_TIME_NS.get(configuration, Long.class); + int numReducers = PcapOptions.NUM_REDUCERS.get(configuration, Integer.class); + T fields = (T) PcapOptions.FIELDS.get(configuration, Object.class); + PcapFilterConfigurator filterImpl = PcapOptions.FILTER_IMPL.get(configuration, PcapFilterConfigurator.class); try { return query(jobName, @@ -261,7 +258,7 @@ public Statusable query(Optional jobName, LOG.debug("Executing query {} on timerange from {} to {}", filterImpl.queryToString(fields), from, to); } Path interimResultPath = new Path(baseInterimResultPath, outputDirName); - job = createJob(jobName + mrJob = createJob(jobName , basePath , interimResultPath , beginNS @@ -272,7 +269,7 @@ public Statusable query(Optional jobName, , fs , filterImpl ); - job.submit(); + mrJob.submit(); startJobStatusTimerThread(statusInterval); return this; } @@ -283,23 +280,40 @@ private void startJobStatusTimerThread(long interval) { @Override public void run() { try { - getStatus(); - if (job.isComplete() && !finalized) { - results = finalizer.finalizeJob(configuration); - finalized = true; - jobStatus.withState(State.SUCCEEDED); - cancel(); // be gone, ye! + synchronized (this) { + if (jobState == State.RUNNING) { + if (mrJob.isComplete()) { + switch (mrJob.getStatus().getState()) { + case SUCCEEDED: + jobState = State.FINALIZING; + setFinalResults(finalizer.finalizeJob(configuration)); + jobState = State.SUCCEEDED; + break; + case FAILED: + jobState = State.FAILED; + break; + case KILLED: + jobState = State.KILLED; + break; + } + } + cancel(); // be gone, ye! + } } - } catch (IOException | JobException e) { + } catch (InterruptedException | IOException | JobException e) { + jobState = State.FAILED; throw new RuntimeException("Error getting job status.", e); } - } }, interval, interval); } + private synchronized void setFinalResults(Pageable results) { + finalResults = results; + } + /** - * Creates, but does not submit the job. This is the core MapReduce job. + * Creates, but does not submit the job. This is the core MapReduce mrJob. */ public Job createJob(Optional jobName ,Path basePath @@ -360,71 +374,82 @@ public JobType getJobType() { @Override public JobStatus getStatus() throws JobException { - // Note: this method is only reading state from the underlying job, so locking not needed - if (job == null) { - jobStatus.withPercentComplete(100).withState(State.SUCCEEDED); + JobStatus status = new JobStatus(); + if (mrJob == null) { + status.withPercentComplete(100).withState(State.SUCCEEDED); } else { try { - jobStatus.withJobId(job.getStatus().getJobID().toString()); - if (!finalized) { - if (job.isComplete()) { - jobStatus.withPercentComplete(100); - switch (job.getStatus().getState()) { - case SUCCEEDED: - jobStatus.withState(State.FINALIZING).withDescription(State.FINALIZING.toString()); - break; - case FAILED: - jobStatus.withState(State.FAILED).withDescription(State.FAILED.toString()); - break; - case KILLED: - jobStatus.withState(State.KILLED).withDescription(State.KILLED.toString()); - break; - default: - throw new IllegalStateException( - "Unknown job state reported as 'complete' by mapreduce framework: " + job - .getStatus().getState()); - } + synchronized (this) { + org.apache.hadoop.mapreduce.JobStatus mrJobStatus = mrJob.getStatus(); + status.withJobId(mrJobStatus.getJobID().toString()); + if (jobState == State.SUCCEEDED) { + status.withPercentComplete(100).withState(State.SUCCEEDED) + .withDescription("Job complete"); } else { - float mapProg = job.mapProgress(); - float reduceProg = job.reduceProgress(); - float totalProgress = ((mapProg / 2) + (reduceProg / 2)) * 100; - String description = String - .format("map: %s%%, reduce: %s%%", mapProg * 100, reduceProg * 100); - jobStatus.withPercentComplete(totalProgress).withState(State.RUNNING) - .withDescription(description); + if (mrJob.isComplete()) { + status.withPercentComplete(100); + switch (mrJobStatus.getState()) { + case SUCCEEDED: + status.withState(State.FINALIZING).withDescription(State.FINALIZING.toString()); + break; + case FAILED: + status.withState(State.FAILED).withDescription(State.FAILED.toString()); + break; + case KILLED: + status.withState(State.KILLED).withDescription(State.KILLED.toString()); + break; + default: + throw new IllegalStateException( + "Unknown job state reported as 'complete' by mapreduce framework: " + mrJobStatus.getState()); + } + } else { + float mapProg = mrJob.mapProgress(); + float reduceProg = mrJob.reduceProgress(); + float totalProgress = ((mapProg / 2) + (reduceProg / 2)) * 100; + String description = String + .format("map: %s%%, reduce: %s%%", mapProg * 100, reduceProg * 100); + status.withPercentComplete(totalProgress).withState(State.RUNNING) + .withDescription(description); + } } } } catch (Exception e) { - throw new RuntimeException("Error occurred while attempting to retrieve job status.", e); + throw new JobException("Error occurred while attempting to retrieve job status.", e); } } - return jobStatus; + return status; } + /** + * Synchronous call blocks until completion. + */ @Override public Pageable get() throws JobException, InterruptedException { for (; ; ) { - getStatus(); - if (jobStatus.getState() == State.SUCCEEDED) { - return results; - } else if (jobStatus.getState() == State.KILLED - || jobStatus.getState() == State.FAILED) { + JobStatus status = getStatus(); + if (status.getState() == State.SUCCEEDED) { + return getFinalResults(); + } else if (status.getState() == State.KILLED + || status.getState() == State.FAILED) { return null; } Thread.sleep(completeCheckInterval); } } + private synchronized Pageable getFinalResults() { + return finalResults; + } + @Override - public boolean isDone() throws JobException { - getStatus(); + public boolean isDone() { return finalized; } @Override public void kill() throws JobException { try { - job.killJob(); + mrJob.killJob(); } catch (IOException e) { throw new JobException("Unable to kill pcap job.", e); } diff --git a/metron-platform/pom.xml b/metron-platform/pom.xml index c373dede68..27a994190c 100644 --- a/metron-platform/pom.xml +++ b/metron-platform/pom.xml @@ -52,7 +52,6 @@ metron-pcap metron-integration-test metron-test-utilities - metron-api metron-indexing metron-management metron-writer From 69cb9640fab4cb645e62923565f71c37f083da9d Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Fri, 13 Jul 2018 13:21:21 -0600 Subject: [PATCH 19/36] Move pcapoptions to config package --- .../apache/metron/rest/model/pcap/PcapRequest.java | 2 +- .../apache/metron/job/manager/InMemoryJobManager.java | 11 +++++++++-- .../java/org/apache/metron/pcap/query/PcapCli.java | 2 +- .../test/java/org/apache/metron/pcap/PcapJobTest.java | 1 + .../metron/pcap/{PcapFiles.java => PcapPages.java} | 4 ++-- .../apache/metron/pcap/config/FixedPcapConfig.java | 2 -- .../org/apache/metron/pcap/config/PcapConfig.java | 1 - .../apache/metron/pcap/{ => config}/PcapOptions.java | 2 +- .../apache/metron/pcap/config/QueryPcapConfig.java | 2 -- .../metron/pcap/finalizer/PcapCliFinalizer.java | 2 +- .../apache/metron/pcap/finalizer/PcapFinalizer.java | 6 +++--- .../metron/pcap/finalizer/PcapRestFinalizer.java | 2 +- .../main/java/org/apache/metron/pcap/mr/PcapJob.java | 8 +++++--- 13 files changed, 25 insertions(+), 20 deletions(-) rename metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/{PcapFiles.java => PcapPages.java} (93%) rename metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/{ => config}/PcapOptions.java (98%) diff --git a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java index b65bb1b91f..cfc212aa84 100644 --- a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java +++ b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java @@ -20,7 +20,7 @@ // TODO reconcile with pcapmrjob import org.apache.commons.collections4.map.AbstractMapDecorator; -import org.apache.metron.pcap.PcapOptions; +import org.apache.metron.pcap.config.PcapOptions; public class PcapRequest extends AbstractMapDecorator { diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java index 593ee82829..f80a474502 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java @@ -19,7 +19,9 @@ package org.apache.metron.job.manager; import java.lang.invoke.MethodHandles; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.function.Supplier; import org.apache.metron.job.JobException; @@ -33,13 +35,13 @@ public class InMemoryJobManager implements JobManager { private Map>> jobs; public InMemoryJobManager() { - this.jobs = new HashMap<>(); + this.jobs = Collections.synchronizedMap(new HashMap<>()); } @Override public JobStatus submit(Supplier> jobSupplier, String username) throws JobException { Statusable job = jobSupplier.get(); - Map> userJobs = new HashMap<>(); + Map> userJobs = Collections.synchronizedMap(new HashMap<>()); userJobs.put(job.getStatus().getJobId(), job); jobs.put(username, userJobs); return job.getStatus(); @@ -65,4 +67,9 @@ public Statusable getJob(String username, String jobId) throws JobExcept return null; } + @Override + public List> getJobs(String username, String jobId) throws JobException { + return null; + } + } diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java index 7b0b0a109a..34629215f1 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java @@ -30,7 +30,7 @@ import org.apache.metron.common.utils.timestamp.TimestampConverters; import org.apache.metron.job.JobException; import org.apache.metron.job.Pageable; -import org.apache.metron.pcap.PcapOptions; +import org.apache.metron.pcap.config.PcapOptions; import org.apache.metron.pcap.config.FixedPcapConfig; import org.apache.metron.pcap.config.PcapConfig; import org.apache.metron.pcap.config.QueryPcapConfig; diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java index 385887218e..1a3e6e6f37 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java @@ -40,6 +40,7 @@ import org.apache.metron.job.JobStatus.State; import org.apache.metron.job.Pageable; import org.apache.metron.job.Statusable; +import org.apache.metron.pcap.config.PcapOptions; import org.apache.metron.pcap.filter.PcapFilterConfigurator; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.mr.PcapJob; diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapPages.java similarity index 93% rename from metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java rename to metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapPages.java index c2547daf53..6f027aea8c 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapFiles.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapPages.java @@ -22,11 +22,11 @@ import org.apache.hadoop.fs.Path; import org.apache.metron.job.Pageable; -public class PcapFiles implements Pageable { +public class PcapPages implements Pageable { private final List files; - public PcapFiles(List files) { + public PcapPages(List files) { this.files = files; } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/FixedPcapConfig.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/FixedPcapConfig.java index 2bd93f41ce..c40407b0c5 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/FixedPcapConfig.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/FixedPcapConfig.java @@ -17,8 +17,6 @@ */ package org.apache.metron.pcap.config; -import org.apache.metron.pcap.PcapOptions; - import java.util.LinkedHashMap; import java.util.Map; diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java index 5013e8e980..20b9a0eff7 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java @@ -21,7 +21,6 @@ import org.apache.commons.lang3.StringUtils; import org.apache.metron.common.system.Clock; import org.apache.metron.common.configuration.ConfigOption; -import org.apache.metron.pcap.PcapOptions; import java.text.DateFormat; import java.text.SimpleDateFormat; diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapOptions.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java similarity index 98% rename from metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapOptions.java rename to metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java index bf6b46cecd..633378a059 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapOptions.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.metron.pcap; +package org.apache.metron.pcap.config; import java.util.function.BiFunction; import org.apache.hadoop.fs.Path; diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/QueryPcapConfig.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/QueryPcapConfig.java index 6158509c32..ef32839039 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/QueryPcapConfig.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/QueryPcapConfig.java @@ -17,8 +17,6 @@ */ package org.apache.metron.pcap.config; -import org.apache.metron.pcap.PcapOptions; - public class QueryPcapConfig extends PcapConfig { public QueryPcapConfig(PrefixStrategy prefixStrategy) { diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java index f506b21b1c..ecdce4ecbf 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java @@ -20,7 +20,7 @@ import java.util.Map; import org.apache.hadoop.fs.Path; -import org.apache.metron.pcap.PcapOptions; +import org.apache.metron.pcap.config.PcapOptions; /** * Write to local FS. diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java index d7e01af64e..a2fada9b3b 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java @@ -34,8 +34,8 @@ import org.apache.metron.job.Finalizer; import org.apache.metron.job.JobException; import org.apache.metron.job.Pageable; -import org.apache.metron.pcap.PcapFiles; -import org.apache.metron.pcap.PcapOptions; +import org.apache.metron.pcap.PcapPages; +import org.apache.metron.pcap.config.PcapOptions; import org.apache.metron.pcap.writer.PcapResultsWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -91,7 +91,7 @@ public Pageable finalizeJob(Map config) throws JobExceptio LOG.warn("Unable to cleanup files in HDFS", e); } } - return new PcapFiles(outFiles); + return new PcapPages(outFiles); } protected abstract String getOutputFileName(Map config, int partition); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java index de7e78016b..665b103370 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java @@ -20,7 +20,7 @@ import java.util.Map; import org.apache.hadoop.fs.Path; -import org.apache.metron.pcap.PcapOptions; +import org.apache.metron.pcap.config.PcapOptions; /** * Write to HDFS. diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index e2fd561efc..8d343fe9ef 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -58,7 +58,7 @@ import org.apache.metron.job.Statusable; import org.apache.metron.pcap.PacketInfo; import org.apache.metron.pcap.PcapHelper; -import org.apache.metron.pcap.PcapOptions; +import org.apache.metron.pcap.config.PcapOptions; import org.apache.metron.pcap.filter.PcapFilter; import org.apache.metron.pcap.filter.PcapFilterConfigurator; import org.apache.metron.pcap.filter.PcapFilters; @@ -431,7 +431,7 @@ public Pageable get() throws JobException, InterruptedException { return getFinalResults(); } else if (status.getState() == State.KILLED || status.getState() == State.FAILED) { - return null; + return new EmptyResults(); } Thread.sleep(completeCheckInterval); } @@ -443,7 +443,9 @@ private synchronized Pageable getFinalResults() { @Override public boolean isDone() { - return finalized; + return (jobState == State.SUCCEEDED + || jobState == State.KILLED + || jobState == State.FAILED); } @Override From 0615ae60f7542a182f8dbd31cc07ede6b038be43 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Fri, 13 Jul 2018 17:12:23 -0600 Subject: [PATCH 20/36] Revert PcapServiceImpl --- .../rest/service/impl/PcapServiceImpl.java | 103 +++++------------- 1 file changed, 29 insertions(+), 74 deletions(-) diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java index 4acf36061d..4dae1e5ec6 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java @@ -17,31 +17,15 @@ */ package org.apache.metron.rest.service.impl; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.metron.common.Constants; -import org.apache.metron.common.system.Clock; -import org.apache.metron.common.utils.HDFSUtils; +import org.apache.metron.common.hadoop.SequenceFileIterable; import org.apache.metron.common.utils.timestamp.TimestampConverters; -import org.apache.metron.job.JobException; -import org.apache.metron.job.JobStatus; -import org.apache.metron.job.Pageable; -import org.apache.metron.job.Statusable; -import org.apache.metron.job.manager.JobManager; -import org.apache.metron.job.service.JobServiceStrategies; -import org.apache.metron.job.writer.PrefixStrategy; import org.apache.metron.pcap.PcapHelper; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.mr.PcapJob; -import org.apache.metron.pcap.mr.PcapMRJobConfig; -import org.apache.metron.pcap.writer.PcapResultsWriter; import org.apache.metron.rest.MetronRestConstants; import org.apache.metron.rest.RestException; import org.apache.metron.rest.model.PcapResponse; @@ -51,84 +35,55 @@ import org.springframework.core.env.Environment; import org.springframework.stereotype.Service; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + @Service public class PcapServiceImpl implements PcapService { private Environment environment; private Configuration configuration; - private JobManager jobManager; + private PcapJob pcapJob; @Autowired - public PcapServiceImpl(Environment environment, Configuration configuration) { + public PcapServiceImpl(Environment environment, Configuration configuration, PcapJob pcapJob) { this.environment = environment; this.configuration = configuration; - this.jobManager = new JobManager<>(JobServiceStrategies.HDFS); + this.pcapJob = pcapJob; } @Override - public JobStatus fixed(FixedPcapRequest fixedPcapRequest) throws RestException { + public PcapResponse fixed(FixedPcapRequest fixedPcapRequest) throws RestException { if (fixedPcapRequest.getBasePath() == null) { fixedPcapRequest.setBasePath(environment.getProperty(MetronRestConstants.PCAP_INPUT_PATH_SPRING_PROPERTY)); } if (fixedPcapRequest.getBaseOutputPath() == null) { fixedPcapRequest.setBaseOutputPath(environment.getProperty(MetronRestConstants.PCAP_OUTPUT_PATH_SPRING_PROPERTY)); } - - PrefixStrategy prefixStrategy = clock -> { - String timestamp = new Clock().currentTimeFormatted("yyyyMMddHHmm"); - String uuid = UUID.randomUUID().toString().replaceAll("-", ""); - return String.format("%s-%s", timestamp, uuid); - }; - - PcapMRJobConfig> config = new PcapMRJobConfig(); - try { - config.setBasePath(new Path(fixedPcapRequest.getBasePath())) - .setBaseOutputPath(new Path(fixedPcapRequest.getBaseOutputPath())) - .setBeginNS(TimestampConverters.MILLISECONDS.toNanoseconds(fixedPcapRequest.getStartTime())) - .setEndNS(TimestampConverters.MILLISECONDS.toNanoseconds(fixedPcapRequest.getEndTime())) - .setNumReducers(fixedPcapRequest.getNumReducers()) - .setFields(getFixedFields(fixedPcapRequest)) - .setConf(configuration) - .setFs(getFileSystem()) - .setFilterImpl(new FixedPcapFilter.Configurator()) - .setResultsWriter(new PcapResultsWriter()) - .setFinalOutputPath(new Path("/apps/metron/pcap/final")) - .setNumRecordsPerFile(10) - .setOutputFilePrefix(prefixStrategy.apply(new Clock())); - - Statusable pcapJob = new PcapJob(config); - return jobManager.submit(pcapJob, new HashMap<>(), "metron"); - } catch (IOException | JobException e) { - throw new RestException(e); - } - } - - @Override - public JobStatus getJobStatus(String username, String jobId) throws RestException { - try { - return jobManager.getStatus(username, jobId); - } catch (JobException e) { - throw new RestException("Unable to get job status", e); - } - } - - @Override - public PcapResponse getPage(String username, String jobId, int pageNum) throws RestException { PcapResponse response = new PcapResponse(); - Statusable job = jobManager.getJob(username, jobId); + SequenceFileIterable results; try { - if (job.isDone()) { - Pageable results = job.getFinalResults(); - Path resultsPage = results.getPage(pageNum); - if (resultsPage != null) { - List pcaps = new ArrayList<>(); - byte[] pcap = HDFSUtils.readBytes(resultsPage.toString()); - pcaps.add(pcap); - response.setPcaps(pcaps); - } + results = pcapJob.query( + new Path(fixedPcapRequest.getBasePath()), + new Path(fixedPcapRequest.getBaseOutputPath()), + TimestampConverters.MILLISECONDS.toNanoseconds(fixedPcapRequest.getStartTime()), + TimestampConverters.MILLISECONDS.toNanoseconds(fixedPcapRequest.getEndTime()), + fixedPcapRequest.getNumReducers(), + getFixedFields(fixedPcapRequest), + configuration, + getFileSystem(), + new FixedPcapFilter.Configurator() + ); + if (results != null) { + List pcaps = new ArrayList<>(); + results.iterator().forEachRemaining(pcaps::add); + response.setPcaps(pcaps); } - } catch (JobException | IOException e) { - throw new RestException("Unable to get results", e); + } catch (IOException | ClassNotFoundException | InterruptedException e) { + throw new RestException(e); } return response; } From 433d0a37501ae0bdc8b3e5a7b0fc3ceb9c125ebf Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Fri, 13 Jul 2018 22:02:03 -0600 Subject: [PATCH 21/36] Get job manager working. Fix pcap cli tests. Make pcap pageable better for synchronicity. Kill remnants of the job service. --- .../metron/rest/service/PcapService.java | 7 +- .../java/org/apache/metron/job/Pageable.java | 9 +- .../job/manager/InMemoryJobManager.java | 21 ++- .../apache/metron/job/manager/JobManager.java | 2 +- .../job/manager/InMemoryJobManagerTest.java | 125 ++++++++------ .../job/service/HdfsJobServiceTest.java | 110 ------------ metron-platform/metron-pcap-backend/pom.xml | 6 + .../org/apache/metron/pcap/PcapJobTest.java | 47 +++--- .../apache/metron/pcap/query/PcapCliTest.java | 157 +++++++++++------- .../org/apache/metron/pcap/PcapPages.java | 49 +++++- .../org/apache/metron/pcap/mr/PcapJob.java | 52 ++++-- .../org/apache/metron/pcap/PcapPagesTest.java | 76 +++++++++ 12 files changed, 380 insertions(+), 281 deletions(-) delete mode 100644 metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java create mode 100644 metron-platform/metron-pcap/src/test/java/org/apache/metron/pcap/PcapPagesTest.java diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/PcapService.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/PcapService.java index a6fdf161b5..ce8372c5b7 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/PcapService.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/PcapService.java @@ -17,16 +17,11 @@ */ package org.apache.metron.rest.service; -import org.apache.metron.job.JobStatus; import org.apache.metron.rest.RestException; import org.apache.metron.rest.model.PcapResponse; import org.apache.metron.rest.model.pcap.FixedPcapRequest; public interface PcapService { - JobStatus fixed(FixedPcapRequest fixedPcapRequest) throws RestException; - - JobStatus getJobStatus(String username, String jobId) throws RestException; - - PcapResponse getPage(String username, String jobId, int pageNum) throws RestException; + PcapResponse fixed(FixedPcapRequest fixedPcapRequest) throws RestException; } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Pageable.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Pageable.java index 9b92c8c0b0..d93c7def17 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Pageable.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Pageable.java @@ -18,14 +18,7 @@ package org.apache.metron.job; -public interface Pageable { - - /** - * Transform into an Iterable. - * - * @return Iterable version of this Pageable. - */ - Iterable asIterable(); +public interface Pageable extends Iterable { /** * Provides access to a specific page of results in the result set. diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java index f80a474502..bf0baa7c1d 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java @@ -19,6 +19,7 @@ package org.apache.metron.job.manager; import java.lang.invoke.MethodHandles; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -31,6 +32,7 @@ import org.slf4j.LoggerFactory; public class InMemoryJobManager implements JobManager { + private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private Map>> jobs; @@ -39,9 +41,10 @@ public InMemoryJobManager() { } @Override - public JobStatus submit(Supplier> jobSupplier, String username) throws JobException { + public JobStatus submit(Supplier> jobSupplier, String username) + throws JobException { + Map> userJobs = getUserJobs(username); Statusable job = jobSupplier.get(); - Map> userJobs = Collections.synchronizedMap(new HashMap<>()); userJobs.put(job.getStatus().getJobId(), job); jobs.put(username, userJobs); return job.getStatus(); @@ -54,22 +57,26 @@ public JobStatus getStatus(String username, String jobId) throws JobException { @Override public boolean done(String username, String jobId) throws JobException { - return false; + return getJob(username, jobId).isDone(); } @Override public void killJob(String username, String jobId) throws JobException { - + getJob(username, jobId).kill(); } @Override public Statusable getJob(String username, String jobId) throws JobException { - return null; + return getUserJobs(username).get(jobId); + } + + private Map> getUserJobs(String username) { + return jobs.getOrDefault(username, Collections.synchronizedMap(new HashMap<>())); } @Override - public List> getJobs(String username, String jobId) throws JobException { - return null; + public List> getJobs(String username) throws JobException { + return new ArrayList>(getUserJobs(username).values()); } } diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java index 70ee0acdf9..eff60e5973 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/JobManager.java @@ -36,6 +36,6 @@ public interface JobManager { Statusable getJob(String username, String jobId) throws JobException; - List> getJobs(String username, String jobId) throws JobException; + List> getJobs(String username) throws JobException; } diff --git a/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java b/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java index 09721c61c9..9aaf96a042 100644 --- a/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java +++ b/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java @@ -2,13 +2,15 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.function.Supplier; import org.apache.hadoop.fs.Path; import org.apache.metron.job.Finalizer; import org.apache.metron.job.JobException; @@ -20,7 +22,6 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import org.mockito.Mock; import org.mockito.MockitoAnnotations; @@ -52,6 +53,8 @@ public class InMemoryJobManagerTest { @Mock private Statusable job2; @Mock + private Statusable job3; + @Mock private Finalizer finalizer; @Mock private Pageable results; @@ -61,6 +64,7 @@ public class InMemoryJobManagerTest { private String username2; private String jobId1; private String jobId2; + private String jobId3; private String basePath; @Before @@ -72,24 +76,21 @@ public void setup() throws JobException { username2 = "user456"; jobId1 = "job_abc_123"; jobId2 = "job_def_456"; + jobId3 = "job_ghi_789"; basePath = tempDir.getRoot().getAbsolutePath(); when(job1.getJobType()).thenReturn(JobType.MAP_REDUCE); when(job2.getJobType()).thenReturn(JobType.MAP_REDUCE); + when(job3.getJobType()).thenReturn(JobType.MAP_REDUCE); + when(job1.submit(finalizer, config)).thenReturn(job1); + when(job2.submit(finalizer, config)).thenReturn(job2); + when(job3.submit(finalizer, config)).thenReturn(job3); when(finalizer.finalizeJob(any())).thenReturn(results); } @Test public void submits_job_and_returns_status() throws JobException { - when(job1.validate(any())).thenReturn(true); - when(job1.submit(finalizer, config)).thenReturn(job1); when(job1.getStatus()).thenReturn(new JobStatus().withState(State.RUNNING).withJobId(jobId1)); - JobStatus status = jm.submit(() -> { - try { - return job1.submit(finalizer, config); - } catch (JobException e) { - throw new RuntimeException("Something went wrong", e); - } - }, username1); + JobStatus status = jm.submit(newSupplier(job1), username1); assertThat(status.getState(), equalTo(State.RUNNING)); assertThat(status.getJobId(), equalTo(jobId1)); when(job1.getStatus()).thenReturn(new JobStatus().withState(State.SUCCEEDED).withJobId(jobId1)); @@ -98,17 +99,46 @@ public void submits_job_and_returns_status() throws JobException { assertThat(status.getJobId(), equalTo(jobId1)); } + @Test + public void submits_multiple_jobs_and_returns_status() throws JobException { + when(job1.getStatus()).thenReturn(new JobStatus().withState(State.RUNNING).withJobId(jobId1)); + when(job2.getStatus()).thenReturn(new JobStatus().withState(State.RUNNING).withJobId(jobId2)); + when(job3.getStatus()).thenReturn(new JobStatus().withState(State.RUNNING).withJobId(jobId3)); + + // user has 1 job + jm.submit(newSupplier(job1), username1); + assertThat(jm.getJob(username1, jobId1), equalTo(job1)); + + // user has 2 jobs + jm.submit(newSupplier(job2), username1); + assertThat(jm.getJob(username1, jobId1), equalTo(job1)); + assertThat(jm.getJob(username1, jobId2), equalTo(job2)); + + // user has 3 jobs + jm.submit(newSupplier(job3), username1); + assertThat(jm.getJob(username1, jobId1), equalTo(job1)); + assertThat(jm.getJob(username1, jobId2), equalTo(job2)); + assertThat(jm.getJob(username1, jobId3), equalTo(job3)); + + // multiple users have 3 jobs + jm.submit(newSupplier(job1), username2); + jm.submit(newSupplier(job2), username2); + jm.submit(newSupplier(job3), username2); + // user 1 still good + assertThat(jm.getJob(username1, jobId1), equalTo(job1)); + assertThat(jm.getJob(username1, jobId2), equalTo(job2)); + assertThat(jm.getJob(username1, jobId3), equalTo(job3)); + // and also user 2 + assertThat(jm.getJob(username2, jobId1), equalTo(job1)); + assertThat(jm.getJob(username2, jobId2), equalTo(job2)); + assertThat(jm.getJob(username2, jobId3), equalTo(job3)); + } + @Test public void returns_job_status() throws JobException { JobStatus expected = new JobStatus().withState(State.SUCCEEDED).withJobId(jobId1); when(job1.getStatus()).thenReturn(expected); - jm.submit(() -> { - try { - return job1.submit(finalizer, config); - } catch (JobException e) { - throw new RuntimeException("Something went wrong", e); - } - }, username1); + jm.submit(newSupplier(job1), username1); JobStatus status = jm.getStatus(username1, jobId1); assertThat(status, equalTo(expected)); } @@ -118,50 +148,45 @@ public void returns_job_is_done() throws JobException { JobStatus expected = new JobStatus().withState(State.SUCCEEDED).withJobId(jobId1); when(job1.getStatus()).thenReturn(expected); when(job1.isDone()).thenReturn(true); - jm.submit(() -> { - try { - return job1.submit(finalizer, config); - } catch (JobException e) { - throw new RuntimeException("Something went wrong", e); - } - }, username1); + jm.submit(newSupplier(job1), username1); boolean done = jm.done(username1, jobId1); assertThat(done, equalTo(true)); } @Test public void kills_job() throws JobException { - JobStatus expected = new JobStatus().withState(State.SUCCEEDED).withJobId(jobId1); - jm.submit(() -> { - try { - return job1.submit(finalizer, config); - } catch (JobException e) { - throw new RuntimeException("Something went wrong", e); - } - }, username1); + when(job1.getStatus()).thenReturn(new JobStatus().withState(State.SUCCEEDED).withJobId(jobId1)); + jm.submit(newSupplier(job1), username1); jm.killJob(username1, jobId1); verify(job1).kill(); } @Test - public void returns_statusable_job() throws JobException { - when(job1.isDone()).thenReturn(true); - Statusable done = jm.getJob(username1, jobId1); - assertThat(done, equalTo(job1)); + public void gets_list_of_user_jobs() throws JobException { + when(job1.getStatus()).thenReturn(new JobStatus().withState(State.RUNNING).withJobId(jobId1)); + when(job2.getStatus()).thenReturn(new JobStatus().withState(State.RUNNING).withJobId(jobId2)); + when(job3.getStatus()).thenReturn(new JobStatus().withState(State.RUNNING).withJobId(jobId3)); + jm.submit(newSupplier(job1), username1); + jm.submit(newSupplier(job2), username1); + jm.submit(newSupplier(job3), username1); + jm.submit(newSupplier(job1), username2); + jm.submit(newSupplier(job2), username2); + jm.submit(newSupplier(job3), username2); + List> jobsUser1 = jm.getJobs(username1); + List> jobsUser2 = jm.getJobs(username2); + assertThat("Wrong size", jobsUser1.size(), equalTo(3)); + assertThat("Wrong size", jobsUser2.size(), equalTo(3)); + assertThat("", jobsUser1.containsAll(Arrays.asList(job1, job2, job3)), equalTo(true)); + assertThat("", jobsUser2.containsAll(Arrays.asList(job1, job2, job3)), equalTo(true)); } - @Rule - public ExpectedException exception = ExpectedException.none(); - - @Test - public void job_submission_exception_returned() throws JobException { - JobException expected = new JobException("test exception"); - when(job1.validate(any())).thenReturn(true); -// when(job.submit()).thenThrow(expected); - exception.expect(JobException.class); - exception.expect(equalTo(expected)); -// jm.submit(job, new HashMap<>(), username); - fail(); + private Supplier> newSupplier(Statusable job) { + return () -> { + try { + return job.submit(finalizer, config); + } catch (JobException e) { + throw new RuntimeException("Something went wrong", e); + } + }; } - } diff --git a/metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java b/metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java deleted file mode 100644 index 903c18a09e..0000000000 --- a/metron-platform/metron-job/src/test/java/org/apache/metron/job/service/HdfsJobServiceTest.java +++ /dev/null @@ -1,110 +0,0 @@ -/** - * 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.metron.job.service; - -import static org.mockito.Mockito.when; - -import java.util.HashMap; -import java.util.Map; -import org.apache.metron.job.Statusable; -import org.apache.metron.job.Statusable.JobType; -import org.junit.Before; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; - -public class HdfsJobServiceTest { - - @Rule - public TemporaryFolder tempDir = new TemporaryFolder(); - - @Mock - private Statusable job1; - @Mock - private Statusable job2; - private String username; - private String jobId1; - private String jobId2; - private String basePath; - - @Before - public void setup() { - MockitoAnnotations.initMocks(this); - username = "user123"; - jobId1 = "job_abc_123"; - jobId2 = "job_def_456"; - when(job1.getJobType()).thenReturn(JobType.MAP_REDUCE); - when(job2.getJobType()).thenReturn(JobType.MAP_REDUCE); - basePath = tempDir.getRoot().getAbsolutePath(); - Map config = new HashMap<>(); - config.put("basePath", basePath); - } -/* - @Test - public void adds_jobs() { - js.add(job1, username, jobId1); - Statusable actual = js.getJob(username, jobId1); - assertThat("Job 1 should exist", actual, equalTo(job1)); - js.add(job2, username, jobId2); - actual = js.getJob(username, jobId1); - assertThat("Job 1 should still exist after adding job 2", actual, equalTo(job1)); - actual = js.getJob(username, jobId2); - assertThat("Job 2 should exist", actual, equalTo(job2)); - } - - @Test - public void job_exists_true_for_submitted_jobs() { - js.add(job1, username, jobId1); - js.add(job2, username, jobId2); - boolean actual = js.jobExists(username, jobId1); - assertThat("Job 1 should exist", actual, equalTo(true)); - actual = js.jobExists(username, jobId2); - assertThat("Job 2 should exist", actual, equalTo(true)); - } - - @Test - public void job_exists_false_for_non_existent_jobs() { - boolean actual = js.jobExists(username, jobId1); - assertThat(actual, equalTo(false)); - js.add(job1, username, jobId1); - actual = js.jobExists(username, "this_job_id_does_not_exist"); - assertThat(actual, equalTo(false)); - } - - @Test - public void returns_null_for_non_existent_job() { - Statusable actual = js.getJob(username, jobId1); - assertThat(actual, equalTo(null)); - js.add(job1, username, jobId1); - actual = js.getJob(username, "this_job_id_does_not_exist"); - assertThat(actual, equalTo(null)); - } - - @Test - public void writes_job_info_to_hdfs() { - // /base/path/jobs/metron_user/MAP_REDUCE/job_abc_123 - js.add(job1, username, jobId1); - File jobFile = new File(String.format("%s/jobs/%s/%s/%s", basePath, username, JobType.MAP_REDUCE.toString(), - jobId1)); - assertThat("File should exist", jobFile.exists(), equalTo(true)); - assertThat("File should be a file", jobFile.isFile(), equalTo(true)); - } -*/ -} diff --git a/metron-platform/metron-pcap-backend/pom.xml b/metron-platform/metron-pcap-backend/pom.xml index c3b1a6998f..fb523ee32d 100644 --- a/metron-platform/metron-pcap-backend/pom.xml +++ b/metron-platform/metron-pcap-backend/pom.xml @@ -218,6 +218,12 @@ ${project.parent.version} test + + org.hamcrest + hamcrest-all + 1.3 + test + diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java index 1a3e6e6f37..ccce608fbc 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java @@ -24,6 +24,7 @@ import static org.mockito.Mockito.when; import java.io.IOException; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -40,11 +41,11 @@ import org.apache.metron.job.JobStatus.State; import org.apache.metron.job.Pageable; import org.apache.metron.job.Statusable; +import org.apache.metron.pcap.config.FixedPcapConfig; import org.apache.metron.pcap.config.PcapOptions; import org.apache.metron.pcap.filter.PcapFilterConfigurator; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.mr.PcapJob; -import org.apache.metron.pcap.config.FixedPcapConfig; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -54,14 +55,13 @@ public class PcapJobTest { @Mock - private Job job; + private Job mrJob; @Mock private org.apache.hadoop.mapreduce.JobStatus mrStatus; @Mock private JobID jobId; @Mock private Finalizer finalizer; - @Mock private Pageable pageableResult; private FixedPcapConfig config; private Configuration hadoopConfig; @@ -94,6 +94,7 @@ public void setup() throws IOException { finalOutputPath = new Path("finaloutpath"); when(jobId.toString()).thenReturn(jobIdVal); when(mrStatus.getJobID()).thenReturn(jobId); + pageableResult = new PcapPages(); // handles setting the file name prefix under the hood config = new FixedPcapConfig(clock -> "clockprefix"); PcapOptions.HADOOP_CONF.put(config, hadoopConfig); @@ -140,17 +141,18 @@ public Job createJob(Optional jobName, Configuration conf, FileSystem fs, PcapFilterConfigurator filterImpl) throws IOException { - return job; + return mrJob; } } @Test public void job_succeeds_synchronously() throws Exception { - when(pageableResult.getSize()).thenReturn(3); + pageableResult = new PcapPages( + Arrays.asList(new Path("1.txt"), new Path("2.txt"), new Path("3.txt"))); when(finalizer.finalizeJob(any())).thenReturn(pageableResult); - when(job.isComplete()).thenReturn(true); + when(mrJob.isComplete()).thenReturn(true); when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.SUCCEEDED); - when(job.getStatus()).thenReturn(mrStatus); + when(mrJob.getStatus()).thenReturn(mrStatus); Statusable statusable = testJob.submit(finalizer, config); Pageable results = statusable.get(); Assert.assertThat(results.getSize(), equalTo(3)); @@ -162,37 +164,38 @@ public void job_succeeds_synchronously() throws Exception { @Test public void job_fails_synchronously() throws Exception { - when(job.isComplete()).thenReturn(true); + when(mrJob.isComplete()).thenReturn(true); when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.FAILED); - when(job.getStatus()).thenReturn(mrStatus); + when(mrJob.getStatus()).thenReturn(mrStatus); Statusable statusable = testJob.submit(finalizer, config); Pageable results = statusable.get(); JobStatus status = statusable.getStatus(); Assert.assertThat(status.getState(), equalTo(State.FAILED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); - Assert.assertThat(results, equalTo(null)); + Assert.assertThat(results.getSize(), equalTo(0)); } @Test public void job_fails_with_killed_status_synchronously() throws Exception { - when(job.isComplete()).thenReturn(true); + when(mrJob.isComplete()).thenReturn(true); when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.KILLED); - when(job.getStatus()).thenReturn(mrStatus); + when(mrJob.getStatus()).thenReturn(mrStatus); Statusable statusable = testJob.submit(finalizer, config); Pageable results = statusable.get(); JobStatus status = statusable.getStatus(); Assert.assertThat(status.getState(), equalTo(State.KILLED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); - Assert.assertThat(results, equalTo(null)); + Assert.assertThat(results.getSize(), equalTo(0)); } @Test public void job_succeeds_asynchronously() throws Exception { - when(job.isComplete()).thenReturn(true); + when(mrJob.isComplete()).thenReturn(true); when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.SUCCEEDED); - when(job.getStatus()).thenReturn(mrStatus); + when(mrJob.getStatus()).thenReturn(mrStatus); Statusable statusable = testJob.submit(finalizer, config); - while(!statusable.isDone()) {} + while (!statusable.isDone()) { + } JobStatus status = statusable.getStatus(); Assert.assertThat(status.getState(), equalTo(State.SUCCEEDED)); Assert.assertThat(status.getPercentComplete(), equalTo(100.0)); @@ -200,18 +203,18 @@ public void job_succeeds_asynchronously() throws Exception { @Test public void job_reports_percent_complete() throws Exception { - when(job.isComplete()).thenReturn(false); + when(mrJob.isComplete()).thenReturn(false); when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.RUNNING); - when(job.getStatus()).thenReturn(mrStatus); + when(mrJob.getStatus()).thenReturn(mrStatus); Statusable statusable = testJob.submit(finalizer, config); - when(job.mapProgress()).thenReturn(0.5f); - when(job.reduceProgress()).thenReturn(0f); + when(mrJob.mapProgress()).thenReturn(0.5f); + when(mrJob.reduceProgress()).thenReturn(0f); JobStatus status = statusable.getStatus(); Assert.assertThat(status.getState(), equalTo(State.RUNNING)); Assert.assertThat(status.getDescription(), equalTo("map: 50.0%, reduce: 0.0%")); Assert.assertThat(status.getPercentComplete(), equalTo(25.0)); - when(job.mapProgress()).thenReturn(1.0f); - when(job.reduceProgress()).thenReturn(0.5f); + when(mrJob.mapProgress()).thenReturn(1.0f); + when(mrJob.reduceProgress()).thenReturn(0.5f); status = statusable.getStatus(); Assert.assertThat(status.getPercentComplete(), equalTo(75.0)); Assert.assertThat(status.getDescription(), equalTo("map: 100.0%, reduce: 50.0%")); diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java index 58d1f085d3..2ad0657919 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java @@ -19,7 +19,9 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertThat; -import static org.mockito.Mockito.mock; +import static org.mockito.Matchers.argThat; +import static org.mockito.Matchers.isA; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.io.BufferedOutputStream; @@ -29,19 +31,22 @@ import java.nio.charset.StandardCharsets; import java.text.ParseException; import java.text.SimpleDateFormat; -import java.util.Arrays; import java.util.Date; import java.util.HashMap; -import java.util.Iterator; -import java.util.List; import java.util.Map; -import org.apache.hadoop.fs.Path; +import java.util.Map.Entry; import org.apache.metron.common.Constants; -import org.apache.metron.common.hadoop.SequenceFileIterable; import org.apache.metron.common.system.Clock; import org.apache.metron.common.utils.timestamp.TimestampConverters; +import org.apache.metron.job.Finalizer; import org.apache.metron.pcap.PcapHelper; +import org.apache.metron.pcap.config.FixedPcapConfig; +import org.apache.metron.pcap.config.PcapConfig.PrefixStrategy; +import org.apache.metron.pcap.config.PcapOptions; import org.apache.metron.pcap.mr.PcapJob; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeMatcher; import org.junit.Before; import org.junit.Test; import org.mockito.Mock; @@ -54,12 +59,13 @@ public class PcapCliTest { @Mock private Clock clock; private String execDir; + private PrefixStrategy prefixStrategy; @Before public void setup() throws IOException { MockitoAnnotations.initMocks(this); -// doCallRealMethod().when(jobRunner).writeFinalResults(anyObject(), anyObject(), anyObject(), anyInt(), anyObject()); execDir = System.getProperty("user.dir"); + prefixStrategy = clock -> "random_prefix"; } @Test @@ -74,13 +80,7 @@ public void runs_fixed_pcap_filter_job_with_default_argument_list() throws Excep "-protocol", "6", "-packet_filter", "`casey`" }; - List pcaps = Arrays.asList(new byte[][]{asBytes("abc"), asBytes("def"), asBytes("ghi")}); - Iterator iterator = pcaps.iterator(); - SequenceFileIterable iterable = mock(SequenceFileIterable.class); - when(iterable.iterator()).thenReturn(iterator); - Path base_path = new Path(CliParser.BASE_PATH_DEFAULT); - Path base_output_path = new Path(CliParser.BASE_OUTPUT_PATH_DEFAULT); HashMap query = new HashMap() {{ put(Constants.Fields.SRC_ADDR.getName(), "192.168.1.1"); put(Constants.Fields.DST_ADDR.getName(), "192.168.1.2"); @@ -90,14 +90,47 @@ public void runs_fixed_pcap_filter_job_with_default_argument_list() throws Excep put(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName(), "false"); put(PcapHelper.PacketFields.PACKET_FILTER.getName(), "`casey`"); }}; + FixedPcapConfig config = new FixedPcapConfig(prefixStrategy); + PcapOptions.BASE_PATH.put(config, CliParser.BASE_PATH_DEFAULT); + PcapOptions.INTERIM_RESULT_PATH.put(config, CliParser.BASE_OUTPUT_PATH_DEFAULT); + PcapOptions.FIELDS.put(config, query); + PcapOptions.NUM_REDUCERS.put(config, 10); + PcapOptions.START_TIME_MS.put(config, 500L); -// when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); + when(jobRunner.submit(isA(Finalizer.class), argThat(mapContaining(config)))).thenReturn(jobRunner); - PcapCli cli = new PcapCli(jobRunner, clock -> "random_prefix"); + PcapCli cli = new PcapCli(jobRunner, prefixStrategy); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); -// Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); + verify(jobRunner).get(); } + /** + * Check that "map" entries exist in the tested map "item". Note, will not work for complex + * Objects where equals() does not compare contents favorably. e.g. Configurator() did not work. + */ + private Matcher> mapContaining(Map map) { + return new TypeSafeMatcher>() { + @Override + protected boolean matchesSafely(Map item) { + return item.entrySet().containsAll(map.entrySet()); + } + + @Override + public void describeTo(Description description) { + description.appendText("Should contain items: "); + for (Entry entry : map.entrySet()) { + StringBuilder sb = new StringBuilder(); + sb.append("key="); + sb.append(entry.getKey()); + sb.append(",value="); + sb.append(entry.getValue()); + description.appendText(sb.toString()); + } + } + }; + } + + @Test public void runs_fixed_pcap_filter_job_with_full_argument_list_and_default_dateformat() throws Exception { String[] args = { @@ -115,13 +148,6 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list_and_default_datef "-num_reducers", "10", "-records_per_file", "1000" }; - List pcaps = Arrays.asList(new byte[][]{asBytes("abc"), asBytes("def"), asBytes("ghi")}); - Iterator iterator = pcaps.iterator(); - SequenceFileIterable iterable = mock(SequenceFileIterable.class); - when(iterable.iterator()).thenReturn(iterator); - - Path base_path = new Path("/base/path"); - Path base_output_path = new Path("/base/output/path"); Map query = new HashMap() {{ put(Constants.Fields.SRC_ADDR.getName(), "192.168.1.1"); put(Constants.Fields.DST_ADDR.getName(), "192.168.1.2"); @@ -130,12 +156,20 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list_and_default_datef put(Constants.Fields.PROTOCOL.getName(), "6"); put(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName(), "true"); }}; - -// when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); - - PcapCli cli = new PcapCli(jobRunner, clock -> "random_prefix"); + FixedPcapConfig config = new FixedPcapConfig(prefixStrategy); + PcapOptions.BASE_PATH.put(config, "/base/path"); + PcapOptions.INTERIM_RESULT_PATH.put(config, "/base/output/path"); + PcapOptions.FIELDS.put(config, query); + PcapOptions.NUM_REDUCERS.put(config, 10); + PcapOptions.START_TIME_MS.put(config, 500L); + PcapOptions.END_TIME_MS.put(config, 1000L); + PcapOptions.NUM_RECORDS_PER_FILE.put(config, 1000); + + when(jobRunner.submit(isA(Finalizer.class), argThat(mapContaining(config)))).thenReturn(jobRunner); + + PcapCli cli = new PcapCli(jobRunner, prefixStrategy); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); -// Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); + verify(jobRunner).get(); } @Test @@ -156,13 +190,6 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list() throws Exceptio "-num_reducers", "10", "-records_per_file", "1000" }; - List pcaps = Arrays.asList(new byte[][]{asBytes("abc"), asBytes("def"), asBytes("ghi")}); - Iterator iterator = pcaps.iterator(); - SequenceFileIterable iterable = mock(SequenceFileIterable.class); - when(iterable.iterator()).thenReturn(iterator); - - Path base_path = new Path("/base/path"); - Path base_output_path = new Path("/base/output/path"); Map query = new HashMap() {{ put(Constants.Fields.SRC_ADDR.getName(), "192.168.1.1"); put(Constants.Fields.DST_ADDR.getName(), "192.168.1.2"); @@ -174,11 +201,23 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list() throws Exceptio long startAsNanos = asNanos("2016-06-13-18:35.00", "yyyy-MM-dd-HH:mm.ss"); long endAsNanos = asNanos("2016-06-15-18:35.00", "yyyy-MM-dd-HH:mm.ss"); -// when(jobRunner.query(eq(base_path), eq(base_output_path), eq(startAsNanos), eq(endAsNanos), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(FixedPcapFilter.Configurator.class))).thenReturn(iterable); - PcapCli cli = new PcapCli(jobRunner, clock -> "random_prefix"); + FixedPcapConfig config = new FixedPcapConfig(prefixStrategy); + PcapOptions.BASE_PATH.put(config, "/base/path"); + PcapOptions.INTERIM_RESULT_PATH.put(config, "/base/output/path"); + PcapOptions.FIELDS.put(config, query); + PcapOptions.NUM_REDUCERS.put(config, 10); + PcapOptions.START_TIME_MS.put(config, startAsNanos / 1000000L); // needed bc defaults in config + PcapOptions.END_TIME_MS.put(config, endAsNanos / 1000000L); // needed bc defaults in config + PcapOptions.START_TIME_NS.put(config, startAsNanos); + PcapOptions.END_TIME_NS.put(config, endAsNanos); + PcapOptions.NUM_RECORDS_PER_FILE.put(config, 1000); + + when(jobRunner.submit(isA(Finalizer.class), argThat(mapContaining(config)))).thenReturn(jobRunner); + + PcapCli cli = new PcapCli(jobRunner, prefixStrategy); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); -// Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); + verify(jobRunner).get(); } private long asNanos(String inDate, String format) throws ParseException { @@ -198,20 +237,20 @@ public void runs_query_pcap_filter_job_with_default_argument_list() throws Excep "-start_time", "500", "-query", "some query string" }; - List pcaps = Arrays.asList(new byte[][]{asBytes("abc"), asBytes("def"), asBytes("ghi")}); - Iterator iterator = pcaps.iterator(); - SequenceFileIterable iterable = mock(SequenceFileIterable.class); - when(iterable.iterator()).thenReturn(iterator); - Path base_path = new Path(CliParser.BASE_PATH_DEFAULT); - Path base_output_path = new Path(CliParser.BASE_OUTPUT_PATH_DEFAULT); String query = "some query string"; + FixedPcapConfig config = new FixedPcapConfig(prefixStrategy); + PcapOptions.BASE_PATH.put(config, CliParser.BASE_PATH_DEFAULT); + PcapOptions.INTERIM_RESULT_PATH.put(config, CliParser.BASE_OUTPUT_PATH_DEFAULT); + PcapOptions.FIELDS.put(config, query); + PcapOptions.NUM_REDUCERS.put(config, 10); + PcapOptions.START_TIME_MS.put(config, 500L); -// when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(QueryPcapFilter.Configurator.class))).thenReturn(iterable); + when(jobRunner.submit(isA(Finalizer.class), argThat(mapContaining(config)))).thenReturn(jobRunner); - PcapCli cli = new PcapCli(jobRunner, clock -> "random_prefix"); + PcapCli cli = new PcapCli(jobRunner, prefixStrategy); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); -// Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); + verify(jobRunner).get(); } @Test @@ -226,20 +265,22 @@ public void runs_query_pcap_filter_job_with_full_argument_list() throws Exceptio "-query", "some query string", "-records_per_file", "1000" }; - List pcaps = Arrays.asList(new byte[][]{asBytes("abc"), asBytes("def"), asBytes("ghi")}); - Iterator iterator = pcaps.iterator(); - SequenceFileIterable iterable = mock(SequenceFileIterable.class); - when(iterable.iterator()).thenReturn(iterator); - Path base_path = new Path("/base/path"); - Path base_output_path = new Path("/base/output/path"); String query = "some query string"; - -// when(jobRunner.query(eq(base_path), eq(base_output_path), anyLong(), anyLong(), anyInt(), eq(query), isA(Configuration.class), isA(FileSystem.class), isA(QueryPcapFilter.Configurator.class))).thenReturn(iterable); - - PcapCli cli = new PcapCli(jobRunner, clock -> "random_prefix"); + FixedPcapConfig config = new FixedPcapConfig(prefixStrategy); + PcapOptions.BASE_PATH.put(config, "/base/path"); + PcapOptions.INTERIM_RESULT_PATH.put(config, "/base/output/path"); + PcapOptions.FIELDS.put(config, query); + PcapOptions.NUM_REDUCERS.put(config, 10); + PcapOptions.START_TIME_MS.put(config, 500L); // needed bc defaults in config + PcapOptions.END_TIME_MS.put(config, 1000L); // needed bc defaults in config + PcapOptions.NUM_RECORDS_PER_FILE.put(config, 1000); + + when(jobRunner.submit(isA(Finalizer.class), argThat(mapContaining(config)))).thenReturn(jobRunner); + + PcapCli cli = new PcapCli(jobRunner, prefixStrategy); assertThat("Expect no errors on run", cli.run(args), equalTo(0)); -// Mockito.verify(resultsWriter).write(eq(pcaps), eq("file:" + execDir + "/pcap-data-random_prefix+0001.pcap")); + verify(jobRunner).get(); } // INVALID OPTION CHECKS diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapPages.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapPages.java index 6f027aea8c..c98e6818f9 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapPages.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/PcapPages.java @@ -18,6 +18,8 @@ package org.apache.metron.pcap; +import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import org.apache.hadoop.fs.Path; import org.apache.metron.job.Pageable; @@ -26,13 +28,25 @@ public class PcapPages implements Pageable { private final List files; - public PcapPages(List files) { - this.files = files; + /** + * Copy constructor. + */ + public PcapPages(Pageable pages) { + this.files = new ArrayList<>(); + for (Path path : pages) { + files.add(new Path(path.toString())); + } } - @Override - public Iterable asIterable() { - return files; + /** + * Defaults with empty list. + */ + public PcapPages() { + this.files = new ArrayList<>(); + } + + public PcapPages(List paths) { + files = new ArrayList<>(paths); } @Override @@ -44,4 +58,29 @@ public Path getPage(int num) { public int getSize() { return files.size(); } + + @Override + public Iterator iterator() { + return new PcapIterator(files.iterator()); + } + + private class PcapIterator implements Iterator { + + private Iterator delegateIt; + + public PcapIterator(Iterator iterator) { + this.delegateIt = iterator; + } + + @Override + public boolean hasNext() { + return delegateIt.hasNext(); + } + + @Override + public Path next() { + return delegateIt.next(); + } + } + } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index 8d343fe9ef..d848cb6264 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -58,6 +58,7 @@ import org.apache.metron.job.Statusable; import org.apache.metron.pcap.PacketInfo; import org.apache.metron.pcap.PcapHelper; +import org.apache.metron.pcap.PcapPages; import org.apache.metron.pcap.config.PcapOptions; import org.apache.metron.pcap.filter.PcapFilter; import org.apache.metron.pcap.filter.PcapFilterConfigurator; @@ -79,7 +80,7 @@ public class PcapJob implements Statusable { private static final long THREE_SECONDS = 3000; private static final long ONE_SECOND = 1000; private Job mrJob; // store a running MR job reference for async status check - private volatile State jobState; // overall job state, including finalization step + private State jobState; // overall job state, including finalization step private Finalizer finalizer; private Map configuration; private Pageable finalResults; @@ -180,6 +181,8 @@ protected void reduce(LongWritable key, Iterable values, Context } public PcapJob() { + jobState = State.NOT_RUNNING; + finalResults = new PcapPages(); statusInterval = THREE_SECONDS; completeCheckInterval = ONE_SECOND; } @@ -270,6 +273,7 @@ public Statusable query(Optional jobName, , filterImpl ); mrJob.submit(); + jobState = State.RUNNING; startJobStatusTimerThread(statusInterval); return this; } @@ -280,14 +284,17 @@ private void startJobStatusTimerThread(long interval) { @Override public void run() { try { - synchronized (this) { + synchronized (jobState) { if (jobState == State.RUNNING) { if (mrJob.isComplete()) { switch (mrJob.getStatus().getState()) { case SUCCEEDED: jobState = State.FINALIZING; - setFinalResults(finalizer.finalizeJob(configuration)); - jobState = State.SUCCEEDED; + if (setFinalResults(finalizer, configuration)) { + jobState = State.SUCCEEDED; + } else { + jobState = State.FAILED; + } break; case FAILED: jobState = State.FAILED; @@ -300,16 +307,34 @@ public void run() { cancel(); // be gone, ye! } } - } catch (InterruptedException | IOException | JobException e) { + } catch (InterruptedException | IOException e) { jobState = State.FAILED; - throw new RuntimeException("Error getting job status.", e); + cancel(); } } }, interval, interval); } - private synchronized void setFinalResults(Pageable results) { - finalResults = results; + /** + * Writes results using finalizer. Returns true on success, false otherwise. + * + * @param finalizer Writes results. + * @param configuration Configure the finalizer. + * @return Returns true on success, false otherwise. + */ + private boolean setFinalResults(Finalizer finalizer, Map configuration) { + boolean success = true; + Pageable results = new PcapPages(); + try { + results = finalizer.finalizeJob(configuration); + } catch (JobException e) { + LOG.error("Failed to finalize job.", e); + success = false; + } + synchronized (this) { + finalResults = results; + } + return success; } /** @@ -427,22 +452,21 @@ public JobStatus getStatus() throws JobException { public Pageable get() throws JobException, InterruptedException { for (; ; ) { JobStatus status = getStatus(); - if (status.getState() == State.SUCCEEDED) { - return getFinalResults(); - } else if (status.getState() == State.KILLED + if (status.getState() == State.SUCCEEDED + || status.getState() == State.KILLED || status.getState() == State.FAILED) { - return new EmptyResults(); + return getFinalResults(); } Thread.sleep(completeCheckInterval); } } private synchronized Pageable getFinalResults() { - return finalResults; + return new PcapPages(finalResults); } @Override - public boolean isDone() { + public synchronized boolean isDone() { return (jobState == State.SUCCEEDED || jobState == State.KILLED || jobState == State.FAILED); diff --git a/metron-platform/metron-pcap/src/test/java/org/apache/metron/pcap/PcapPagesTest.java b/metron-platform/metron-pcap/src/test/java/org/apache/metron/pcap/PcapPagesTest.java new file mode 100644 index 0000000000..0be2bb515c --- /dev/null +++ b/metron-platform/metron-pcap/src/test/java/org/apache/metron/pcap/PcapPagesTest.java @@ -0,0 +1,76 @@ +/** + * 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.metron.pcap; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.sameInstance; +import static org.junit.Assert.assertThat; + +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.fs.Path; +import org.junit.Test; + +public class PcapPagesTest { + + @Test + public void iterates_paths() { + Path path1 = new Path("/1.txt"); + Path path2 = new Path("/2.txt"); + Path path3 = new Path("/3.txt"); + List paths = new ArrayList<>(); + paths.add(path1); + paths.add(path2); + paths.add(path3); + PcapPages pages = new PcapPages(paths); + assertThat("Wrong num pages.", pages.getSize(), equalTo(3)); + + for (int i = 0; i < pages.getSize(); i++) { + assertThat("Page should be equal", pages.getPage(i).toString(), + equalTo(paths.get(i).toString())); + } + + } + + @Test + public void clones_with_copy_constructor() { + Path path1 = new Path("/1.txt"); + Path path2 = new Path("/2.txt"); + Path path3 = new Path("/3.txt"); + List paths = new ArrayList<>(); + paths.add(path1); + paths.add(path2); + paths.add(path3); + + PcapPages pages = new PcapPages(paths); + PcapPages clonedPages = new PcapPages(pages); + assertThat(clonedPages, notNullValue()); + assertThat(clonedPages.getSize(), equalTo(3)); + assertThat(clonedPages, not(sameInstance(pages))); + + for (int i = 0; i < pages.getSize(); i++) { + assertThat("Page should be different instance.", pages.getPage(i), + not(sameInstance(clonedPages.getPage(i)))); + assertThat("Page should be same path.", pages.getPage(i), equalTo(clonedPages.getPage(i))); + } + } + +} From 71f387160dad1d9ad53b94462ea437d29ee11323 Mon Sep 17 00:00:00 2001 From: merrimanr Date: Mon, 16 Jul 2018 09:49:25 -0500 Subject: [PATCH 22/36] updated pcap integration test --- .../PcapTopologyIntegrationTest.java | 114 +++++++++--------- 1 file changed, 57 insertions(+), 57 deletions(-) diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java index ed705b8082..c403eacd27 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java @@ -62,10 +62,10 @@ import org.apache.metron.job.Finalizer; import org.apache.metron.job.JobStatus; import org.apache.metron.job.Statusable; -import org.apache.metron.pcap.ConfigOptions; import org.apache.metron.pcap.PacketInfo; import org.apache.metron.pcap.PcapHelper; import org.apache.metron.pcap.PcapMerger; +import org.apache.metron.pcap.config.PcapOptions; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.filter.query.QueryPcapFilter; import org.apache.metron.pcap.finalizer.PcapFinalizer; @@ -256,20 +256,20 @@ public ProcessorResult getResult() { Map configuration = new HashMap<>(); Configuration hadoopConf = new Configuration(); - ConfigOptions.JOB_NAME.put(configuration, "jobName"); - ConfigOptions.HADOOP_CONF.put(configuration, hadoopConf); - ConfigOptions.FILESYSTEM.put(configuration, FileSystem.get(hadoopConf)); - ConfigOptions.BASE_PATH.put(configuration, new Path(inputDir.getAbsolutePath())); - ConfigOptions.BASE_INTERRIM_RESULT_PATH.put(configuration, new Path(interimResultDir.getAbsolutePath())); - ConfigOptions.START_TIME_NS.put(configuration, getTimestamp(4, pcapEntries)); - ConfigOptions.END_TIME_NS.put(configuration, getTimestamp(5, pcapEntries)); - ConfigOptions.NUM_REDUCERS.put(configuration, 10); - ConfigOptions.NUM_RECORDS_PER_FILE.put(configuration, 2); - ConfigOptions.FINAL_OUTPUT_PATH.put(configuration, new Path(outputDir.getAbsolutePath())); + PcapOptions.JOB_NAME.put(configuration, "jobName"); + PcapOptions.HADOOP_CONF.put(configuration, hadoopConf); + PcapOptions.FILESYSTEM.put(configuration, FileSystem.get(hadoopConf)); + PcapOptions.BASE_PATH.put(configuration, new Path(inputDir.getAbsolutePath())); + PcapOptions.BASE_INTERRIM_RESULT_PATH.put(configuration, new Path(interimResultDir.getAbsolutePath())); + PcapOptions.START_TIME_NS.put(configuration, getTimestamp(4, pcapEntries)); + PcapOptions.END_TIME_NS.put(configuration, getTimestamp(5, pcapEntries)); + PcapOptions.NUM_REDUCERS.put(configuration, 10); + PcapOptions.NUM_RECORDS_PER_FILE.put(configuration, 2); + PcapOptions.FINAL_OUTPUT_PATH.put(configuration, new Path(outputDir.getAbsolutePath())); { //Ensure that only two pcaps are returned when we look at 4 and 5 - ConfigOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); - ConfigOptions.FIELDS.put(configuration, new HashMap()); + PcapOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); + PcapOptions.FIELDS.put(configuration, new HashMap()); PcapJob> job = new PcapJob<>(); Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); @@ -277,7 +277,7 @@ public ProcessorResult getResult() { waitForJob(results); Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); - Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + Iterable bytes = Iterables.transform(results.get(), path -> { try { return HDFSUtils.readBytes(path); } catch (IOException e) { @@ -290,8 +290,8 @@ public ProcessorResult getResult() { { // Ensure that only two pcaps are returned when we look at 4 and 5 // test with empty query filter - ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); - ConfigOptions.FIELDS.put(configuration, ""); + PcapOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + PcapOptions.FIELDS.put(configuration, ""); PcapJob job = new PcapJob<>(); Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); @@ -299,7 +299,7 @@ public ProcessorResult getResult() { waitForJob(results); Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); - Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + Iterable bytes = Iterables.transform(results.get(), path -> { try { return HDFSUtils.readBytes(path); } catch (IOException e) { @@ -311,8 +311,8 @@ public ProcessorResult getResult() { } { //ensure that none get returned since that destination IP address isn't in the dataset - ConfigOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); - ConfigOptions.FIELDS.put(configuration, new HashMap() {{ + PcapOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); + PcapOptions.FIELDS.put(configuration, new HashMap() {{ put(Constants.Fields.DST_ADDR.getName(), "207.28.210.1"); }}); PcapJob> job = new PcapJob<>(); @@ -322,7 +322,7 @@ public ProcessorResult getResult() { waitForJob(results); Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); - Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + Iterable bytes = Iterables.transform(results.get(), path -> { try { return HDFSUtils.readBytes(path); } catch (IOException e) { @@ -335,8 +335,8 @@ public ProcessorResult getResult() { { // ensure that none get returned since that destination IP address isn't in the dataset // test with query filter - ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); - ConfigOptions.FIELDS.put(configuration, "ip_dst_addr == '207.28.210.1'"); + PcapOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + PcapOptions.FIELDS.put(configuration, "ip_dst_addr == '207.28.210.1'"); PcapJob job = new PcapJob<>(); Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); @@ -344,7 +344,7 @@ public ProcessorResult getResult() { waitForJob(results); Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); - Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + Iterable bytes = Iterables.transform(results.get(), path -> { try { return HDFSUtils.readBytes(path); } catch (IOException e) { @@ -356,8 +356,8 @@ public ProcessorResult getResult() { } { //same with protocol as before with the destination addr - ConfigOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); - ConfigOptions.FIELDS.put(configuration, new HashMap() {{ + PcapOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); + PcapOptions.FIELDS.put(configuration, new HashMap() {{ put(Constants.Fields.PROTOCOL.getName(), "foo"); }}); PcapJob> job = new PcapJob<>(); @@ -367,7 +367,7 @@ public ProcessorResult getResult() { waitForJob(results); Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); - Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + Iterable bytes = Iterables.transform(results.get(), path -> { try { return HDFSUtils.readBytes(path); } catch (IOException e) { @@ -380,8 +380,8 @@ public ProcessorResult getResult() { { //same with protocol as before with the destination addr //test with query filter - ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); - ConfigOptions.FIELDS.put(configuration, "protocol == 'foo'"); + PcapOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + PcapOptions.FIELDS.put(configuration, "protocol == 'foo'"); PcapJob job = new PcapJob<>(); Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); @@ -389,7 +389,7 @@ public ProcessorResult getResult() { waitForJob(results); Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); - Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + Iterable bytes = Iterables.transform(results.get(), path -> { try { return HDFSUtils.readBytes(path); } catch (IOException e) { @@ -401,10 +401,10 @@ public ProcessorResult getResult() { } { //make sure I get them all. - ConfigOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); - ConfigOptions.FIELDS.put(configuration, new HashMap<>()); - ConfigOptions.START_TIME_NS.put(configuration, getTimestamp(0, pcapEntries)); - ConfigOptions.END_TIME_NS.put(configuration, getTimestamp(pcapEntries.size()-1, pcapEntries) + 1); + PcapOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); + PcapOptions.FIELDS.put(configuration, new HashMap<>()); + PcapOptions.START_TIME_NS.put(configuration, getTimestamp(0, pcapEntries)); + PcapOptions.END_TIME_NS.put(configuration, getTimestamp(pcapEntries.size()-1, pcapEntries) + 1); PcapJob> job = new PcapJob<>(); Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); @@ -412,7 +412,7 @@ public ProcessorResult getResult() { waitForJob(results); Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); - Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + Iterable bytes = Iterables.transform(results.get(), path -> { try { return HDFSUtils.readBytes(path); } catch (IOException e) { @@ -425,10 +425,10 @@ public ProcessorResult getResult() { { //make sure I get them all. //with query filter - ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); - ConfigOptions.FIELDS.put(configuration, ""); - ConfigOptions.START_TIME_NS.put(configuration, getTimestamp(0, pcapEntries)); - ConfigOptions.END_TIME_NS.put(configuration, getTimestamp(pcapEntries.size()-1, pcapEntries) + 1); + PcapOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + PcapOptions.FIELDS.put(configuration, ""); + PcapOptions.START_TIME_NS.put(configuration, getTimestamp(0, pcapEntries)); + PcapOptions.END_TIME_NS.put(configuration, getTimestamp(pcapEntries.size()-1, pcapEntries) + 1); PcapJob job = new PcapJob<>(); Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); @@ -436,7 +436,7 @@ public ProcessorResult getResult() { waitForJob(results); Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); - Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + Iterable bytes = Iterables.transform(results.get(), path -> { try { return HDFSUtils.readBytes(path); } catch (IOException e) { @@ -447,11 +447,11 @@ public ProcessorResult getResult() { Assert.assertEquals(10, results.get().getSize()); } { - ConfigOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); - ConfigOptions.FIELDS.put(configuration, new HashMap() {{ + PcapOptions.FILTER_IMPL.put(configuration, new FixedPcapFilter.Configurator()); + PcapOptions.FIELDS.put(configuration, new HashMap() {{ put(Constants.Fields.DST_PORT.getName(), "22"); }}); - ConfigOptions.NUM_RECORDS_PER_FILE.put(configuration, 1); + PcapOptions.NUM_RECORDS_PER_FILE.put(configuration, 1); PcapJob> job = new PcapJob<>(); Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); @@ -459,7 +459,7 @@ public ProcessorResult getResult() { waitForJob(results); Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); - Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + Iterable bytes = Iterables.transform(results.get(), path -> { try { return HDFSUtils.readBytes(path); } catch (IOException e) { @@ -485,8 +485,8 @@ public boolean apply(@Nullable JSONObject input) { { //same with protocol as before with the destination addr //test with query filter - ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); - ConfigOptions.FIELDS.put(configuration, "ip_dst_port == 22"); + PcapOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + PcapOptions.FIELDS.put(configuration, "ip_dst_port == 22"); PcapJob job = new PcapJob<>(); Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); @@ -494,7 +494,7 @@ public boolean apply(@Nullable JSONObject input) { waitForJob(results); Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); - Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + Iterable bytes = Iterables.transform(results.get(), path -> { try { return HDFSUtils.readBytes(path); } catch (IOException e) { @@ -518,8 +518,8 @@ public boolean apply(@Nullable JSONObject input) { } { // test with query filter ip_dst_port > 20 and ip_dst_port < 55792 - ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); - ConfigOptions.FIELDS.put(configuration, "ip_dst_port > 20 and ip_dst_port < 55792"); + PcapOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + PcapOptions.FIELDS.put(configuration, "ip_dst_port > 20 and ip_dst_port < 55792"); PcapJob job = new PcapJob<>(); Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); @@ -527,7 +527,7 @@ public boolean apply(@Nullable JSONObject input) { waitForJob(results); Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); - Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + Iterable bytes = Iterables.transform(results.get(), path -> { try { return HDFSUtils.readBytes(path); } catch (IOException e) { @@ -551,8 +551,8 @@ public boolean apply(@Nullable JSONObject input) { } { //test with query filter ip_dst_port > 55790 - ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); - ConfigOptions.FIELDS.put(configuration, "ip_dst_port > 55790"); + PcapOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + PcapOptions.FIELDS.put(configuration, "ip_dst_port > 55790"); PcapJob job = new PcapJob<>(); Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); @@ -560,7 +560,7 @@ public boolean apply(@Nullable JSONObject input) { waitForJob(results); Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); - Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + Iterable bytes = Iterables.transform(results.get(), path -> { try { return HDFSUtils.readBytes(path); } catch (IOException e) { @@ -584,10 +584,10 @@ public boolean apply(@Nullable JSONObject input) { } { //test with query filter and byte array matching - ConfigOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); - ConfigOptions.FIELDS.put(configuration, "BYTEARRAY_MATCHER('2f56abd814bc56420489ca38e7faf8cec3d4', packet)"); - ConfigOptions.START_TIME_NS.put(configuration, getTimestamp(0, pcapEntries)); - ConfigOptions.END_TIME_NS.put(configuration, getTimestamp(pcapEntries.size()-1, pcapEntries) + 1); + PcapOptions.FILTER_IMPL.put(configuration, new QueryPcapFilter.Configurator()); + PcapOptions.FIELDS.put(configuration, "BYTEARRAY_MATCHER('2f56abd814bc56420489ca38e7faf8cec3d4', packet)"); + PcapOptions.START_TIME_NS.put(configuration, getTimestamp(0, pcapEntries)); + PcapOptions.END_TIME_NS.put(configuration, getTimestamp(pcapEntries.size()-1, pcapEntries) + 1); PcapJob job = new PcapJob<>(); Statusable results = job.submit(PcapFinalizerStrategies.CLI, configuration); Assert.assertEquals(Statusable.JobType.MAP_REDUCE, results.getJobType()); @@ -595,7 +595,7 @@ public boolean apply(@Nullable JSONObject input) { waitForJob(results); Assert.assertEquals(JobStatus.State.SUCCEEDED, results.getStatus().getState()); - Iterable bytes = Iterables.transform(results.get().asIterable(), path -> { + Iterable bytes = Iterables.transform(results.get(), path -> { try { return HDFSUtils.readBytes(path); } catch (IOException e) { From 7052982eb6626b02c90c42d55fdcf8bfc0db3a59 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Mon, 16 Jul 2018 14:45:40 -0600 Subject: [PATCH 23/36] Add README to metron-job --- metron-platform/metron-job/README.md | 26 +++++++++++++++++++ .../metron-job_state_statechart_diagram.svg | 14 ++++++++++ .../metron-job_state_statechart_diagram.xml | 14 ++++++++++ 3 files changed, 54 insertions(+) create mode 100644 metron-platform/metron-job/README.md create mode 100644 metron-platform/metron-job/metron-job_state_statechart_diagram.svg create mode 100644 metron-platform/metron-job/metron-job_state_statechart_diagram.xml diff --git a/metron-platform/metron-job/README.md b/metron-platform/metron-job/README.md new file mode 100644 index 0000000000..19b006c11f --- /dev/null +++ b/metron-platform/metron-job/README.md @@ -0,0 +1,26 @@ + + +# Metron Job + +This module holds abstractions for creating jobs. The main actors are a JobManager interface and subsequent implementation, InMemoryJobManger, that handles maintaining a cache of running and completed Statusable jobs. Each Statusable +can provide a Finalizer implementation that should be executed on completion of the underlying job. Successful jobs should return a Pageable object that allow consumers to request results on a per-page basis. + +## Job State Statechart + +![Job State Statechart](metron-job_state_statechart_diagram.svg) diff --git a/metron-platform/metron-job/metron-job_state_statechart_diagram.svg b/metron-platform/metron-job/metron-job_state_statechart_diagram.svg new file mode 100644 index 0000000000..a99c5ad9e4 --- /dev/null +++ b/metron-platform/metron-job/metron-job_state_statechart_diagram.svg @@ -0,0 +1,14 @@ + + +
NOT_RUNNING
NOT_RUNNING
RUNNING
RUNNING
FINALIZING
FINALIZING
KILLED
KILLED
FAILED
FAILED
SUCCEEDED
SUCCEEDED
diff --git a/metron-platform/metron-job/metron-job_state_statechart_diagram.xml b/metron-platform/metron-job/metron-job_state_statechart_diagram.xml new file mode 100644 index 0000000000..b9ee8aab0c --- /dev/null +++ b/metron-platform/metron-job/metron-job_state_statechart_diagram.xml @@ -0,0 +1,14 @@ + + +7VrbctowEP0aHtvxTdg8BkJSpintlKZN+tJRbGFrIixGiAL9+kq2fAfiOhjIFF6wdqWVtOfsauVxxxzM1rcMzoNP1EOkY2jeumNedwzD0GxN/EnJJpb0eiAW+Ax7sUjPBBP8BymhGucvsYcWhY6cUsLxvCh0aRgilxdkkDG6KnabUlKcdQ59VBFMXEiq0h/Y40EsdYCWyT8g7AfJzLqmNE/QffYZXYZqvo5hTqNfrJ7BxJbqvwigR1c5kTnsmANGKY+fZusBItK3idvicTc7tOm6GQp5rQGOY1sAOK6NtCdNs98pC78hWaJkC9FC+SZxjrAgcBCNfsBnRMh08Sg2Mpf6BYeMTzjkUj/FhAwooSwaaGrRT3bmjD6jnGY6VZrIdcjLjMbekSqXzrCrngl8QqSf+jqxFNIwmpaG/AbOMJHs+46YB0OoxIpquqHa21YHCfZDIXOFD5FQ9j3MBMkwlcIFXUoI+8pNiHG03ul7PUVURAqiM8TZRnRRAxzFARUjaXuVMc5UoiBHtkQGFcf91HCGs3hQUNeE3ajAPv787dfX+/F4NL6tMKCI0irAHE3m0JXalUgIRWZA5iqvG1aVEzI23P2cOBMWHABxvVeCvAsqkOvGFsy7bWBuViPbE1lQNSnjAfVpCMkwk+ZxRWvMH3LPjxKV90C2QrG0BwVS1Mh0+zFDoXcl07ecf47CWKLQcvaSpCbaCaoETXmKqdx2AVER5MxFO9xmqfMIMh/xHX1AbWYwRCDHv4vzHxRmqxLal7A+bFhb2jmFNajgfTMaX92Nfl4gPxzkXeOcIO++LpO//Zyc3BH25WTjjHKyfTl6m8Bs1IDZOjzMaugXioXFNAWYpWLOLBfm8VbUqPwd7AVDhlYyFO+3YihiXbqfZkR0KofFx9Hd3fD6clC0VBuYzikPil61NrgaXeBuD26rd0q4k9dpObwn94PBcHh9gby1UvC0Ea7rFWAblRaqnFDFhV4oLbJK47EWaG+htKhzq3eOVVoAs0QpCzQrLbovGWqxtNC3vDn+Jya+Gb7tjpkmTAQ1mJgk9vapaNslBoGmVLR729PkMaj42ledCRX1zpnfqlIqFm6GNmiRi71jUbFbuXA1pWLZUO+IVLQOlRX/MyrWOaCPR8XyuWo0vPuXT3pQ5vQOKgok4SbXbS47LHYv2OyWFqxre9dV7g8MUIqEeAWHiwtwSdGtpeijFa4nj4uXqSia2ecVcffsGxZz+Bc= From bcb2af02a1718b12da75b78c192a840c12bb4789 Mon Sep 17 00:00:00 2001 From: merrimanr Date: Mon, 16 Jul 2018 16:23:15 -0500 Subject: [PATCH 24/36] rest endpoint for job status --- .../rest/model/pcap/FixedPcapOptions.java | 42 ++++ .../rest/model/pcap/FixedPcapRequest.java | 72 ++++-- .../metron/rest/model/pcap/PcapRequest.java | 65 +++--- .../metron/rest/model/pcap/PcapStatus.java | 91 ++++++++ .../metron/rest/MetronRestConstants.java | 6 +- .../apache/metron/rest/config/PcapConfig.java | 14 +- .../metron/rest/config/PcapJobSupplier.java | 53 +++++ .../rest/controller/PcapController.java | 34 +-- .../metron/rest/service/PcapService.java | 6 +- .../rest/service/impl/PcapServiceImpl.java | 117 +++++----- .../src/main/resources/application.yml | 6 +- .../apache/metron/rest/config/TestConfig.java | 17 +- .../PcapControllerIntegrationTest.java | 119 +++++++++- .../apache/metron/rest/mock/MockPcapJob.java | 102 ++++++--- .../metron/rest/mock/MockPcapJobSupplier.java | 36 +++ .../service/impl/PcapServiceImplTest.java | 210 ++++++++++++------ .../common/configuration/ConfigOption.java | 12 +- .../metron/job/JobNotFoundException.java | 30 +++ .../job/manager/InMemoryJobManager.java | 11 +- .../org/apache/metron/pcap/mr/PcapJob.java | 24 +- 20 files changed, 807 insertions(+), 260 deletions(-) create mode 100644 metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/FixedPcapOptions.java create mode 100644 metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapStatus.java create mode 100644 metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapJobSupplier.java create mode 100644 metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJobSupplier.java create mode 100644 metron-platform/metron-job/src/main/java/org/apache/metron/job/JobNotFoundException.java diff --git a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/FixedPcapOptions.java b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/FixedPcapOptions.java new file mode 100644 index 0000000000..5e77005b1c --- /dev/null +++ b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/FixedPcapOptions.java @@ -0,0 +1,42 @@ +/** + * 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.metron.rest.model.pcap; + +import org.apache.metron.common.configuration.ConfigOption; + +public enum FixedPcapOptions implements ConfigOption { + IP_SRC_ADDR("ipSrcAddr"), + IP_DST_ADDR("ipDstAddr"), + IP_SRC_PORT("ipSrcPort"), + IP_DST_PORT("ipDstPort"), + PROTOCOL("protocol"), + PACKET_FILTER("packetFilter"), + INCLUDE_REVERSE("includeReverse") + ; + + String key; + + FixedPcapOptions(String key) { + this.key = key; + } + + @Override + public String getKey() { + return key; + } +} diff --git a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/FixedPcapRequest.java b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/FixedPcapRequest.java index 758340bbdb..a2d345bd07 100644 --- a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/FixedPcapRequest.java +++ b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/FixedPcapRequest.java @@ -17,70 +17,100 @@ */ package org.apache.metron.rest.model.pcap; +import org.apache.metron.common.Constants; +import org.apache.metron.pcap.config.PcapOptions; +import org.apache.metron.pcap.PcapHelper; +import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; + +import java.util.HashMap; +import java.util.Map; + public class FixedPcapRequest extends PcapRequest { - private String ipSrcAddr; - private String ipDstAddr; - private Integer ipSrcPort; - private Integer ipDstPort; - private String protocol; - private String packetFilter; - private Boolean includeReverse = false; + public FixedPcapRequest() { + PcapOptions.FILTER_IMPL.put(this, new FixedPcapFilter.Configurator()); + } public String getIpSrcAddr() { - return ipSrcAddr; + return FixedPcapOptions.IP_SRC_ADDR.get(this, String.class); } public void setIpSrcAddr(String ipSrcAddr) { - this.ipSrcAddr = ipSrcAddr; + FixedPcapOptions.IP_SRC_ADDR.put(this, ipSrcAddr); } public String getIpDstAddr() { - return ipDstAddr; + return FixedPcapOptions.IP_DST_ADDR.get(this, String.class); } public void setIpDstAddr(String ipDstAddr) { - this.ipDstAddr = ipDstAddr; + FixedPcapOptions.IP_DST_ADDR.put(this, ipDstAddr); } public Integer getIpSrcPort() { - return ipSrcPort; + return FixedPcapOptions.IP_SRC_PORT.get(this, Integer.class); } public void setIpSrcPort(Integer ipSrcPort) { - this.ipSrcPort = ipSrcPort; + FixedPcapOptions.IP_SRC_PORT.put(this, ipSrcPort); } public Integer getIpDstPort() { - return ipDstPort; + return FixedPcapOptions.IP_DST_PORT.get(this, Integer.class); } public void setIpDstPort(Integer ipDstPort) { - this.ipDstPort = ipDstPort; + FixedPcapOptions.IP_DST_PORT.put(this, ipDstPort); } public String getProtocol() { - return protocol; + return FixedPcapOptions.PROTOCOL.get(this, String.class); } public void setProtocol(String protocol) { - this.protocol = protocol; + FixedPcapOptions.PROTOCOL.put(this, protocol); } public String getPacketFilter() { - return packetFilter; + return FixedPcapOptions.PACKET_FILTER.get(this, String.class); } public void setPacketFilter(String packetFilter) { - this.packetFilter = packetFilter; + FixedPcapOptions.PACKET_FILTER.put(this, packetFilter); } public Boolean getIncludeReverse() { - return includeReverse; + return FixedPcapOptions.INCLUDE_REVERSE.get(this, Boolean.class); } public void setIncludeReverse(Boolean includeReverse) { - this.includeReverse = includeReverse; + FixedPcapOptions.INCLUDE_REVERSE.put(this, includeReverse); + } + + public void setFields() { + Map fields = new HashMap<>(); + if (getIpSrcAddr() != null) { + fields.put(Constants.Fields.SRC_ADDR.getName(), getIpSrcAddr()); + } + if (getIpDstAddr() != null) { + fields.put(Constants.Fields.DST_ADDR.getName(), getIpDstAddr()); + } + if (getIpSrcPort() != null) { + fields.put(Constants.Fields.SRC_PORT.getName(), getIpSrcPort().toString()); + } + if (getIpDstPort() != null) { + fields.put(Constants.Fields.DST_PORT.getName(), getIpDstPort().toString()); + } + if (getProtocol() != null) { + fields.put(Constants.Fields.PROTOCOL.getName(), getProtocol()); + } + if (getIncludeReverse() != null) { + fields.put(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName(), getIncludeReverse().toString()); + } + if (getPacketFilter() != null) { + fields.put(PcapHelper.PacketFields.PACKET_FILTER.getName(), getPacketFilter()); + } + PcapOptions.FIELDS.put(this, fields); } @Override diff --git a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java index cfc212aa84..6d08dd28b6 100644 --- a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java +++ b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java @@ -17,27 +17,20 @@ */ package org.apache.metron.rest.model.pcap; -// TODO reconcile with pcapmrjob - import org.apache.commons.collections4.map.AbstractMapDecorator; import org.apache.metron.pcap.config.PcapOptions; +import java.util.HashMap; + public class PcapRequest extends AbstractMapDecorator { public PcapRequest() { - setStartTime(0L); - setEndTime(System.currentTimeMillis()); + super(new HashMap<>()); + setStartTimeMs(0L); + setEndTimeMs(System.currentTimeMillis()); setNumReducers(1); } - public String getBaseOutputPath() { - return PcapOptions.INTERIM_RESULT_PATH.get(this, String.class); - } - - public void setBaseOutputPath(String baseOutputPath) { - PcapOptions.INTERIM_RESULT_PATH.put(this, baseOutputPath); - } - public String getBasePath() { return PcapOptions.BASE_PATH.get(this, String.class); } @@ -46,19 +39,35 @@ public void setBasePath(String basePath) { PcapOptions.BASE_PATH.put(this, basePath); } - public Long getStartTime() { + public String getBaseInterimResultPath() { + return PcapOptions.BASE_INTERRIM_RESULT_PATH.get(this, String.class); + } + + public void setBaseInterimResultPath(String baseInterimResultPath) { + PcapOptions.BASE_INTERRIM_RESULT_PATH.put(this, baseInterimResultPath); + } + + public String getFinalOutputPath() { + return PcapOptions.FINAL_OUTPUT_PATH.get(this, String.class); + } + + public void setFinalOutputPath(String finalOutputPath) { + PcapOptions.FINAL_OUTPUT_PATH.put(this, finalOutputPath); + } + + public Long getStartTimeMs() { return PcapOptions.START_TIME_MS.get(this, Long.class); } - public void setStartTime(Long startTime) { + public void setStartTimeMs(Long startTime) { PcapOptions.START_TIME_MS.put(this, startTime); } - public Long getEndTime() { + public Long getEndTimeMs() { return PcapOptions.END_TIME_MS.get(this, Long.class); } - public void setEndTime(Long endTime) { + public void setEndTimeMs(Long endTime) { PcapOptions.END_TIME_MS.put(this, endTime); } @@ -69,28 +78,4 @@ public Integer getNumReducers() { public void setNumReducers(Integer numReducers) { PcapOptions.NUM_REDUCERS.put(this, numReducers); } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - PcapRequest pcapRequest = (PcapRequest) o; - - return (getBaseOutputPath() != null ? getBaseOutputPath().equals(pcapRequest.getBaseOutputPath()) : pcapRequest.getBaseOutputPath() != null) && - (getBasePath() != null ? getBasePath().equals(pcapRequest.getBasePath()) : pcapRequest.getBasePath() == null) && - (getStartTime() != null ? getStartTime().equals(pcapRequest.getStartTime()) : pcapRequest.getStartTime() == null) && - (getEndTime() != null ? getEndTime().equals(pcapRequest.getEndTime()) : pcapRequest.getEndTime() == null) && - (getNumReducers() != null ? getNumReducers().equals(pcapRequest.getNumReducers()) : pcapRequest.getNumReducers() == null); - } - - @Override - public int hashCode() { - int result = getBaseOutputPath() != null ? getBaseOutputPath().hashCode() : 0; - result = 31 * result + (getBasePath() != null ? getBasePath().hashCode() : 0); - result = 31 * result + (getStartTime() != null ? getStartTime().hashCode() : 0); - result = 31 * result + (getEndTime() != null ? getEndTime().hashCode() : 0); - result = 31 * result + (getNumReducers() != null ? getNumReducers().hashCode() : 0); - return result; - } } diff --git a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapStatus.java b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapStatus.java new file mode 100644 index 0000000000..fe0e51c54e --- /dev/null +++ b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapStatus.java @@ -0,0 +1,91 @@ +/** + * 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.metron.rest.model.pcap; + +public class PcapStatus { + + private String jobId; + private String jobStatus; + private String description; + private Double percentComplete = 0.0; + private Integer size = 0; + + public String getJobId() { + return jobId; + } + + public void setJobId(String jobId) { + this.jobId = jobId; + } + + public String getJobStatus() { + return jobStatus; + } + + public void setJobStatus(String jobStatus) { + this.jobStatus = jobStatus; + } + + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + public Double getPercentComplete() { + return percentComplete; + } + + public void setPercentComplete(Double percentComplete) { + this.percentComplete = percentComplete; + } + + public Integer getSize() { + return size; + } + + public void setSize(Integer size) { + this.size = size; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + PcapStatus pcapStatus = (PcapStatus) o; + + return (getJobId() != null ? getJobId().equals(pcapStatus.getJobId()) : pcapStatus.getJobId() != null) && + (getJobStatus() != null ? getJobStatus().equals(pcapStatus.getJobStatus()) : pcapStatus.getJobStatus() != null) && + (getDescription() != null ? getDescription().equals(pcapStatus.getDescription()) : pcapStatus.getDescription() != null) && + (getPercentComplete() != null ? getPercentComplete().equals(pcapStatus.getPercentComplete()) : pcapStatus.getPercentComplete() != null) && + (getSize() != null ? getSize().equals(pcapStatus.getSize()) : pcapStatus.getSize() != null); + } + + @Override + public int hashCode() { + int result = (getJobId() != null ? getJobId().hashCode() : 0); + result = 31 * result + (getJobStatus() != null ? getJobStatus().hashCode() : 0); + result = 31 * result + (getDescription() != null ? getDescription().hashCode() : 0); + result = 31 * result + (getPercentComplete() != null ? getPercentComplete().hashCode() : 0); + result = 31 * result + (getSize() != null ? getSize().hashCode() : 0); + return result; + } +} diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java index 0989d12cb9..8e14e388e7 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java @@ -75,6 +75,8 @@ public class MetronRestConstants { public static final String LOGGING_SYSTEM_PROPERTY = "org.springframework.boot.logging.LoggingSystem"; - public static final String PCAP_INPUT_PATH_SPRING_PROPERTY = "pcap.input.path"; - public static final String PCAP_OUTPUT_PATH_SPRING_PROPERTY = "pcap.output.path"; + public static final String PCAP_BASE_PATH_SPRING_PROPERTY = "pcap.base.path"; + public static final String PCAP_BASE_INTERIM_RESULT_PATH_SPRING_PROPERTY = "pcap.base.interim.result.path"; + public static final String PCAP_FINAL_OUTPUT_PATH_SPRING_PROPERTY = "pcap.final.output.path"; + public static final String PCAP_PAGE_SIZE_SPRING_PROPERTY = "pcap.page.size"; } diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapConfig.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapConfig.java index 8da5f963ab..a0b7f18198 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapConfig.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapConfig.java @@ -17,7 +17,8 @@ */ package org.apache.metron.rest.config; -import org.apache.metron.pcap.mr.PcapJob; +import org.apache.metron.job.manager.InMemoryJobManager; +import org.apache.metron.job.manager.JobManager; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; import org.springframework.context.annotation.Profile; @@ -29,7 +30,14 @@ public class PcapConfig { @Bean - public PcapJob pcapJob() { - return new PcapJob(); + public JobManager jobManager() { + return new InMemoryJobManager(); } + + @Bean + public PcapJobSupplier pcapJobSupplier() { + return new PcapJobSupplier(); + } + + } diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapJobSupplier.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapJobSupplier.java new file mode 100644 index 0000000000..3e1e5f69c7 --- /dev/null +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapJobSupplier.java @@ -0,0 +1,53 @@ +/** + * 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.metron.rest.config; + +import org.apache.hadoop.fs.Path; +import org.apache.metron.job.JobException; +import org.apache.metron.job.JobStatus; +import org.apache.metron.job.Statusable; +import org.apache.metron.pcap.finalizer.PcapFinalizerStrategies; +import org.apache.metron.pcap.mr.PcapJob; +import org.apache.metron.rest.model.pcap.PcapRequest; + +import java.util.function.Supplier; + +public class PcapJobSupplier implements Supplier> { + + private PcapRequest pcapRequest; + + @Override + public Statusable get() { + try { + PcapJob pcapJob = createPcapJob(); + return pcapJob.submit(PcapFinalizerStrategies.REST, pcapRequest); + } catch (JobException e) { + return null; + //return new JobStatus().withState(JobStatus.State.FAILED).withDescription(JobStatus.State.FAILED.toString()); + } + } + + public void setPcapRequest(PcapRequest pcapRequest) { + this.pcapRequest = pcapRequest; + } + + protected PcapJob createPcapJob() { + return new PcapJob(); + } + +} diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/PcapController.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/PcapController.java index 3524a8cd67..38bffb48f3 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/PcapController.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/PcapController.java @@ -21,10 +21,14 @@ import io.swagger.annotations.ApiParam; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; +import org.apache.hadoop.fs.Path; import org.apache.metron.job.JobStatus; +import org.apache.metron.job.Statusable; import org.apache.metron.rest.RestException; import org.apache.metron.rest.model.PcapResponse; import org.apache.metron.rest.model.pcap.FixedPcapRequest; +import org.apache.metron.rest.model.pcap.PcapStatus; +import org.apache.metron.rest.security.SecurityUtils; import org.apache.metron.rest.service.PcapService; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.http.HttpStatus; @@ -33,8 +37,12 @@ import org.springframework.web.bind.annotation.RequestBody; import org.springframework.web.bind.annotation.RequestMapping; import org.springframework.web.bind.annotation.RequestMethod; +import org.springframework.web.bind.annotation.RequestParam; import org.springframework.web.bind.annotation.RestController; +import java.util.List; +import java.util.Set; + @RestController @RequestMapping("/api/v1/pcap") public class PcapController { @@ -45,27 +53,23 @@ public class PcapController { @ApiOperation(value = "Executes a Fixed Pcap Query.") @ApiResponses(value = { @ApiResponse(message = "Returns a job status with job ID.", code = 200)}) @RequestMapping(value = "/fixed", method = RequestMethod.POST) - ResponseEntity fixed(@ApiParam(name="fixedPcapRequest", value="A Fixed Pcap Request" + ResponseEntity fixed(@ApiParam(name="fixedPcapRequest", value="A Fixed Pcap Request" + " which includes fixed filter fields like ip source address and protocol.", required=true)@RequestBody FixedPcapRequest fixedPcapRequest) throws RestException { - JobStatus jobStatus = pcapQueryService.fixed(fixedPcapRequest); - return new ResponseEntity<>(jobStatus, HttpStatus.OK); + PcapStatus pcapStatus = pcapQueryService.fixed(SecurityUtils.getCurrentUser(), fixedPcapRequest); + return new ResponseEntity<>(pcapStatus, HttpStatus.OK); } @ApiOperation(value = "Gets job status for running job.") @ApiResponses(value = { @ApiResponse(message = "Returns a job status for the passed job.", code = 200)}) - @RequestMapping(value = "/getStatus", method = RequestMethod.GET) - ResponseEntity getStatus(@ApiParam(name="jobId", value="Job ID of submitted job" + @RequestMapping(value = "/{jobId}", method = RequestMethod.GET) + ResponseEntity getStatus(@ApiParam(name="jobId", value="Job ID of submitted job" + " which includes fixed filter fields like ip source address and protocol.", required=true)@PathVariable String jobId) throws RestException { - JobStatus jobStatus = pcapQueryService.getJobStatus("metron", jobId); - return new ResponseEntity<>(jobStatus, HttpStatus.OK); - } + PcapStatus jobStatus = pcapQueryService.getJobStatus(SecurityUtils.getCurrentUser(), jobId); + if (jobStatus != null) { + return new ResponseEntity<>(jobStatus, HttpStatus.OK); + } else { + return new ResponseEntity<>(HttpStatus.NOT_FOUND); + } - @ApiOperation(value = "Gets results of a pcap job.") - @ApiResponses(value = { @ApiResponse(message = "Returns a PcapResponse containing an array of pcaps.", code = 200)}) - @RequestMapping(value = "/getPage", method = RequestMethod.GET) - ResponseEntity getPage(@ApiParam(name="fixedPcapRequest", value="Job ID of submitted job" - + " which includes fixed filter fields like ip source address and protocol.", required=true)@RequestBody String jobId, int pageNum) throws RestException { - PcapResponse pcapResponse = pcapQueryService.getPage("metron", jobId, pageNum); - return new ResponseEntity<>(pcapResponse, HttpStatus.OK); } } diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/PcapService.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/PcapService.java index ce8372c5b7..603e0138ae 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/PcapService.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/PcapService.java @@ -18,10 +18,12 @@ package org.apache.metron.rest.service; import org.apache.metron.rest.RestException; -import org.apache.metron.rest.model.PcapResponse; import org.apache.metron.rest.model.pcap.FixedPcapRequest; +import org.apache.metron.rest.model.pcap.PcapStatus; public interface PcapService { - PcapResponse fixed(FixedPcapRequest fixedPcapRequest) throws RestException; + PcapStatus fixed(String username, FixedPcapRequest fixedPcapRequest) throws RestException; + + PcapStatus getJobStatus(String username, String jobId) throws RestException; } diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java index 4dae1e5ec6..1b18f49860 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java @@ -20,101 +20,98 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.metron.common.Constants; -import org.apache.metron.common.hadoop.SequenceFileIterable; -import org.apache.metron.common.utils.timestamp.TimestampConverters; -import org.apache.metron.pcap.PcapHelper; -import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; -import org.apache.metron.pcap.mr.PcapJob; +import org.apache.metron.job.JobException; +import org.apache.metron.job.JobNotFoundException; +import org.apache.metron.job.JobStatus; +import org.apache.metron.job.Statusable; +import org.apache.metron.job.manager.JobManager; +import org.apache.metron.pcap.config.PcapOptions; import org.apache.metron.rest.MetronRestConstants; import org.apache.metron.rest.RestException; -import org.apache.metron.rest.model.PcapResponse; +import org.apache.metron.rest.config.PcapJobSupplier; import org.apache.metron.rest.model.pcap.FixedPcapRequest; +import org.apache.metron.rest.model.pcap.PcapRequest; +import org.apache.metron.rest.model.pcap.PcapStatus; import org.apache.metron.rest.service.PcapService; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.core.env.Environment; import org.springframework.stereotype.Service; import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; @Service public class PcapServiceImpl implements PcapService { private Environment environment; private Configuration configuration; - private PcapJob pcapJob; + private PcapJobSupplier pcapJobSupplier; + private JobManager jobManager; @Autowired - public PcapServiceImpl(Environment environment, Configuration configuration, PcapJob pcapJob) { + public PcapServiceImpl(Environment environment, Configuration configuration, PcapJobSupplier pcapJobSupplier, JobManager jobManager) { this.environment = environment; this.configuration = configuration; - this.pcapJob = pcapJob; + this.pcapJobSupplier = pcapJobSupplier; + this.jobManager = jobManager; } @Override - public PcapResponse fixed(FixedPcapRequest fixedPcapRequest) throws RestException { - if (fixedPcapRequest.getBasePath() == null) { - fixedPcapRequest.setBasePath(environment.getProperty(MetronRestConstants.PCAP_INPUT_PATH_SPRING_PROPERTY)); - } - if (fixedPcapRequest.getBaseOutputPath() == null) { - fixedPcapRequest.setBaseOutputPath(environment.getProperty(MetronRestConstants.PCAP_OUTPUT_PATH_SPRING_PROPERTY)); + public PcapStatus fixed(String username, FixedPcapRequest fixedPcapRequest) throws RestException { + try { + setPcapOptions(fixedPcapRequest); + fixedPcapRequest.setFields(); + pcapJobSupplier.setPcapRequest(fixedPcapRequest); + JobStatus jobStatus = jobManager.submit(pcapJobSupplier, username); + return jobStatusToPcapStatus(jobStatus); + } catch (IOException | JobException e) { + throw new RestException(e); } - PcapResponse response = new PcapResponse(); - SequenceFileIterable results; + } + + @Override + public PcapStatus getJobStatus(String username, String jobId) throws RestException { + PcapStatus pcapStatus = null; try { - results = pcapJob.query( - new Path(fixedPcapRequest.getBasePath()), - new Path(fixedPcapRequest.getBaseOutputPath()), - TimestampConverters.MILLISECONDS.toNanoseconds(fixedPcapRequest.getStartTime()), - TimestampConverters.MILLISECONDS.toNanoseconds(fixedPcapRequest.getEndTime()), - fixedPcapRequest.getNumReducers(), - getFixedFields(fixedPcapRequest), - configuration, - getFileSystem(), - new FixedPcapFilter.Configurator() - ); - if (results != null) { - List pcaps = new ArrayList<>(); - results.iterator().forEachRemaining(pcaps::add); - response.setPcaps(pcaps); + Statusable statusable = jobManager.getJob(username, jobId); + if (statusable != null) { + pcapStatus = jobStatusToPcapStatus(statusable.getStatus()); } - } catch (IOException | ClassNotFoundException | InterruptedException e) { + } catch (JobNotFoundException e) { + // do nothing and return null pcapStatus + } catch (JobException e) { throw new RestException(e); } - return response; + return pcapStatus; } - protected Map getFixedFields(FixedPcapRequest fixedPcapRequest) { - Map fixedFields = new HashMap<>(); - if (fixedPcapRequest.getIpSrcAddr() != null) { - fixedFields.put(Constants.Fields.SRC_ADDR.getName(), fixedPcapRequest.getIpSrcAddr()); - } - if (fixedPcapRequest.getIpDstAddr() != null) { - fixedFields.put(Constants.Fields.DST_ADDR.getName(), fixedPcapRequest.getIpDstAddr()); - } - if (fixedPcapRequest.getIpSrcPort() != null) { - fixedFields.put(Constants.Fields.SRC_PORT.getName(), fixedPcapRequest.getIpSrcPort().toString()); - } - if (fixedPcapRequest.getIpDstPort() != null) { - fixedFields.put(Constants.Fields.DST_PORT.getName(), fixedPcapRequest.getIpDstPort().toString()); - } - if (fixedPcapRequest.getProtocol() != null) { - fixedFields.put(Constants.Fields.PROTOCOL.getName(), fixedPcapRequest.getProtocol()); + protected void setPcapOptions(PcapRequest pcapRequest) throws IOException { + PcapOptions.JOB_NAME.put(pcapRequest, "jobName"); + PcapOptions.HADOOP_CONF.put(pcapRequest, configuration); + PcapOptions.FILESYSTEM.put(pcapRequest, getFileSystem()); + + if (pcapRequest.getBasePath() == null) { + pcapRequest.setBasePath(environment.getProperty(MetronRestConstants.PCAP_BASE_PATH_SPRING_PROPERTY)); } - if (fixedPcapRequest.getIncludeReverse() != null) { - fixedFields.put(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName(), fixedPcapRequest.getIncludeReverse().toString()); + if (pcapRequest.getBaseInterimResultPath() == null) { + pcapRequest.setBaseInterimResultPath(environment.getProperty(MetronRestConstants.PCAP_BASE_INTERIM_RESULT_PATH_SPRING_PROPERTY)); } - if (fixedPcapRequest.getPacketFilter() != null) { - fixedFields.put(PcapHelper.PacketFields.PACKET_FILTER.getName(), fixedPcapRequest.getPacketFilter()); + if (pcapRequest.getFinalOutputPath() == null) { + pcapRequest.setFinalOutputPath(environment.getProperty(MetronRestConstants.PCAP_FINAL_OUTPUT_PATH_SPRING_PROPERTY)); } - return fixedFields; + + PcapOptions.NUM_RECORDS_PER_FILE.put(pcapRequest, Integer.parseInt(environment.getProperty(MetronRestConstants.PCAP_PAGE_SIZE_SPRING_PROPERTY))); } protected FileSystem getFileSystem() throws IOException { return FileSystem.get(configuration); } + + protected PcapStatus jobStatusToPcapStatus(JobStatus jobStatus) { + PcapStatus pcapStatus = new PcapStatus(); + pcapStatus.setJobId(jobStatus.getJobId()); + pcapStatus.setJobStatus(jobStatus.getState().toString()); + pcapStatus.setDescription(jobStatus.getDescription()); + pcapStatus.setPercentComplete(jobStatus.getPercentComplete()); + return pcapStatus; + } } diff --git a/metron-interface/metron-rest/src/main/resources/application.yml b/metron-interface/metron-rest/src/main/resources/application.yml index 10c2f5061d..5fd9d72d83 100644 --- a/metron-interface/metron-rest/src/main/resources/application.yml +++ b/metron-interface/metron-rest/src/main/resources/application.yml @@ -74,5 +74,7 @@ user: cf: cf pcap: - input.path: /apps/metron/pcap - output.path: /tmp \ No newline at end of file + base.path: /apps/metron/pcap/input + base.interim.result.path: /apps/metron/pcap/interim + final.output.path: /apps/metron/pcap/output + page.size: 10 \ No newline at end of file diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java index a9e70d2c0b..486a7dcaf1 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java @@ -37,9 +37,12 @@ import org.apache.metron.integration.UnableToStartException; import org.apache.metron.integration.components.KafkaComponent; import org.apache.metron.integration.components.ZKServerComponent; +import org.apache.metron.job.manager.InMemoryJobManager; +import org.apache.metron.job.manager.JobManager; import org.apache.metron.pcap.mr.PcapJob; import org.apache.metron.rest.RestException; import org.apache.metron.rest.mock.MockPcapJob; +import org.apache.metron.rest.mock.MockPcapJobSupplier; import org.apache.metron.rest.mock.MockStormCLIClientWrapper; import org.apache.metron.rest.mock.MockStormRestTemplate; import org.apache.metron.rest.service.impl.StormCLIWrapper; @@ -189,7 +192,19 @@ public UserSettingsClient userSettingsClient() throws RestException, IOException } @Bean - public PcapJob mockPcapJob() { + public JobManager jobManager() { + return new InMemoryJobManager(); + } + + @Bean + public MockPcapJob mockPcapJob() { return new MockPcapJob(); } + + @Bean + public PcapJobSupplier pcapJobSupplier(MockPcapJob mockPcapJob) { + MockPcapJobSupplier mockPcapJobSupplier = new MockPcapJobSupplier(); + mockPcapJobSupplier.setMockPcapJob(mockPcapJob); + return mockPcapJobSupplier; + } } diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/PcapControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/PcapControllerIntegrationTest.java index 5e4875aad9..1b43bdf28c 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/PcapControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/PcapControllerIntegrationTest.java @@ -20,6 +20,7 @@ import org.adrianwalker.multilinestring.Multiline; import org.apache.metron.common.Constants; import org.apache.metron.common.utils.JSONUtils; +import org.apache.metron.job.JobStatus; import org.apache.metron.pcap.PcapHelper; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.rest.mock.MockPcapJob; @@ -43,11 +44,14 @@ import java.util.Map; import static org.apache.metron.rest.MetronRestConstants.TEST_PROFILE; +import static org.hamcrest.Matchers.hasSize; import static org.springframework.security.test.web.servlet.request.SecurityMockMvcRequestPostProcessors.csrf; import static org.springframework.security.test.web.servlet.request.SecurityMockMvcRequestPostProcessors.httpBasic; import static org.springframework.security.test.web.servlet.setup.SecurityMockMvcConfigurers.springSecurity; +import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get; import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post; import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content; +import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.jsonPath; import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status; @RunWith(SpringRunner.class) @@ -57,23 +61,38 @@ public class PcapControllerIntegrationTest { /** { - "basePath": "/apps/metron/pcap", - "baseOutputPath": "/tmp", - "endTime": 10, + "basePath": "/base/path", + "baseInterimResultPath": "/base/interim/result/path", + "finalOutputPath": "/final/output/path", + "startTimeMs": 10, + "endTimeMs": 20, + "numReducers": 2, "includeReverse": "true", "ipDstAddr": "192.168.1.1", "ipDstPort": "1000", "ipSrcAddr": "192.168.1.2", "ipSrcPort": "2000", - "numReducers": 2, "packetFilter": "filter", - "protocol": "TCP", - "startTime": 1 + "protocol": "TCP" } */ @Multiline public static String fixedJson; + /** + { + "includeReverse": "true", + "ipDstAddr": "192.168.1.1", + "ipDstPort": "1000", + "ipSrcAddr": "192.168.1.2", + "ipSrcPort": "2000", + "packetFilter": "filter", + "protocol": "TCP" + } + */ + @Multiline + public static String fixedWithDefaultsJson; + @Autowired private PcapService pcapService; @@ -84,6 +103,7 @@ public class PcapControllerIntegrationTest { private String pcapUrl = "/api/v1/pcap"; private String user = "user"; + private String user2 = "user2"; private String password = "password"; @Before @@ -98,22 +118,24 @@ public void testSecurity() throws Exception { } @Test - public void testFixed() throws Exception { + public void testFixedRequest() throws Exception { MockPcapJob mockPcapJob = (MockPcapJob) wac.getBean("mockPcapJob"); List results = Arrays.asList("pcap1".getBytes(), "pcap2".getBytes()); mockPcapJob.setResults(results); + mockPcapJob.setStatus(new JobStatus().withState(JobStatus.State.RUNNING)); PcapResponse expectedReponse = new PcapResponse(); expectedReponse.setPcaps(results); this.mockMvc.perform(post(pcapUrl + "/fixed").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(fixedJson)) .andExpect(status().isOk()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) - .andExpect(content().json(JSONUtils.INSTANCE.toJSON(expectedReponse, false))); + .andExpect(jsonPath("$.jobStatus").value("RUNNING")); - Assert.assertEquals("/apps/metron/pcap", mockPcapJob.getBasePath()); - Assert.assertEquals("/tmp", mockPcapJob.getBaseOutputPath()); - Assert.assertEquals(1, mockPcapJob.getStartTime()); - Assert.assertEquals(10, mockPcapJob.getEndTime()); + Assert.assertEquals("/base/path", mockPcapJob.getBasePath()); + Assert.assertEquals("/base/interim/result/path", mockPcapJob.getBaseInterrimResultPath()); + Assert.assertEquals("/final/output/path", mockPcapJob.getFinalOutputPath()); + Assert.assertEquals(10000000, mockPcapJob.getStartTimeNs()); + Assert.assertEquals(20000000, mockPcapJob.getEndTimeNs()); Assert.assertEquals(2, mockPcapJob.getNumReducers()); Assert.assertTrue(mockPcapJob.getFilterImpl() instanceof FixedPcapFilter.Configurator); Map actualFixedFields = mockPcapJob.getFixedFields(); @@ -124,6 +146,79 @@ public void testFixed() throws Exception { Assert.assertEquals("true", actualFixedFields.get(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName())); Assert.assertEquals("TCP", actualFixedFields.get(Constants.Fields.PROTOCOL.getName())); Assert.assertEquals("filter", actualFixedFields.get(PcapHelper.PacketFields.PACKET_FILTER.getName())); + } + + @Test + public void testFixedRequestDefaults() throws Exception { + MockPcapJob mockPcapJob = (MockPcapJob) wac.getBean("mockPcapJob"); + mockPcapJob.setStatus(new JobStatus().withState(JobStatus.State.RUNNING)); + long beforeJobTime = System.currentTimeMillis(); + + this.mockMvc.perform(post(pcapUrl + "/fixed").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(fixedWithDefaultsJson)) + .andExpect(status().isOk()) + .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) + .andExpect(jsonPath("$.jobStatus").value("RUNNING")); + + Assert.assertEquals("/apps/metron/pcap/input", mockPcapJob.getBasePath()); + Assert.assertEquals("/apps/metron/pcap/interim", mockPcapJob.getBaseInterrimResultPath()); + Assert.assertEquals("/apps/metron/pcap/output", mockPcapJob.getFinalOutputPath()); + Assert.assertEquals(0, mockPcapJob.getStartTimeNs()); + Assert.assertTrue(beforeJobTime < mockPcapJob.getEndTimeNs() / 1000000); + Assert.assertTrue(System.currentTimeMillis() > mockPcapJob.getEndTimeNs() / 1000000); + Assert.assertEquals(1, mockPcapJob.getNumReducers()); + Assert.assertTrue(mockPcapJob.getFilterImpl() instanceof FixedPcapFilter.Configurator); + Map actualFixedFields = mockPcapJob.getFixedFields(); + Assert.assertEquals("192.168.1.2", actualFixedFields.get(Constants.Fields.SRC_ADDR.getName())); + Assert.assertEquals("2000", actualFixedFields.get(Constants.Fields.SRC_PORT.getName())); + Assert.assertEquals("192.168.1.1", actualFixedFields.get(Constants.Fields.DST_ADDR.getName())); + Assert.assertEquals("1000", actualFixedFields.get(Constants.Fields.DST_PORT.getName())); + Assert.assertEquals("true", actualFixedFields.get(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName())); + Assert.assertEquals("TCP", actualFixedFields.get(Constants.Fields.PROTOCOL.getName())); + Assert.assertEquals("filter", actualFixedFields.get(PcapHelper.PacketFields.PACKET_FILTER.getName())); + } + + @Test + public void testGetStatus() throws Exception { + MockPcapJob mockPcapJob = (MockPcapJob) wac.getBean("mockPcapJob"); + + this.mockMvc.perform(get(pcapUrl + "/jobId").with(httpBasic(user, password))) + .andExpect(status().isNotFound()); + + mockPcapJob.setStatus(new JobStatus().withJobId("jobId").withState(JobStatus.State.RUNNING)); + + this.mockMvc.perform(post(pcapUrl + "/fixed").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(fixedJson)) + .andExpect(status().isOk()) + .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) + .andExpect(jsonPath("$.jobId").value("jobId")) + .andExpect(jsonPath("$.jobStatus").value("RUNNING")); + + mockPcapJob.setStatus(new JobStatus().withJobId("jobId").withState(JobStatus.State.SUCCEEDED)); + + this.mockMvc.perform(get(pcapUrl + "/jobId").with(httpBasic(user, password))) + .andExpect(status().isOk()) + .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) + .andExpect(jsonPath("$.jobStatus").value("SUCCEEDED")); + + mockPcapJob.setStatus(new JobStatus().withJobId("jobId").withState(JobStatus.State.FINALIZING)); + + this.mockMvc.perform(get(pcapUrl + "/jobId").with(httpBasic(user, password))) + .andExpect(status().isOk()) + .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) + .andExpect(jsonPath("$.jobStatus").value("FINALIZING")); + + mockPcapJob.setStatus(new JobStatus().withJobId("jobId").withState(JobStatus.State.FAILED)); + + this.mockMvc.perform(get(pcapUrl + "/jobId").with(httpBasic(user, password))) + .andExpect(status().isOk()) + .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) + .andExpect(jsonPath("$.jobStatus").value("FAILED")); + + mockPcapJob.setStatus(new JobStatus().withJobId("jobId").withState(JobStatus.State.KILLED)); + + this.mockMvc.perform(get(pcapUrl + "/jobId").with(httpBasic(user, password))) + .andExpect(status().isOk()) + .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) + .andExpect(jsonPath("$.jobStatus").value("KILLED")); } } diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java index a7eca319b4..2df5fa07cc 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java @@ -17,47 +17,75 @@ */ package org.apache.metron.rest.mock; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.metron.common.hadoop.SequenceFileIterable; +import org.apache.metron.job.Finalizer; +import org.apache.metron.job.JobException; +import org.apache.metron.job.JobStatus; +import org.apache.metron.job.Pageable; +import org.apache.metron.job.Statusable; +import org.apache.metron.pcap.config.PcapOptions; import org.apache.metron.pcap.filter.PcapFilterConfigurator; import org.apache.metron.pcap.mr.PcapJob; -public class MockPcapJob extends PcapJob { +import java.util.List; +import java.util.Map; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class MockPcapJob extends PcapJob { private String basePath; - private String baseOutputPath; - private long beginNS; - private long endNS; + private String baseInterrimResultPath; + private String finalOutputPath; + private long startTimeNs; + private long endTimeNs; private int numReducers; private Map fixedFields; private PcapFilterConfigurator filterImpl; + private int recPerFile; private SequenceFileIterable sequenceFileIterable; + private Statusable statusable; public MockPcapJob() { sequenceFileIterable = mock(SequenceFileIterable.class); + statusable = mock(Statusable.class); } - @SuppressWarnings(value = "unchecked") @Override - public SequenceFileIterable query(Path basePath, Path baseOutputPath, long beginNS, long endNS, int numReducers, T fields, Configuration conf, FileSystem fs, PcapFilterConfigurator filterImpl) throws IOException, ClassNotFoundException, InterruptedException { - this.basePath = basePath.toString(); - this.baseOutputPath = baseOutputPath.toString(); - this.beginNS = beginNS; - this.endNS = endNS; - this.numReducers = numReducers; + public Statusable submit(Finalizer finalizer, Map configuration) throws JobException { + this.basePath = PcapOptions.BASE_PATH.get(configuration, String.class); + this.baseInterrimResultPath = PcapOptions.BASE_INTERRIM_RESULT_PATH.get(configuration, String.class); + this.finalOutputPath = PcapOptions.FINAL_OUTPUT_PATH.get(configuration, String.class); + this.startTimeNs = PcapOptions.START_TIME_MS.get(configuration, Long.class) * 1000000; + this.endTimeNs = PcapOptions.END_TIME_MS.get(configuration, Long.class) * 1000000; + this.numReducers = PcapOptions.NUM_REDUCERS.get(configuration, Integer.class); + Object fields = PcapOptions.FIELDS.get(configuration, Object.class); if (fields instanceof Map) { this.fixedFields = (Map) fields; } - this.filterImpl = filterImpl; - return sequenceFileIterable; + this.filterImpl = PcapOptions.FILTER_IMPL.get(configuration, PcapFilterConfigurator.class); + this.recPerFile = PcapOptions.NUM_RECORDS_PER_FILE.get(configuration, Integer.class); + return statusable; + } + + @Override + public JobStatus getStatus() throws JobException { + return statusable.getStatus(); + } + + @Override + public Pageable get() throws JobException, InterruptedException { + return statusable.get(); + } + + public void setStatus(JobStatus jobStatus) throws JobException { + when(statusable.getStatus()).thenReturn(jobStatus); + } + + public void setPageable(Pageable pageable) throws JobException, InterruptedException { + when(statusable.get()).thenReturn(pageable); } public void setResults(List pcaps) { @@ -68,16 +96,32 @@ public String getBasePath() { return basePath; } - public String getBaseOutputPath() { - return baseOutputPath; + public void setBasePath(String basePath) { + this.basePath = basePath; + } + + public String getBaseInterrimResultPath() { + return baseInterrimResultPath; } - public long getStartTime() { - return beginNS / 1000000; + public void setBaseInterrimResultPath(String baseInterrimResultPath) { + this.baseInterrimResultPath = baseInterrimResultPath; } - public long getEndTime() { - return endNS / 1000000; + public String getFinalOutputPath() { + return finalOutputPath; + } + + public void setFinalOutputPath(String finalOutputPath) { + this.finalOutputPath = finalOutputPath; + } + + public long getStartTimeNs() { + return startTimeNs; + } + + public long getEndTimeNs() { + return endTimeNs; } public int getNumReducers() { @@ -91,4 +135,8 @@ public Map getFixedFields() { public PcapFilterConfigurator getFilterImpl() { return filterImpl; } + + public int getRecPerFile() { + return recPerFile; + } } diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJobSupplier.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJobSupplier.java new file mode 100644 index 0000000000..9a1ac7fb60 --- /dev/null +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJobSupplier.java @@ -0,0 +1,36 @@ +/* + * 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.metron.rest.mock; + +import org.apache.metron.pcap.mr.PcapJob; +import org.apache.metron.rest.config.PcapJobSupplier; + +public class MockPcapJobSupplier extends PcapJobSupplier { + + private MockPcapJob mockPcapJob = new MockPcapJob(); + + @Override + protected PcapJob createPcapJob() { + return mockPcapJob; + } + + public void setMockPcapJob(MockPcapJob mockPcapJob) { + this.mockPcapJob = mockPcapJob; + } +} diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java index 1a11c797be..cc3f9eedbd 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java @@ -17,12 +17,22 @@ */ package org.apache.metron.rest.service.impl; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - import org.apache.hadoop.conf.Configuration; -import org.apache.metron.pcap.mr.PcapJob; +import org.apache.hadoop.fs.FileSystem; +import org.apache.metron.common.Constants; +import org.apache.metron.job.JobException; +import org.apache.metron.job.JobStatus; +import org.apache.metron.job.manager.InMemoryJobManager; +import org.apache.metron.job.manager.JobManager; +import org.apache.metron.pcap.PcapHelper; +import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.rest.MetronRestConstants; +import org.apache.metron.rest.RestException; +import org.apache.metron.rest.config.PcapJobSupplier; +import org.apache.metron.rest.mock.MockPcapJob; +import org.apache.metron.rest.mock.MockPcapJobSupplier; +import org.apache.metron.rest.model.pcap.FixedPcapRequest; +import org.apache.metron.rest.model.pcap.PcapStatus; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -30,6 +40,14 @@ import org.junit.rules.ExpectedException; import org.springframework.core.env.Environment; +import java.util.HashMap; +import java.util.Map; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; +import static org.powermock.api.mockito.PowerMockito.doReturn; + @SuppressWarnings("ALL") public class PcapServiceImplTest { @Rule @@ -37,32 +55,28 @@ public class PcapServiceImplTest { Environment environment; Configuration configuration; - PcapJob pcapJob; + MockPcapJobSupplier mockPcapJobSupplier; @Before public void setUp() throws Exception { environment = mock(Environment.class); - pcapJob = mock(PcapJob.class); configuration = mock(Configuration.class); + mockPcapJobSupplier = new MockPcapJobSupplier(); - when(environment.getProperty(MetronRestConstants.PCAP_INPUT_PATH_SPRING_PROPERTY)).thenReturn("/input/path"); - when(environment.getProperty(MetronRestConstants.PCAP_OUTPUT_PATH_SPRING_PROPERTY)).thenReturn("/output/path"); + when(environment.getProperty(MetronRestConstants.PCAP_BASE_PATH_SPRING_PROPERTY)).thenReturn("/base/path"); + when(environment.getProperty(MetronRestConstants.PCAP_BASE_INTERIM_RESULT_PATH_SPRING_PROPERTY)).thenReturn("/base/interim/result/path"); + when(environment.getProperty(MetronRestConstants.PCAP_FINAL_OUTPUT_PATH_SPRING_PROPERTY)).thenReturn("/final/output/path"); + when(environment.getProperty(MetronRestConstants.PCAP_PAGE_SIZE_SPRING_PROPERTY)).thenReturn("100"); } - // TODO - - @Test - public void placeholder() { - Assert.assertTrue(true); - } -/* @Test public void fixedShouldProperlyCallPcapJobQuery() throws Exception { FixedPcapRequest fixedPcapRequest = new FixedPcapRequest(); - fixedPcapRequest.setBaseOutputPath("baseOutputPath"); fixedPcapRequest.setBasePath("basePath"); - fixedPcapRequest.setStartTime(1L); - fixedPcapRequest.setEndTime(2L); + fixedPcapRequest.setBaseInterimResultPath("baseOutputPath"); + fixedPcapRequest.setFinalOutputPath("finalOutputPath"); + fixedPcapRequest.setStartTimeMs(1L); + fixedPcapRequest.setEndTimeMs(2L); fixedPcapRequest.setNumReducers(2); fixedPcapRequest.setIpSrcAddr("ip_src_addr"); fixedPcapRequest.setIpDstAddr("ip_dst_addr"); @@ -71,10 +85,19 @@ public void fixedShouldProperlyCallPcapJobQuery() throws Exception { fixedPcapRequest.setProtocol("tcp"); fixedPcapRequest.setPacketFilter("filter"); fixedPcapRequest.setIncludeReverse(true); + MockPcapJob mockPcapJob = new MockPcapJob(); + mockPcapJobSupplier.setMockPcapJob(mockPcapJob); + JobManager jobManager = new InMemoryJobManager<>(); - PcapServiceImpl pcapService = spy(new PcapServiceImpl(environment, configuration, pcapJob)); + PcapServiceImpl pcapService = spy(new PcapServiceImpl(environment, configuration, mockPcapJobSupplier, jobManager)); FileSystem fileSystem = mock(FileSystem.class); doReturn(fileSystem).when(pcapService).getFileSystem(); + mockPcapJob.setStatus(new JobStatus() + .withJobId("jobId") + .withDescription("description") + .withPercentComplete(0L) + .withState(JobStatus.State.RUNNING)); + Map expectedFields = new HashMap() {{ put(Constants.Fields.SRC_ADDR.getName(), "ip_src_addr"); put(Constants.Fields.DST_ADDR.getName(), "ip_dst_addr"); @@ -84,72 +107,123 @@ public void fixedShouldProperlyCallPcapJobQuery() throws Exception { put(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName(), "true"); put(PcapHelper.PacketFields.PACKET_FILTER.getName(), "filter"); }}; - List expectedPcaps = Arrays.asList("pcap1".getBytes(), "pcap2".getBytes()); - SequenceFileIterable results = mock(SequenceFileIterable.class); - when(results.iterator()).thenReturn(expectedPcaps.iterator()); - when(pcapJob.query(eq(new Path("basePath")), - eq(new Path("baseOutputPath")), - eq(1000000L), - eq(2000000L), - eq(2), - eq(expectedFields), - eq(configuration), - any(FileSystem.class), - any(FixedPcapFilter.Configurator.class))).thenReturn(results); - - PcapResponse pcapsResponse = pcapService.fixed(fixedPcapRequest); - Assert.assertEquals(expectedPcaps, pcapsResponse.getPcaps()); + PcapStatus expectedPcapStatus = new PcapStatus(); + expectedPcapStatus.setJobId("jobId"); + expectedPcapStatus.setJobStatus(JobStatus.State.RUNNING.name()); + expectedPcapStatus.setDescription("description"); + + Assert.assertEquals(expectedPcapStatus, pcapService.fixed("user", fixedPcapRequest)); + Assert.assertEquals(expectedPcapStatus, pcapService.jobStatusToPcapStatus(jobManager.getJob("user", "jobId").getStatus())); + Assert.assertEquals("basePath", mockPcapJob.getBasePath()); + Assert.assertEquals("baseOutputPath", mockPcapJob.getBaseInterrimResultPath()); + Assert.assertEquals("finalOutputPath", mockPcapJob.getFinalOutputPath()); + Assert.assertEquals(1000000, mockPcapJob.getStartTimeNs()); + Assert.assertEquals(2000000, mockPcapJob.getEndTimeNs()); + Assert.assertEquals(2, mockPcapJob.getNumReducers()); + Assert.assertEquals(100, mockPcapJob.getRecPerFile()); + Assert.assertTrue(mockPcapJob.getFilterImpl() instanceof FixedPcapFilter.Configurator); + Map actualFixedFields = mockPcapJob.getFixedFields(); + Assert.assertEquals("ip_src_addr", actualFixedFields.get(Constants.Fields.SRC_ADDR.getName())); + Assert.assertEquals("1000", actualFixedFields.get(Constants.Fields.SRC_PORT.getName())); + Assert.assertEquals("ip_dst_addr", actualFixedFields.get(Constants.Fields.DST_ADDR.getName())); + Assert.assertEquals("2000", actualFixedFields.get(Constants.Fields.DST_PORT.getName())); + Assert.assertEquals("true", actualFixedFields.get(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName())); + Assert.assertEquals("tcp", actualFixedFields.get(Constants.Fields.PROTOCOL.getName())); + Assert.assertEquals("filter", actualFixedFields.get(PcapHelper.PacketFields.PACKET_FILTER.getName())); } @Test public void fixedShouldProperlyCallPcapJobQueryWithDefaults() throws Exception { + long beforeJobTime = System.currentTimeMillis(); + FixedPcapRequest fixedPcapRequest = new FixedPcapRequest(); + MockPcapJob mockPcapJob = new MockPcapJob(); + mockPcapJobSupplier.setMockPcapJob(mockPcapJob); + JobManager jobManager = new InMemoryJobManager<>(); - PcapServiceImpl pcapService = spy(new PcapServiceImpl(environment, configuration, pcapJob)); + PcapServiceImpl pcapService = spy(new PcapServiceImpl(environment, configuration, mockPcapJobSupplier, jobManager)); FileSystem fileSystem = mock(FileSystem.class); doReturn(fileSystem).when(pcapService).getFileSystem(); - Map expectedFields = new HashMap() {{ - put(Constants.Fields.INCLUDES_REVERSE_TRAFFIC.getName(), "false"); - }}; - List expectedPcaps = Arrays.asList("pcap1".getBytes(), "pcap2".getBytes()); - SequenceFileIterable results = mock(SequenceFileIterable.class); - when(results.iterator()).thenReturn(expectedPcaps.iterator()); - when(pcapJob.query(eq(new Path("/input/path")), - eq(new Path("/output/path")), - eq(0L), - eq(fixedPcapRequest.getEndTime() * 1000000), - eq(1), - eq(expectedFields), - eq(configuration), - any(FileSystem.class), - any(FixedPcapFilter.Configurator.class))).thenReturn(results); - - PcapResponse pcapsResponse = pcapService.fixed(fixedPcapRequest); - Assert.assertEquals(expectedPcaps, pcapsResponse.getPcaps()); + mockPcapJob.setStatus(new JobStatus() + .withJobId("jobId") + .withDescription("description") + .withPercentComplete(0L) + .withState(JobStatus.State.RUNNING)); + + PcapStatus expectedPcapStatus = new PcapStatus(); + expectedPcapStatus.setJobId("jobId"); + expectedPcapStatus.setJobStatus(JobStatus.State.RUNNING.name()); + expectedPcapStatus.setDescription("description"); + + Assert.assertEquals(expectedPcapStatus, pcapService.fixed("user", fixedPcapRequest)); + Assert.assertEquals("/base/path", mockPcapJob.getBasePath()); + Assert.assertEquals("/base/interim/result/path", mockPcapJob.getBaseInterrimResultPath()); + Assert.assertEquals("/final/output/path", mockPcapJob.getFinalOutputPath()); + Assert.assertEquals(0, mockPcapJob.getStartTimeNs()); + Assert.assertTrue(beforeJobTime <= mockPcapJob.getEndTimeNs() / 1000000); + Assert.assertTrue(System.currentTimeMillis() >= mockPcapJob.getEndTimeNs() / 1000000); + Assert.assertEquals(1, mockPcapJob.getNumReducers()); + Assert.assertEquals(100, mockPcapJob.getRecPerFile()); + Assert.assertTrue(mockPcapJob.getFilterImpl() instanceof FixedPcapFilter.Configurator); + Assert.assertEquals(new HashMap<>(), mockPcapJob.getFixedFields()); } @Test public void fixedShouldThrowRestException() throws Exception { exception.expect(RestException.class); - exception.expectMessage("some exception"); + exception.expectMessage("some job exception"); FixedPcapRequest fixedPcapRequest = new FixedPcapRequest(); - - PcapServiceImpl pcapService = spy(new PcapServiceImpl(environment, configuration, pcapJob)); + JobManager jobManager = mock(JobManager.class); + PcapJobSupplier pcapJobSupplier = new PcapJobSupplier(); + PcapServiceImpl pcapService = spy(new PcapServiceImpl(environment, configuration, pcapJobSupplier, jobManager)); FileSystem fileSystem = mock(FileSystem.class); doReturn(fileSystem).when(pcapService).getFileSystem(); + when(jobManager.submit(pcapJobSupplier, "user")).thenThrow(new JobException("some job exception")); + + pcapService.fixed("user", fixedPcapRequest); + } + + @Test + public void getStatusShouldProperlyReturnStatus() throws Exception { + MockPcapJob mockPcapJob = mock(MockPcapJob.class); + JobManager jobManager = mock(JobManager.class); + JobStatus actualJobStatus = new JobStatus() + .withJobId("jobId") + .withState(JobStatus.State.SUCCEEDED) + .withDescription("description") + .withPercentComplete(100.0); + when(mockPcapJob.getStatus()).thenReturn(actualJobStatus); + when(jobManager.getJob("user", "jobId")).thenReturn(mockPcapJob); + + PcapServiceImpl pcapService = new PcapServiceImpl(environment, configuration, mockPcapJobSupplier, jobManager); + PcapStatus expectedPcapStatus = new PcapStatus(); + expectedPcapStatus.setJobId("jobId"); + expectedPcapStatus.setJobStatus(JobStatus.State.SUCCEEDED.name()); + expectedPcapStatus.setDescription("description"); + expectedPcapStatus.setPercentComplete(100.0); + + Assert.assertEquals(expectedPcapStatus, pcapService.getJobStatus("user", "jobId")); + } + + @Test + public void getStatusShouldReturnNullOnMissingStatus() throws Exception { + JobManager jobManager = new InMemoryJobManager(); + PcapServiceImpl pcapService = new PcapServiceImpl(environment, configuration, new PcapJobSupplier(), jobManager); - when(pcapJob.query(any(), - any(), - eq(0L), - eq(fixedPcapRequest.getEndTime() * 1000000), - eq(1), - any(), - any(), - any(FileSystem.class), - any(FixedPcapFilter.Configurator.class))).thenThrow(new IOException("some exception")); - - pcapService.fixed(fixedPcapRequest); + Assert.assertNull(pcapService.getJobStatus("user", "jobId")); } - */ + + @Test + public void getStatusShouldThrowRestException() throws Exception { + exception.expect(RestException.class); + exception.expectMessage("some job exception"); + + JobManager jobManager = mock(JobManager.class); + when(jobManager.getJob("user", "jobId")).thenThrow(new JobException("some job exception")); + + PcapServiceImpl pcapService = new PcapServiceImpl(environment, configuration, new PcapJobSupplier(), jobManager); + pcapService.getJobStatus("user", "jobId"); + } + } diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigOption.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigOption.java index 473664c6e6..8e4211bf1d 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigOption.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigOption.java @@ -18,6 +18,8 @@ package org.apache.metron.common.configuration; +import org.apache.metron.stellar.common.utils.ConversionUtils; + import java.util.Map; import java.util.function.BiFunction; @@ -32,11 +34,17 @@ default void put(Map map, Object value) { } default T get(Map map, Class clazz) { - return clazz.cast(map.get(getKey())); + Object obj = map.get(getKey()); + if(clazz.isInstance(obj)) { + return clazz.cast(obj); + } + else { + return ConversionUtils.convert(obj, clazz); + } } default T get(Map map, BiFunction transform, Class clazz) { - return clazz.cast(map.get(getKey())); + return clazz.cast(transform.apply(getKey(), map.get(getKey()))); } default T getTransformed(Map map, Class clazz) { diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobNotFoundException.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobNotFoundException.java new file mode 100644 index 0000000000..6a677bfc77 --- /dev/null +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/JobNotFoundException.java @@ -0,0 +1,30 @@ +/** + * 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.metron.job; + +public class JobNotFoundException extends JobException { + + public JobNotFoundException(String message) { + super(message); + } + + public JobNotFoundException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java index bf0baa7c1d..1340aa5d1c 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/manager/InMemoryJobManager.java @@ -26,6 +26,7 @@ import java.util.Map; import java.util.function.Supplier; import org.apache.metron.job.JobException; +import org.apache.metron.job.JobNotFoundException; import org.apache.metron.job.JobStatus; import org.apache.metron.job.Statusable; import org.slf4j.Logger; @@ -52,7 +53,7 @@ public JobStatus submit(Supplier> jobSupplier, String usernam @Override public JobStatus getStatus(String username, String jobId) throws JobException { - return jobs.get(username).get(jobId).getStatus(); + return getJob(username, jobId).getStatus(); } @Override @@ -67,7 +68,11 @@ public void killJob(String username, String jobId) throws JobException { @Override public Statusable getJob(String username, String jobId) throws JobException { - return getUserJobs(username).get(jobId); + Map> jobStatusables = getUserJobs(username); + if (jobStatusables.size() > 0 && jobStatusables.containsKey(jobId)) { + return jobStatusables.get(jobId); + } + throw new JobNotFoundException("Could not find job " + jobId + " for user " + username); } private Map> getUserJobs(String username) { @@ -76,7 +81,7 @@ private Map> getUserJobs(String username) { @Override public List> getJobs(String username) throws JobException { - return new ArrayList>(getUserJobs(username).values()); + return new ArrayList<>(getUserJobs(username).values()); } } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index c685a147fb..209ec9f920 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -215,8 +215,18 @@ public Statusable submit(Finalizer finalizer, Map co FileSystem fileSystem = PcapOptions.FILESYSTEM.get(configuration, FileSystem.class); Path basePath = PcapOptions.BASE_PATH.getTransformed(configuration, Path.class); Path baseInterimResultPath = PcapOptions.BASE_INTERRIM_RESULT_PATH.getTransformed(configuration, Path.class); - long startTime = PcapOptions.START_TIME_NS.get(configuration, Long.class); - long endTime = PcapOptions.END_TIME_NS.get(configuration, Long.class); + long startTime; + if (configuration.containsKey(PcapOptions.START_TIME_NS.getKey())) { + startTime = PcapOptions.START_TIME_NS.get(configuration, Long.class); + } else { + startTime = PcapOptions.START_TIME_MS.get(configuration, Long.class) * 1000000; + } + long endTime; + if (configuration.containsKey(PcapOptions.END_TIME_NS.getKey())) { + endTime = PcapOptions.END_TIME_NS.get(configuration, Long.class); + } else { + endTime = PcapOptions.END_TIME_MS.get(configuration, Long.class) * 1000000; + } int numReducers = PcapOptions.NUM_REDUCERS.get(configuration, Integer.class); T fields = (T) PcapOptions.FIELDS.get(configuration, Object.class); PcapFilterConfigurator filterImpl = PcapOptions.FILTER_IMPL.get(configuration, PcapFilterConfigurator.class); @@ -281,12 +291,14 @@ public Statusable query(Optional jobName, } private void startJobStatusTimerThread(long interval) { + System.out.println("startJobStatusTimerThread"); timer = new Timer(); timer.scheduleAtFixedRate(new TimerTask() { @Override public void run() { try { synchronized (jobState) { + System.out.println("Checking job state for finalizer "+ jobState); if (jobState == State.RUNNING) { if (mrJob.isComplete()) { switch (mrJob.getStatus().getState()) { @@ -327,6 +339,7 @@ public void run() { private boolean setFinalResults(Finalizer finalizer, Map configuration) { boolean success = true; Pageable results = new PcapPages(); + System.out.println("Setting final results"); try { results = finalizer.finalizeJob(configuration); } catch (JobException e) { @@ -369,8 +382,15 @@ public Job createJob(Optional jobName job.setPartitionerClass(PcapPartitioner.class); job.setOutputKeyClass(LongWritable.class); job.setOutputValueClass(BytesWritable.class); + System.out.println("beginNs = " + beginNS); + System.out.println("endNS = " + endNS); + System.out.println("basePath = " + basePath); + for(Path path: listFiles(fs, basePath)) { + System.out.println(path.toUri().toString()); + } Iterable filteredPaths = FileFilterUtil.getPathsInTimeRange(beginNS, endNS, listFiles(fs, basePath)); String inputPaths = Joiner.on(',').join(filteredPaths); + System.out.println(inputPaths); if (StringUtils.isEmpty(inputPaths)) { return null; } From 61ed1ade8d0a141849ad2d139aee1c740fa5d627 Mon Sep 17 00:00:00 2001 From: merrimanr Date: Tue, 17 Jul 2018 08:29:52 -0500 Subject: [PATCH 25/36] fixed failing tests --- .../apache/metron/rest/model/pcap/PcapRequest.java | 4 ++-- .../org/apache/metron/rest/mock/MockPcapJob.java | 2 +- .../java/org/apache/metron/pcap/PcapJobTest.java | 2 +- .../integration/PcapTopologyIntegrationTest.java | 11 +++++------ .../org/apache/metron/pcap/config/PcapOptions.java | 2 +- .../main/java/org/apache/metron/pcap/mr/PcapJob.java | 12 +----------- 6 files changed, 11 insertions(+), 22 deletions(-) diff --git a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java index 6d08dd28b6..7b7d61b58d 100644 --- a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java +++ b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java @@ -40,11 +40,11 @@ public void setBasePath(String basePath) { } public String getBaseInterimResultPath() { - return PcapOptions.BASE_INTERRIM_RESULT_PATH.get(this, String.class); + return PcapOptions.BASE_INTERIM_RESULT_PATH.get(this, String.class); } public void setBaseInterimResultPath(String baseInterimResultPath) { - PcapOptions.BASE_INTERRIM_RESULT_PATH.put(this, baseInterimResultPath); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(this, baseInterimResultPath); } public String getFinalOutputPath() { diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java index 2df5fa07cc..0efb1667cf 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java @@ -56,7 +56,7 @@ public MockPcapJob() { @Override public Statusable submit(Finalizer finalizer, Map configuration) throws JobException { this.basePath = PcapOptions.BASE_PATH.get(configuration, String.class); - this.baseInterrimResultPath = PcapOptions.BASE_INTERRIM_RESULT_PATH.get(configuration, String.class); + this.baseInterrimResultPath = PcapOptions.BASE_INTERIM_RESULT_PATH.get(configuration, String.class); this.finalOutputPath = PcapOptions.FINAL_OUTPUT_PATH.get(configuration, String.class); this.startTimeNs = PcapOptions.START_TIME_MS.get(configuration, Long.class) * 1000000; this.endTimeNs = PcapOptions.END_TIME_MS.get(configuration, Long.class) * 1000000; diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java index ccce608fbc..967cbc502b 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java @@ -100,7 +100,7 @@ public void setup() throws IOException { PcapOptions.HADOOP_CONF.put(config, hadoopConfig); PcapOptions.FILESYSTEM.put(config, FileSystem.get(hadoopConfig)); PcapOptions.BASE_PATH.put(config, basePath); - PcapOptions.INTERIM_RESULT_PATH.put(config, baseOutPath); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(config, baseOutPath); PcapOptions.START_TIME_NS.put(config, startTime); PcapOptions.END_TIME_NS.put(config, endTime); PcapOptions.NUM_REDUCERS.put(config, numReducers); diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java index c403eacd27..1c85d3a444 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java @@ -34,7 +34,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Properties; import javax.annotation.Nullable; import kafka.consumer.ConsumerIterator; @@ -59,7 +58,6 @@ import org.apache.metron.integration.components.MRComponent; import org.apache.metron.integration.components.ZKServerComponent; import org.apache.metron.integration.utils.KafkaUtil; -import org.apache.metron.job.Finalizer; import org.apache.metron.job.JobStatus; import org.apache.metron.job.Statusable; import org.apache.metron.pcap.PacketInfo; @@ -68,7 +66,6 @@ import org.apache.metron.pcap.config.PcapOptions; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.filter.query.QueryPcapFilter; -import org.apache.metron.pcap.finalizer.PcapFinalizer; import org.apache.metron.pcap.finalizer.PcapFinalizerStrategies; import org.apache.metron.pcap.mr.PcapJob; import org.apache.metron.spout.pcap.Endianness; @@ -260,7 +257,7 @@ public ProcessorResult getResult() { PcapOptions.HADOOP_CONF.put(configuration, hadoopConf); PcapOptions.FILESYSTEM.put(configuration, FileSystem.get(hadoopConf)); PcapOptions.BASE_PATH.put(configuration, new Path(inputDir.getAbsolutePath())); - PcapOptions.BASE_INTERRIM_RESULT_PATH.put(configuration, new Path(interimResultDir.getAbsolutePath())); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(configuration, new Path(interimResultDir.getAbsolutePath())); PcapOptions.START_TIME_NS.put(configuration, getTimestamp(4, pcapEntries)); PcapOptions.END_TIME_NS.put(configuration, getTimestamp(5, pcapEntries)); PcapOptions.NUM_REDUCERS.put(configuration, 10); @@ -618,8 +615,10 @@ public boolean apply(@Nullable JSONObject input) { private void waitForJob(Statusable statusable) throws Exception { for (int t = 0; t < MAX_RETRIES; ++t, Thread.sleep(SLEEP_MS)) { - if (statusable.isDone()) { - return; + if (!statusable.getStatus().getState().equals(JobStatus.State.RUNNING)) { + if (statusable.isDone()) { + return; + } } } throw new Exception("Job did not complete within " + (MAX_RETRIES * SLEEP_MS) + " seconds"); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java index 68bb17dd2b..bf632f6cea 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java @@ -26,7 +26,7 @@ public enum PcapOptions implements ConfigOption { JOB_NAME("jobName"), FINAL_FILENAME_PREFIX("finalFilenamePrefix"), BASE_PATH("basePath", (s, o) -> o == null ? null : new Path(o.toString())), - BASE_INTERRIM_RESULT_PATH("baseInterimResultPath", (s,o) -> o == null?null:new Path(o.toString())), + BASE_INTERIM_RESULT_PATH("baseInterimResultPath", (s, o) -> o == null?null:new Path(o.toString())), INTERIM_RESULT_PATH("interimResultPath", (s, o) -> o == null ? null : new Path(o.toString())), FINAL_OUTPUT_PATH("finalOutputPath", (s, o) -> o == null ? null : new Path(o.toString())), NUM_REDUCERS("numReducers"), diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index 209ec9f920..73ebdbe4ba 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -214,7 +214,7 @@ public Statusable submit(Finalizer finalizer, Map co Configuration hadoopConf = PcapOptions.HADOOP_CONF.get(configuration, Configuration.class); FileSystem fileSystem = PcapOptions.FILESYSTEM.get(configuration, FileSystem.class); Path basePath = PcapOptions.BASE_PATH.getTransformed(configuration, Path.class); - Path baseInterimResultPath = PcapOptions.BASE_INTERRIM_RESULT_PATH.getTransformed(configuration, Path.class); + Path baseInterimResultPath = PcapOptions.BASE_INTERIM_RESULT_PATH.getTransformed(configuration, Path.class); long startTime; if (configuration.containsKey(PcapOptions.START_TIME_NS.getKey())) { startTime = PcapOptions.START_TIME_NS.get(configuration, Long.class); @@ -291,14 +291,12 @@ public Statusable query(Optional jobName, } private void startJobStatusTimerThread(long interval) { - System.out.println("startJobStatusTimerThread"); timer = new Timer(); timer.scheduleAtFixedRate(new TimerTask() { @Override public void run() { try { synchronized (jobState) { - System.out.println("Checking job state for finalizer "+ jobState); if (jobState == State.RUNNING) { if (mrJob.isComplete()) { switch (mrJob.getStatus().getState()) { @@ -339,7 +337,6 @@ public void run() { private boolean setFinalResults(Finalizer finalizer, Map configuration) { boolean success = true; Pageable results = new PcapPages(); - System.out.println("Setting final results"); try { results = finalizer.finalizeJob(configuration); } catch (JobException e) { @@ -382,15 +379,8 @@ public Job createJob(Optional jobName job.setPartitionerClass(PcapPartitioner.class); job.setOutputKeyClass(LongWritable.class); job.setOutputValueClass(BytesWritable.class); - System.out.println("beginNs = " + beginNS); - System.out.println("endNS = " + endNS); - System.out.println("basePath = " + basePath); - for(Path path: listFiles(fs, basePath)) { - System.out.println(path.toUri().toString()); - } Iterable filteredPaths = FileFilterUtil.getPathsInTimeRange(beginNS, endNS, listFiles(fs, basePath)); String inputPaths = Joiner.on(',').join(filteredPaths); - System.out.println(inputPaths); if (StringUtils.isEmpty(inputPaths)) { return null; } From a68930b3cb0da32f375d1ec4bff2305edddb3b3b Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Tue, 17 Jul 2018 08:31:23 -0600 Subject: [PATCH 26/36] Address code review comments --- .../org/apache/metron/job/Statusable.java | 3 +- .../job/manager/InMemoryJobManagerTest.java | 36 +++++++++---------- 2 files changed, 19 insertions(+), 20 deletions(-) diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java index 4b28e5d26f..9bdea35d5e 100644 --- a/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/Statusable.java @@ -26,8 +26,7 @@ public interface Statusable { enum JobType { - MAP_REDUCE, - SPARK; + MAP_REDUCE; } /** diff --git a/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java b/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java index 9aaf96a042..f3a3978034 100644 --- a/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java +++ b/metron-platform/metron-job/src/test/java/org/apache/metron/job/manager/InMemoryJobManagerTest.java @@ -1,3 +1,21 @@ +/** + * 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.metron.job.manager; import static org.hamcrest.CoreMatchers.equalTo; @@ -26,24 +44,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -/** - * 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. - */ - public class InMemoryJobManagerTest { @Rule From 3fa118fac478bcc3ebcddcfbf059e54586219819 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Tue, 17 Jul 2018 11:59:16 -0600 Subject: [PATCH 27/36] Remove pcap service from Ansible deployment scripts. --- .../amazon-ec2/conf/defaults.yml | 1 - metron-deployment/amazon-ec2/playbook.yml | 2 +- .../ansible/playbooks/metron_install.yml | 7 -- .../metron_pcapservice/defaults/main.yml | 28 ------- .../roles/metron_pcapservice/meta/main.yml | 19 ----- .../metron_pcapservice/tasks/config-hbase.yml | 26 ------ .../roles/metron_pcapservice/tasks/main.yml | 25 ------ .../metron_pcapservice/tasks/pcapservice.yml | 30 ------- .../metron-pcapservice-logrotate.yml | 27 ------ .../metron_pcapservice/templates/pcapservice | 84 ------------------- .../ubuntu14/ansible/inventory/hosts | 4 - 11 files changed, 1 insertion(+), 252 deletions(-) delete mode 100644 metron-deployment/ansible/roles/metron_pcapservice/defaults/main.yml delete mode 100644 metron-deployment/ansible/roles/metron_pcapservice/meta/main.yml delete mode 100644 metron-deployment/ansible/roles/metron_pcapservice/tasks/config-hbase.yml delete mode 100644 metron-deployment/ansible/roles/metron_pcapservice/tasks/main.yml delete mode 100644 metron-deployment/ansible/roles/metron_pcapservice/tasks/pcapservice.yml delete mode 100644 metron-deployment/ansible/roles/metron_pcapservice/templates/metron-pcapservice-logrotate.yml delete mode 100644 metron-deployment/ansible/roles/metron_pcapservice/templates/pcapservice diff --git a/metron-deployment/amazon-ec2/conf/defaults.yml b/metron-deployment/amazon-ec2/conf/defaults.yml index e572da1028..22b3bb010f 100644 --- a/metron-deployment/amazon-ec2/conf/defaults.yml +++ b/metron-deployment/amazon-ec2/conf/defaults.yml @@ -60,7 +60,6 @@ retention_in_gb: 25 # metron variables metron_version: 0.5.1 metron_directory: /usr/metron/{{ metron_version }} -pcapservice_port: 8081 # sensors install_snort: True diff --git a/metron-deployment/amazon-ec2/playbook.yml b/metron-deployment/amazon-ec2/playbook.yml index 470a1816ba..cb1e326c6d 100644 --- a/metron-deployment/amazon-ec2/playbook.yml +++ b/metron-deployment/amazon-ec2/playbook.yml @@ -32,7 +32,7 @@ - include: tasks/create-open-outbound-security-group.yml - include: tasks/create-hosts.yml host_count=1 host_type=sensors,ambari_master,ec2,monit - include: tasks/create-hosts.yml host_count=4 host_type=ambari_slave,ec2 - - include: tasks/create-hosts.yml host_count=1 host_type=pcap_server,monit,ec2 + - include: tasks/create-hosts.yml host_count=1 host_type=monit,ec2 - include: tasks/create-hosts.yml host_count=1 host_type=ambari_slave,enrichment,metron,ec2,zeppelin - include: tasks/create-hosts.yml host_count=2 host_type=ambari_slave,search,ec2 - include: tasks/create-hosts.yml host_count=1 host_type=ambari_slave,web,ec2 diff --git a/metron-deployment/ansible/playbooks/metron_install.yml b/metron-deployment/ansible/playbooks/metron_install.yml index 49db5e62ba..56f7a7a6c7 100644 --- a/metron-deployment/ansible/playbooks/metron_install.yml +++ b/metron-deployment/ansible/playbooks/metron_install.yml @@ -66,10 +66,3 @@ - role: load_web_templates tags: - load_templates - -- hosts: pcap_server - become: true - roles: - - role: metron_pcapservice - tags: - - pcap-service diff --git a/metron-deployment/ansible/roles/metron_pcapservice/defaults/main.yml b/metron-deployment/ansible/roles/metron_pcapservice/defaults/main.yml deleted file mode 100644 index b129b82c40..0000000000 --- a/metron-deployment/ansible/roles/metron_pcapservice/defaults/main.yml +++ /dev/null @@ -1,28 +0,0 @@ -# -# 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. -# ---- -metron_version: 0.5.1 -metron_directory: /usr/metron/{{ metron_version }} -pcapservice_jar_name: metron-api-{{ metron_version }}.jar -pcapservice_jar_src: "{{ playbook_dir }}/../../../metron-platform/metron-api/target/{{ pcapservice_jar_name }}" -pcapservice_jar_dst: "{{ metron_directory }}/lib/{{ pcapservice_jar_name }}" -pcapservice_port: 8081 -hbase_config_path: "/etc/hbase/conf" -query_hdfs_path: "/tmp" -pcap_hdfs_path: "/apps/metron/pcap" -metron_pcapservice_logrotate_frequency: daily -metron_pcapservice_logrotate_retention: 30 diff --git a/metron-deployment/ansible/roles/metron_pcapservice/meta/main.yml b/metron-deployment/ansible/roles/metron_pcapservice/meta/main.yml deleted file mode 100644 index ddf6aa910d..0000000000 --- a/metron-deployment/ansible/roles/metron_pcapservice/meta/main.yml +++ /dev/null @@ -1,19 +0,0 @@ -# -# 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. -# ---- -dependencies: - - java_jdk diff --git a/metron-deployment/ansible/roles/metron_pcapservice/tasks/config-hbase.yml b/metron-deployment/ansible/roles/metron_pcapservice/tasks/config-hbase.yml deleted file mode 100644 index b77c1ecbb4..0000000000 --- a/metron-deployment/ansible/roles/metron_pcapservice/tasks/config-hbase.yml +++ /dev/null @@ -1,26 +0,0 @@ -# -# 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. -# ---- - - name: Fetch hbase-site.xml - fetch: src=/etc/hbase/conf/hbase-site.xml dest=/tmp/hbase/conf/hbase-site.xml flat=yes - delegate_to: "{{ groups.ambari_slave[0] }}" - - - name: Create hbase conf directory - file: path=/etc/hbase/conf state=directory mode=0755 - - - name: Copy hbase-site.xml - copy: src=/tmp/hbase/conf/hbase-site.xml dest=/etc/hbase/conf/hbase-site.xml mode=0644 diff --git a/metron-deployment/ansible/roles/metron_pcapservice/tasks/main.yml b/metron-deployment/ansible/roles/metron_pcapservice/tasks/main.yml deleted file mode 100644 index ed33354459..0000000000 --- a/metron-deployment/ansible/roles/metron_pcapservice/tasks/main.yml +++ /dev/null @@ -1,25 +0,0 @@ -# -# 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. -# ---- -- include: pcapservice.yml - -- name: Create Logrotate Script for metron_pcapservice - template: - src: "metron-pcapservice-logrotate.yml" - dest: "/etc/logrotate.d/metron-pcapservice" - mode: 0644 - diff --git a/metron-deployment/ansible/roles/metron_pcapservice/tasks/pcapservice.yml b/metron-deployment/ansible/roles/metron_pcapservice/tasks/pcapservice.yml deleted file mode 100644 index d0ac41148b..0000000000 --- a/metron-deployment/ansible/roles/metron_pcapservice/tasks/pcapservice.yml +++ /dev/null @@ -1,30 +0,0 @@ -# -# 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. -# ---- -- name: Create Metron streaming directories - file: path={{ metron_directory }}/{{ item.name }} state=directory mode=0755 - with_items: - - { name: 'lib'} - - { name: 'config'} - -- name: Copy Metron pcapservice jar - copy: - src: "{{ pcapservice_jar_src }}" - dest: "{{ pcapservice_jar_dst }}" - -- name: Install service script - template: src=pcapservice dest=/etc/init.d/pcapservice mode=0755 diff --git a/metron-deployment/ansible/roles/metron_pcapservice/templates/metron-pcapservice-logrotate.yml b/metron-deployment/ansible/roles/metron_pcapservice/templates/metron-pcapservice-logrotate.yml deleted file mode 100644 index 0293ade547..0000000000 --- a/metron-deployment/ansible/roles/metron_pcapservice/templates/metron-pcapservice-logrotate.yml +++ /dev/null @@ -1,27 +0,0 @@ -# -# 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. -# - -#Metron pcapService -/var/log/metron_pcapservice.log { - {{ metron_pcapservice_logrotate_frequency }} - rotate {{ metron_pcapservice_logrotate_retention }} - missingok - notifempty - copytruncate - compress -} - diff --git a/metron-deployment/ansible/roles/metron_pcapservice/templates/pcapservice b/metron-deployment/ansible/roles/metron_pcapservice/templates/pcapservice deleted file mode 100644 index 054133de51..0000000000 --- a/metron-deployment/ansible/roles/metron_pcapservice/templates/pcapservice +++ /dev/null @@ -1,84 +0,0 @@ -#!/usr/bin/env bash -# -# 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. -# -# metron pcap service -# chkconfig: 345 20 80 -# description: Metron PCAP Service Daemon -# processname: pcapservice -# -NAME=pcapservice -DESC="Metron pcap service" -PIDFILE=/var/run/$NAME.pid -SCRIPTNAME=/etc/init.d/$NAME -LOGFILE="/var/log/metron_pcapservice.log" -EXTRA_ARGS="${@:2}" -DAEMON_PATH="/" -DAEMON="/usr/bin/yarn jar" -DAEMONOPTS="{{ pcapservice_jar_dst }} org.apache.metron.pcapservice.rest.PcapService -port {{ pcapservice_port }} -query_hdfs_path {{ query_hdfs_path }} -pcap_hdfs_path {{ pcap_hdfs_path }}" - -case "$1" in - start) - printf "%-50s" "Starting $NAME..." - - # kick-off the daemon - cd $DAEMON_PATH - PID=`$DAEMON $DAEMONOPTS >> $LOGFILE 2>&1 & echo $!` - if [ -z $PID ]; then - printf "%s\n" "Fail" - else - echo $PID > $PIDFILE - printf "%s\n" "Ok" - fi - ;; - - status) - printf "%-50s" "Checking $NAME..." - if [ -f $PIDFILE ]; then - PID=`cat $PIDFILE` - if [ -z "`ps axf | grep ${PID} | grep -v grep`" ]; then - printf "%s\n" "Process dead but pidfile exists" - else - echo "Running" - fi - else - printf "%s\n" "Service not running" - fi - ;; - - stop) - printf "%-50s" "Stopping $NAME" - PID=`cat $PIDFILE` - cd $DAEMON_PATH - if [ -f $PIDFILE ]; then - kill -HUP $PID - printf "%s\n" "Ok" - rm -f $PIDFILE - else - printf "%s\n" "pidfile not found" - fi - ;; - - restart) - $0 stop - $0 start - ;; - - *) - echo "Usage: $0 {status|start|stop|restart}" - exit 1 -esac diff --git a/metron-deployment/development/ubuntu14/ansible/inventory/hosts b/metron-deployment/development/ubuntu14/ansible/inventory/hosts index 9bd9ea1e05..272c8b6706 100644 --- a/metron-deployment/development/ubuntu14/ansible/inventory/hosts +++ b/metron-deployment/development/ubuntu14/ansible/inventory/hosts @@ -30,9 +30,6 @@ node1 [sensors] node1 -[pcap_server] -node1 - [web] node1 @@ -41,7 +38,6 @@ node1 [monit:children] sensors -pcap_server [local] 127.0.0.1 From d58851986060701c6ee796504df7fa12e4df91a5 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Tue, 17 Jul 2018 12:00:03 -0600 Subject: [PATCH 28/36] Fix try with resources in read methods --- .../java/org/apache/metron/common/utils/HDFSUtils.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java index 2e7ab498e8..31a4d0efbc 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java @@ -35,8 +35,9 @@ public static byte[] readBytes(String path) throws IOException { public static byte[] readBytes(Path inPath) throws IOException { FileSystem fs = FileSystem.get(inPath.toUri(), new Configuration()); - FSDataInputStream inputStream = fs.open(inPath); - return IOUtils.toByteArray(inputStream); + try (FSDataInputStream inputStream = fs.open(inPath)) { + return IOUtils.toByteArray(inputStream); + } } /** @@ -68,8 +69,9 @@ public static List readFile(String path) throws IOException { public static List readFile(Configuration config, String path) throws IOException { Path inPath = new Path(path); FileSystem fs = FileSystem.get(inPath.toUri(), config); - FSDataInputStream inputStream = fs.open(inPath); - return IOUtils.readLines(inputStream, "UTF-8"); + try (FSDataInputStream inputStream = fs.open(inPath)) { + return IOUtils.readLines(inputStream, "UTF-8"); + } } /** From 7b3dfbf7ec4ee704d4306d5dca22cdfec8ae2b24 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Tue, 17 Jul 2018 12:56:51 -0600 Subject: [PATCH 29/36] Fix iterim path option. Fix timer thread and synchronization issues. --- .../metron/rest/model/pcap/PcapRequest.java | 4 +- .../org/apache/metron/pcap/PcapJobTest.java | 5 +- .../PcapTopologyIntegrationTest.java | 6 +- .../apache/metron/pcap/query/PcapCliTest.java | 10 +-- .../apache/metron/pcap/config/PcapConfig.java | 4 +- .../metron/pcap/config/PcapOptions.java | 3 +- .../metron/pcap/finalizer/PcapFinalizer.java | 2 +- .../org/apache/metron/pcap/mr/PcapJob.java | 74 ++++++++++--------- 8 files changed, 53 insertions(+), 55 deletions(-) diff --git a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java index cfc212aa84..5941d17328 100644 --- a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java +++ b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java @@ -31,11 +31,11 @@ public PcapRequest() { } public String getBaseOutputPath() { - return PcapOptions.INTERIM_RESULT_PATH.get(this, String.class); + return PcapOptions.BASE_INTERIM_RESULT_PATH.get(this, String.class); } public void setBaseOutputPath(String baseOutputPath) { - PcapOptions.INTERIM_RESULT_PATH.put(this, baseOutputPath); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(this, baseOutputPath); } public String getBasePath() { diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java index ccce608fbc..1e389d9958 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/PcapJobTest.java @@ -100,7 +100,7 @@ public void setup() throws IOException { PcapOptions.HADOOP_CONF.put(config, hadoopConfig); PcapOptions.FILESYSTEM.put(config, FileSystem.get(hadoopConfig)); PcapOptions.BASE_PATH.put(config, basePath); - PcapOptions.INTERIM_RESULT_PATH.put(config, baseOutPath); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(config, baseOutPath); PcapOptions.START_TIME_NS.put(config, startTime); PcapOptions.END_TIME_NS.put(config, endTime); PcapOptions.NUM_REDUCERS.put(config, numReducers); @@ -190,7 +190,8 @@ public void job_fails_with_killed_status_synchronously() throws Exception { @Test public void job_succeeds_asynchronously() throws Exception { - when(mrJob.isComplete()).thenReturn(true); + // not complete a few times to make sure cancel works as expected + when(mrJob.isComplete()).thenReturn(false, false, false, true); when(mrStatus.getState()).thenReturn(org.apache.hadoop.mapreduce.JobStatus.State.SUCCEEDED); when(mrJob.getStatus()).thenReturn(mrStatus); Statusable statusable = testJob.submit(finalizer, config); diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java index c403eacd27..78fc9669ff 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java @@ -23,7 +23,6 @@ import com.google.common.collect.Collections2; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; - import java.io.ByteArrayOutputStream; import java.io.File; import java.io.FilenameFilter; @@ -34,7 +33,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Properties; import javax.annotation.Nullable; import kafka.consumer.ConsumerIterator; @@ -59,7 +57,6 @@ import org.apache.metron.integration.components.MRComponent; import org.apache.metron.integration.components.ZKServerComponent; import org.apache.metron.integration.utils.KafkaUtil; -import org.apache.metron.job.Finalizer; import org.apache.metron.job.JobStatus; import org.apache.metron.job.Statusable; import org.apache.metron.pcap.PacketInfo; @@ -68,7 +65,6 @@ import org.apache.metron.pcap.config.PcapOptions; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.filter.query.QueryPcapFilter; -import org.apache.metron.pcap.finalizer.PcapFinalizer; import org.apache.metron.pcap.finalizer.PcapFinalizerStrategies; import org.apache.metron.pcap.mr.PcapJob; import org.apache.metron.spout.pcap.Endianness; @@ -260,7 +256,7 @@ public ProcessorResult getResult() { PcapOptions.HADOOP_CONF.put(configuration, hadoopConf); PcapOptions.FILESYSTEM.put(configuration, FileSystem.get(hadoopConf)); PcapOptions.BASE_PATH.put(configuration, new Path(inputDir.getAbsolutePath())); - PcapOptions.BASE_INTERRIM_RESULT_PATH.put(configuration, new Path(interimResultDir.getAbsolutePath())); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(configuration, new Path(interimResultDir.getAbsolutePath())); PcapOptions.START_TIME_NS.put(configuration, getTimestamp(4, pcapEntries)); PcapOptions.END_TIME_NS.put(configuration, getTimestamp(5, pcapEntries)); PcapOptions.NUM_REDUCERS.put(configuration, 10); diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java index 2ad0657919..ced57f6398 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java @@ -92,7 +92,7 @@ public void runs_fixed_pcap_filter_job_with_default_argument_list() throws Excep }}; FixedPcapConfig config = new FixedPcapConfig(prefixStrategy); PcapOptions.BASE_PATH.put(config, CliParser.BASE_PATH_DEFAULT); - PcapOptions.INTERIM_RESULT_PATH.put(config, CliParser.BASE_OUTPUT_PATH_DEFAULT); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(config, CliParser.BASE_OUTPUT_PATH_DEFAULT); PcapOptions.FIELDS.put(config, query); PcapOptions.NUM_REDUCERS.put(config, 10); PcapOptions.START_TIME_MS.put(config, 500L); @@ -158,7 +158,7 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list_and_default_datef }}; FixedPcapConfig config = new FixedPcapConfig(prefixStrategy); PcapOptions.BASE_PATH.put(config, "/base/path"); - PcapOptions.INTERIM_RESULT_PATH.put(config, "/base/output/path"); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(config, "/base/output/path"); PcapOptions.FIELDS.put(config, query); PcapOptions.NUM_REDUCERS.put(config, 10); PcapOptions.START_TIME_MS.put(config, 500L); @@ -204,7 +204,7 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list() throws Exceptio FixedPcapConfig config = new FixedPcapConfig(prefixStrategy); PcapOptions.BASE_PATH.put(config, "/base/path"); - PcapOptions.INTERIM_RESULT_PATH.put(config, "/base/output/path"); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(config, "/base/output/path"); PcapOptions.FIELDS.put(config, query); PcapOptions.NUM_REDUCERS.put(config, 10); PcapOptions.START_TIME_MS.put(config, startAsNanos / 1000000L); // needed bc defaults in config @@ -241,7 +241,7 @@ public void runs_query_pcap_filter_job_with_default_argument_list() throws Excep String query = "some query string"; FixedPcapConfig config = new FixedPcapConfig(prefixStrategy); PcapOptions.BASE_PATH.put(config, CliParser.BASE_PATH_DEFAULT); - PcapOptions.INTERIM_RESULT_PATH.put(config, CliParser.BASE_OUTPUT_PATH_DEFAULT); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(config, CliParser.BASE_OUTPUT_PATH_DEFAULT); PcapOptions.FIELDS.put(config, query); PcapOptions.NUM_REDUCERS.put(config, 10); PcapOptions.START_TIME_MS.put(config, 500L); @@ -269,7 +269,7 @@ public void runs_query_pcap_filter_job_with_full_argument_list() throws Exceptio String query = "some query string"; FixedPcapConfig config = new FixedPcapConfig(prefixStrategy); PcapOptions.BASE_PATH.put(config, "/base/path"); - PcapOptions.INTERIM_RESULT_PATH.put(config, "/base/output/path"); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(config, "/base/output/path"); PcapOptions.FIELDS.put(config, query); PcapOptions.NUM_REDUCERS.put(config, 10); PcapOptions.START_TIME_MS.put(config, 500L); // needed bc defaults in config diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java index 20b9a0eff7..3201ef8fbf 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java @@ -78,7 +78,7 @@ public String getBasePath() { } public String getInterimResultPath() { - return PcapOptions.INTERIM_RESULT_PATH.get(this, String.class); + return PcapOptions.BASE_INTERIM_RESULT_PATH.get(this, String.class); } public long getStartTimeMs() { @@ -94,7 +94,7 @@ public void setBasePath(String basePath) { } public void setInterimResultPath(String baseOutputPath) { - PcapOptions.INTERIM_RESULT_PATH.put(this, baseOutputPath); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(this, baseOutputPath); } public void setStartTimeMs(long startTime) { diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java index 68bb17dd2b..4630850be3 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java @@ -26,8 +26,7 @@ public enum PcapOptions implements ConfigOption { JOB_NAME("jobName"), FINAL_FILENAME_PREFIX("finalFilenamePrefix"), BASE_PATH("basePath", (s, o) -> o == null ? null : new Path(o.toString())), - BASE_INTERRIM_RESULT_PATH("baseInterimResultPath", (s,o) -> o == null?null:new Path(o.toString())), - INTERIM_RESULT_PATH("interimResultPath", (s, o) -> o == null ? null : new Path(o.toString())), + BASE_INTERIM_RESULT_PATH("baseInterimResultPath", (s, o) -> o == null ? null : new Path(o.toString())), FINAL_OUTPUT_PATH("finalOutputPath", (s, o) -> o == null ? null : new Path(o.toString())), NUM_REDUCERS("numReducers"), START_TIME_MS("startTimeMs"), diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java index a2fada9b3b..cb54cffa64 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java @@ -57,7 +57,7 @@ protected PcapResultsWriter getResultsWriter() { public Pageable finalizeJob(Map config) throws JobException { Configuration hadoopConfig = PcapOptions.HADOOP_CONF.get(config, Configuration.class); int recPerFile = PcapOptions.NUM_RECORDS_PER_FILE.get(config, Integer.class); - Path interimResultPath = PcapOptions.INTERIM_RESULT_PATH + Path interimResultPath = PcapOptions.BASE_INTERIM_RESULT_PATH .get(config, PcapOptions.STRING_TO_PATH, Path.class); FileSystem fs = PcapOptions.FILESYSTEM.get(config, FileSystem.class); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index c685a147fb..36b78c4da7 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -214,7 +214,7 @@ public Statusable submit(Finalizer finalizer, Map co Configuration hadoopConf = PcapOptions.HADOOP_CONF.get(configuration, Configuration.class); FileSystem fileSystem = PcapOptions.FILESYSTEM.get(configuration, FileSystem.class); Path basePath = PcapOptions.BASE_PATH.getTransformed(configuration, Path.class); - Path baseInterimResultPath = PcapOptions.BASE_INTERRIM_RESULT_PATH.getTransformed(configuration, Path.class); + Path baseInterimResultPath = PcapOptions.BASE_INTERIM_RESULT_PATH.getTransformed(configuration, Path.class); long startTime = PcapOptions.START_TIME_NS.get(configuration, Long.class); long endTime = PcapOptions.END_TIME_NS.get(configuration, Long.class); int numReducers = PcapOptions.NUM_REDUCERS.get(configuration, Integer.class); @@ -262,7 +262,7 @@ public Statusable query(Optional jobName, LOG.debug("Executing query {} on timerange from {} to {}", filterImpl.queryToString(fields), from, to); } Path interimResultPath = new Path(baseInterimResultPath, outputDirName); - PcapOptions.INTERIM_RESULT_PATH.put(configuration, interimResultPath); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(configuration, interimResultPath); mrJob = createJob(jobName , basePath , interimResultPath @@ -286,7 +286,7 @@ private void startJobStatusTimerThread(long interval) { @Override public void run() { try { - synchronized (jobState) { + synchronized (this) { if (jobState == State.RUNNING) { if (mrJob.isComplete()) { switch (mrJob.getStatus().getState()) { @@ -305,8 +305,8 @@ public void run() { jobState = State.KILLED; break; } + cancel(); // be gone, ye! } - cancel(); // be gone, ye! } } } catch (InterruptedException | IOException e) { @@ -399,45 +399,47 @@ public JobType getJobType() { return JobType.MAP_REDUCE; } + /** + * Synchronized for mrJob and jobState + */ @Override - public JobStatus getStatus() throws JobException { + public synchronized JobStatus getStatus() throws JobException { JobStatus status = new JobStatus(); if (mrJob == null) { status.withPercentComplete(100).withState(State.SUCCEEDED); } else { try { - synchronized (this) { - org.apache.hadoop.mapreduce.JobStatus mrJobStatus = mrJob.getStatus(); - status.withJobId(mrJobStatus.getJobID().toString()); - if (jobState == State.SUCCEEDED) { - status.withPercentComplete(100).withState(State.SUCCEEDED) - .withDescription("Job complete"); - } else { - if (mrJob.isComplete()) { - status.withPercentComplete(100); - switch (mrJobStatus.getState()) { - case SUCCEEDED: - status.withState(State.FINALIZING).withDescription(State.FINALIZING.toString()); - break; - case FAILED: - status.withState(State.FAILED).withDescription(State.FAILED.toString()); - break; - case KILLED: - status.withState(State.KILLED).withDescription(State.KILLED.toString()); - break; - default: - throw new IllegalStateException( - "Unknown job state reported as 'complete' by mapreduce framework: " + mrJobStatus.getState()); - } - } else { - float mapProg = mrJob.mapProgress(); - float reduceProg = mrJob.reduceProgress(); - float totalProgress = ((mapProg / 2) + (reduceProg / 2)) * 100; - String description = String - .format("map: %s%%, reduce: %s%%", mapProg * 100, reduceProg * 100); - status.withPercentComplete(totalProgress).withState(State.RUNNING) - .withDescription(description); + org.apache.hadoop.mapreduce.JobStatus mrJobStatus = mrJob.getStatus(); + status.withJobId(mrJobStatus.getJobID().toString()); + if (jobState == State.SUCCEEDED) { + status.withPercentComplete(100).withState(State.SUCCEEDED) + .withDescription("Job complete"); + } else { + if (mrJob.isComplete()) { + status.withPercentComplete(100); + switch (mrJobStatus.getState()) { + case SUCCEEDED: + status.withState(State.FINALIZING).withDescription(State.FINALIZING.toString()); + break; + case FAILED: + status.withState(State.FAILED).withDescription(State.FAILED.toString()); + break; + case KILLED: + status.withState(State.KILLED).withDescription(State.KILLED.toString()); + break; + default: + throw new IllegalStateException( + "Unknown job state reported as 'complete' by mapreduce framework: " + + mrJobStatus.getState()); } + } else { + float mapProg = mrJob.mapProgress(); + float reduceProg = mrJob.reduceProgress(); + float totalProgress = ((mapProg / 2) + (reduceProg / 2)) * 100; + String description = String + .format("map: %s%%, reduce: %s%%", mapProg * 100, reduceProg * 100); + status.withPercentComplete(totalProgress).withState(State.RUNNING) + .withDescription(description); } } } catch (Exception e) { From 72e8ee0115a0478ec24429d46481ba17d0200825 Mon Sep 17 00:00:00 2001 From: merrimanr Date: Wed, 18 Jul 2018 07:34:26 -0500 Subject: [PATCH 30/36] fixed interim path error --- .../java/org/apache/metron/pcap/finalizer/PcapFinalizer.java | 2 +- .../src/main/java/org/apache/metron/pcap/mr/PcapJob.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java index cb54cffa64..a2fada9b3b 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java @@ -57,7 +57,7 @@ protected PcapResultsWriter getResultsWriter() { public Pageable finalizeJob(Map config) throws JobException { Configuration hadoopConfig = PcapOptions.HADOOP_CONF.get(config, Configuration.class); int recPerFile = PcapOptions.NUM_RECORDS_PER_FILE.get(config, Integer.class); - Path interimResultPath = PcapOptions.BASE_INTERIM_RESULT_PATH + Path interimResultPath = PcapOptions.INTERIM_RESULT_PATH .get(config, PcapOptions.STRING_TO_PATH, Path.class); FileSystem fs = PcapOptions.FILESYSTEM.get(config, FileSystem.class); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index 55e077c90b..312a87a2c8 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -272,7 +272,7 @@ public Statusable query(Optional jobName, LOG.debug("Executing query {} on timerange from {} to {}", filterImpl.queryToString(fields), from, to); } Path interimResultPath = new Path(baseInterimResultPath, outputDirName); - PcapOptions.BASE_INTERIM_RESULT_PATH.put(configuration, interimResultPath); + PcapOptions.INTERIM_RESULT_PATH.put(configuration, interimResultPath); mrJob = createJob(jobName , basePath , interimResultPath From 6e6fb61bf74ffc3ceb5bced7081a53e8c2d1a305 Mon Sep 17 00:00:00 2001 From: merrimanr Date: Wed, 18 Jul 2018 08:01:52 -0500 Subject: [PATCH 31/36] pr feedback --- .../metron/rest/model/pcap/PcapRequest.java | 2 +- .../metron/rest/config/PcapJobSupplier.java | 15 ++++++---- .../rest/service/impl/PcapServiceImpl.java | 5 ++++ .../metron/job/RuntimeJobException.java | 30 +++++++++++++++++++ .../org/apache/metron/pcap/query/PcapCli.java | 12 -------- .../apache/metron/pcap/query/PcapCliTest.java | 2 -- .../metron/pcap/config/PcapOptions.java | 1 + .../pcap/finalizer/PcapCliFinalizer.java | 4 +-- .../metron/pcap/finalizer/PcapFinalizer.java | 8 ++--- .../pcap/finalizer/PcapRestFinalizer.java | 16 ++++++++-- .../org/apache/metron/pcap/mr/PcapJob.java | 4 ++- 11 files changed, 69 insertions(+), 30 deletions(-) create mode 100644 metron-platform/metron-job/src/main/java/org/apache/metron/job/RuntimeJobException.java diff --git a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java index 7b7d61b58d..64ed93272d 100644 --- a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java +++ b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java @@ -28,7 +28,7 @@ public PcapRequest() { super(new HashMap<>()); setStartTimeMs(0L); setEndTimeMs(System.currentTimeMillis()); - setNumReducers(1); + setNumReducers(10); } public String getBasePath() { diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapJobSupplier.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapJobSupplier.java index 3e1e5f69c7..76f4e01213 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapJobSupplier.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapJobSupplier.java @@ -18,10 +18,11 @@ package org.apache.metron.rest.config; import org.apache.hadoop.fs.Path; +import org.apache.metron.job.Finalizer; import org.apache.metron.job.JobException; -import org.apache.metron.job.JobStatus; +import org.apache.metron.job.RuntimeJobException; import org.apache.metron.job.Statusable; -import org.apache.metron.pcap.finalizer.PcapFinalizerStrategies; +import org.apache.metron.pcap.finalizer.PcapRestFinalizer; import org.apache.metron.pcap.mr.PcapJob; import org.apache.metron.rest.model.pcap.PcapRequest; @@ -30,15 +31,15 @@ public class PcapJobSupplier implements Supplier> { private PcapRequest pcapRequest; + private Finalizer finalizer; @Override public Statusable get() { try { PcapJob pcapJob = createPcapJob(); - return pcapJob.submit(PcapFinalizerStrategies.REST, pcapRequest); + return pcapJob.submit(finalizer, pcapRequest); } catch (JobException e) { - return null; - //return new JobStatus().withState(JobStatus.State.FAILED).withDescription(JobStatus.State.FAILED.toString()); + throw new RuntimeJobException(e.getMessage()); } } @@ -46,6 +47,10 @@ public void setPcapRequest(PcapRequest pcapRequest) { this.pcapRequest = pcapRequest; } + public void setFinalizer(PcapRestFinalizer pcapRestFinalizer) { + this.finalizer = pcapRestFinalizer; + } + protected PcapJob createPcapJob() { return new PcapJob(); } diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java index 1b18f49860..91dc768b52 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java @@ -26,6 +26,8 @@ import org.apache.metron.job.Statusable; import org.apache.metron.job.manager.JobManager; import org.apache.metron.pcap.config.PcapOptions; +import org.apache.metron.pcap.finalizer.PcapFinalizerStrategies; +import org.apache.metron.pcap.finalizer.PcapRestFinalizer; import org.apache.metron.rest.MetronRestConstants; import org.apache.metron.rest.RestException; import org.apache.metron.rest.config.PcapJobSupplier; @@ -61,6 +63,9 @@ public PcapStatus fixed(String username, FixedPcapRequest fixedPcapRequest) thro setPcapOptions(fixedPcapRequest); fixedPcapRequest.setFields(); pcapJobSupplier.setPcapRequest(fixedPcapRequest); + PcapRestFinalizer pcapRestFinalizer = new PcapRestFinalizer(); + pcapRestFinalizer.setUser(username); + pcapJobSupplier.setFinalizer(pcapRestFinalizer); JobStatus jobStatus = jobManager.submit(pcapJobSupplier, username); return jobStatusToPcapStatus(jobStatus); } catch (IOException | JobException e) { diff --git a/metron-platform/metron-job/src/main/java/org/apache/metron/job/RuntimeJobException.java b/metron-platform/metron-job/src/main/java/org/apache/metron/job/RuntimeJobException.java new file mode 100644 index 0000000000..9013ef8a40 --- /dev/null +++ b/metron-platform/metron-job/src/main/java/org/apache/metron/job/RuntimeJobException.java @@ -0,0 +1,30 @@ +/** + * 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.metron.job; + +public class RuntimeJobException extends RuntimeException { + + public RuntimeJobException(String message) { + super(message); + } + + public RuntimeJobException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java index 34629215f1..1a23740f69 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java @@ -98,12 +98,6 @@ public int run(String[] args) { fixedParser.printHelp(); return 0; } - Pair time = timeAsNanosecondsSinceEpoch(config.getStartTimeMs(), config.getEndTimeMs()); - long startTime = time.getLeft(); - long endTime = time.getRight(); - - PcapOptions.START_TIME_NS.put(commonConfig, startTime); - PcapOptions.END_TIME_NS.put(commonConfig, endTime); PcapOptions.FILTER_IMPL.put(commonConfig, new FixedPcapFilter.Configurator()); PcapOptions.HADOOP_CONF.put(commonConfig, hadoopConf); try { @@ -128,12 +122,6 @@ public int run(String[] args) { queryParser.printHelp(); return 0; } - Pair time = timeAsNanosecondsSinceEpoch(config.getStartTimeMs(), config.getEndTimeMs()); - long startTime = time.getLeft(); - long endTime = time.getRight(); - - PcapOptions.START_TIME_NS.put(commonConfig, startTime); - PcapOptions.END_TIME_NS.put(commonConfig, endTime); PcapOptions.FILTER_IMPL.put(commonConfig, new FixedPcapFilter.Configurator()); PcapOptions.HADOOP_CONF.put(commonConfig, hadoopConf); try { diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java index ced57f6398..d415831ed6 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java @@ -209,8 +209,6 @@ public void runs_fixed_pcap_filter_job_with_full_argument_list() throws Exceptio PcapOptions.NUM_REDUCERS.put(config, 10); PcapOptions.START_TIME_MS.put(config, startAsNanos / 1000000L); // needed bc defaults in config PcapOptions.END_TIME_MS.put(config, endAsNanos / 1000000L); // needed bc defaults in config - PcapOptions.START_TIME_NS.put(config, startAsNanos); - PcapOptions.END_TIME_NS.put(config, endAsNanos); PcapOptions.NUM_RECORDS_PER_FILE.put(config, 1000); when(jobRunner.submit(isA(Finalizer.class), argThat(mapContaining(config)))).thenReturn(jobRunner); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java index bf632f6cea..218b6d814a 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java @@ -24,6 +24,7 @@ public enum PcapOptions implements ConfigOption { JOB_NAME("jobName"), + JOB_ID("jobId"), FINAL_FILENAME_PREFIX("finalFilenamePrefix"), BASE_PATH("basePath", (s, o) -> o == null ? null : new Path(o.toString())), BASE_INTERIM_RESULT_PATH("baseInterimResultPath", (s, o) -> o == null?null:new Path(o.toString())), diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java index ecdce4ecbf..f98309a530 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java @@ -28,10 +28,10 @@ public class PcapCliFinalizer extends PcapFinalizer { @Override - protected String getOutputFileName(Map config, int partition) { + protected Path getOutputPath(Map config, int partition) { Path finalOutputPath = PcapOptions.FINAL_OUTPUT_PATH.get(config, PcapOptions.STRING_TO_PATH, Path.class); String prefix = PcapOptions.FINAL_FILENAME_PREFIX.get(config, String.class); - return String.format("%s/pcap-data-%s+%04d.pcap", finalOutputPath, prefix, partition); + return new Path(String.format("%s/pcap-data-%s+%04d.pcap", finalOutputPath, prefix, partition)); } } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java index a2fada9b3b..ac92addd8a 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java @@ -73,10 +73,10 @@ public Pageable finalizeJob(Map config) throws JobExceptio int part = 1; if (partitions.iterator().hasNext()) { for (List data : partitions) { - String outFileName = getOutputFileName(config, part++); + Path outputPath = getOutputPath(config, part++); if (data.size() > 0) { - getResultsWriter().write(hadoopConfig, data, outFileName); - outFiles.add(new Path(outFileName)); + getResultsWriter().write(hadoopConfig, data, outputPath.toUri().getPath()); + outFiles.add(outputPath); } } } else { @@ -94,7 +94,7 @@ public Pageable finalizeJob(Map config) throws JobExceptio return new PcapPages(outFiles); } - protected abstract String getOutputFileName(Map config, int partition); + protected abstract Path getOutputPath(Map config, int partition); /** * Returns a lazily-read Iterable over a set of sequence files. diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java index 665b103370..a7a9347673 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java @@ -18,8 +18,10 @@ package org.apache.metron.pcap.finalizer; +import java.nio.file.Paths; import java.util.Map; import org.apache.hadoop.fs.Path; +import org.apache.metron.job.Statusable; import org.apache.metron.pcap.config.PcapOptions; /** @@ -27,10 +29,18 @@ */ public class PcapRestFinalizer extends PcapFinalizer { + private String user; + private String jobType = Statusable.JobType.MAP_REDUCE.name(); + + public void setUser(String user) { + this.user = user; + } + @Override - protected String getOutputFileName(Map config, int partition) { - Path finalOutputPath = PcapOptions.FINAL_OUTPUT_PATH.getTransformed(config, Path.class); - return String.format("%s/page-%s", finalOutputPath, partition); + protected Path getOutputPath(Map config, int partition) { + String jobId = PcapOptions.JOB_ID.get(config, String.class); + String finalOutputPath = PcapOptions.FINAL_OUTPUT_PATH.get(config, String.class); + return new Path(String.format("%s/%s/%s/%s/page-%s", finalOutputPath, user, jobType, jobId, partition)); } } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index 312a87a2c8..1dd670dae5 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -232,7 +232,7 @@ public Statusable submit(Finalizer finalizer, Map co PcapFilterConfigurator filterImpl = PcapOptions.FILTER_IMPL.get(configuration, PcapFilterConfigurator.class); try { - return query(jobName, + Statusable statusable = query(jobName, basePath, baseInterimResultPath, startTime, @@ -243,6 +243,8 @@ public Statusable submit(Finalizer finalizer, Map co new Configuration(hadoopConf), fileSystem, filterImpl); + PcapOptions.JOB_ID.put(configuration, statusable.getStatus().getJobId()); + return statusable; } catch (IOException | InterruptedException | ClassNotFoundException e) { throw new JobException("Failed to run pcap query.", e); } From a813f3459f6a9c1a73a92a692d12d0132df3f738 Mon Sep 17 00:00:00 2001 From: merrimanr Date: Wed, 18 Jul 2018 08:26:04 -0500 Subject: [PATCH 32/36] fixed default numReducers in test --- .../metron/rest/controller/PcapControllerIntegrationTest.java | 2 +- .../apache/metron/rest/service/impl/PcapServiceImplTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/PcapControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/PcapControllerIntegrationTest.java index 1b43bdf28c..032429ce5b 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/PcapControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/PcapControllerIntegrationTest.java @@ -166,7 +166,7 @@ public void testFixedRequestDefaults() throws Exception { Assert.assertEquals(0, mockPcapJob.getStartTimeNs()); Assert.assertTrue(beforeJobTime < mockPcapJob.getEndTimeNs() / 1000000); Assert.assertTrue(System.currentTimeMillis() > mockPcapJob.getEndTimeNs() / 1000000); - Assert.assertEquals(1, mockPcapJob.getNumReducers()); + Assert.assertEquals(10, mockPcapJob.getNumReducers()); Assert.assertTrue(mockPcapJob.getFilterImpl() instanceof FixedPcapFilter.Configurator); Map actualFixedFields = mockPcapJob.getFixedFields(); Assert.assertEquals("192.168.1.2", actualFixedFields.get(Constants.Fields.SRC_ADDR.getName())); diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java index cc3f9eedbd..0738329d1b 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java @@ -162,7 +162,7 @@ public void fixedShouldProperlyCallPcapJobQueryWithDefaults() throws Exception { Assert.assertEquals(0, mockPcapJob.getStartTimeNs()); Assert.assertTrue(beforeJobTime <= mockPcapJob.getEndTimeNs() / 1000000); Assert.assertTrue(System.currentTimeMillis() >= mockPcapJob.getEndTimeNs() / 1000000); - Assert.assertEquals(1, mockPcapJob.getNumReducers()); + Assert.assertEquals(10, mockPcapJob.getNumReducers()); Assert.assertEquals(100, mockPcapJob.getRecPerFile()); Assert.assertTrue(mockPcapJob.getFilterImpl() instanceof FixedPcapFilter.Configurator); Assert.assertEquals(new HashMap<>(), mockPcapJob.getFixedFields()); From 6a4df3e83b459940c65733092e452cd167d805dc Mon Sep 17 00:00:00 2001 From: merrimanr Date: Wed, 18 Jul 2018 10:47:47 -0500 Subject: [PATCH 33/36] added pageTotal to pcap status --- .../apache/metron/rest/model/pcap/PcapStatus.java | 14 +++++++------- .../metron/rest/service/impl/PcapServiceImpl.java | 10 ++++++++-- .../controller/PcapControllerIntegrationTest.java | 10 +++++++++- .../org/apache/metron/rest/mock/MockPcapJob.java | 4 ++++ .../rest/service/impl/PcapServiceImplTest.java | 7 +++++++ 5 files changed, 35 insertions(+), 10 deletions(-) diff --git a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapStatus.java b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapStatus.java index fe0e51c54e..f004eb5213 100644 --- a/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapStatus.java +++ b/metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapStatus.java @@ -23,7 +23,7 @@ public class PcapStatus { private String jobStatus; private String description; private Double percentComplete = 0.0; - private Integer size = 0; + private Integer pageTotal = 0; public String getJobId() { return jobId; @@ -57,12 +57,12 @@ public void setPercentComplete(Double percentComplete) { this.percentComplete = percentComplete; } - public Integer getSize() { - return size; + public Integer getPageTotal() { + return pageTotal; } - public void setSize(Integer size) { - this.size = size; + public void setPageTotal(Integer size) { + this.pageTotal = size; } @Override @@ -76,7 +76,7 @@ public boolean equals(Object o) { (getJobStatus() != null ? getJobStatus().equals(pcapStatus.getJobStatus()) : pcapStatus.getJobStatus() != null) && (getDescription() != null ? getDescription().equals(pcapStatus.getDescription()) : pcapStatus.getDescription() != null) && (getPercentComplete() != null ? getPercentComplete().equals(pcapStatus.getPercentComplete()) : pcapStatus.getPercentComplete() != null) && - (getSize() != null ? getSize().equals(pcapStatus.getSize()) : pcapStatus.getSize() != null); + (getPageTotal() != null ? getPageTotal().equals(pcapStatus.getPageTotal()) : pcapStatus.getPageTotal() != null); } @Override @@ -85,7 +85,7 @@ public int hashCode() { result = 31 * result + (getJobStatus() != null ? getJobStatus().hashCode() : 0); result = 31 * result + (getDescription() != null ? getDescription().hashCode() : 0); result = 31 * result + (getPercentComplete() != null ? getPercentComplete().hashCode() : 0); - result = 31 * result + (getSize() != null ? getSize().hashCode() : 0); + result = 31 * result + (getPageTotal() != null ? getPageTotal().hashCode() : 0); return result; } } diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java index 91dc768b52..2b228a43f2 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java @@ -23,10 +23,10 @@ import org.apache.metron.job.JobException; import org.apache.metron.job.JobNotFoundException; import org.apache.metron.job.JobStatus; +import org.apache.metron.job.Pageable; import org.apache.metron.job.Statusable; import org.apache.metron.job.manager.JobManager; import org.apache.metron.pcap.config.PcapOptions; -import org.apache.metron.pcap.finalizer.PcapFinalizerStrategies; import org.apache.metron.pcap.finalizer.PcapRestFinalizer; import org.apache.metron.rest.MetronRestConstants; import org.apache.metron.rest.RestException; @@ -80,8 +80,14 @@ public PcapStatus getJobStatus(String username, String jobId) throws RestExcepti Statusable statusable = jobManager.getJob(username, jobId); if (statusable != null) { pcapStatus = jobStatusToPcapStatus(statusable.getStatus()); + if (statusable.isDone()) { + Pageable pageable = statusable.get(); + if (pageable != null) { + pcapStatus.setPageTotal(pageable.getSize()); + } + } } - } catch (JobNotFoundException e) { + } catch (JobNotFoundException | InterruptedException e) { // do nothing and return null pcapStatus } catch (JobException e) { throw new RestException(e); diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/PcapControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/PcapControllerIntegrationTest.java index 032429ce5b..462d83d75a 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/PcapControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/PcapControllerIntegrationTest.java @@ -18,10 +18,13 @@ package org.apache.metron.rest.controller; import org.adrianwalker.multilinestring.Multiline; +import org.apache.hadoop.fs.Path; import org.apache.metron.common.Constants; import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.job.JobStatus; +import org.apache.metron.job.Pageable; import org.apache.metron.pcap.PcapHelper; +import org.apache.metron.pcap.PcapPages; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.rest.mock.MockPcapJob; import org.apache.metron.rest.model.PcapResponse; @@ -195,10 +198,15 @@ public void testGetStatus() throws Exception { mockPcapJob.setStatus(new JobStatus().withJobId("jobId").withState(JobStatus.State.SUCCEEDED)); + Pageable pageable = new PcapPages(Arrays.asList(new Path("path1"), new Path("path1"))); + mockPcapJob.setIsDone(true); + mockPcapJob.setPageable(pageable); + this.mockMvc.perform(get(pcapUrl + "/jobId").with(httpBasic(user, password))) .andExpect(status().isOk()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) - .andExpect(jsonPath("$.jobStatus").value("SUCCEEDED")); + .andExpect(jsonPath("$.jobStatus").value("SUCCEEDED")) + .andExpect(jsonPath("$.pageTotal").value(2)); mockPcapJob.setStatus(new JobStatus().withJobId("jobId").withState(JobStatus.State.FINALIZING)); diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java index 0efb1667cf..df65635514 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/mock/MockPcapJob.java @@ -88,6 +88,10 @@ public void setPageable(Pageable pageable) throws JobException, Interrupte when(statusable.get()).thenReturn(pageable); } + public void setIsDone(boolean isDone) { + when(statusable.isDone()).thenReturn(isDone); + } + public void setResults(List pcaps) { when(sequenceFileIterable.iterator()).thenReturn(pcaps.iterator()); } diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java index 0738329d1b..2b6bea3599 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/PcapServiceImplTest.java @@ -22,6 +22,7 @@ import org.apache.metron.common.Constants; import org.apache.metron.job.JobException; import org.apache.metron.job.JobStatus; +import org.apache.metron.job.Pageable; import org.apache.metron.job.manager.InMemoryJobManager; import org.apache.metron.job.manager.JobManager; import org.apache.metron.pcap.PcapHelper; @@ -47,6 +48,7 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import static org.powermock.api.mockito.PowerMockito.doReturn; +import static org.powermock.api.mockito.PowerMockito.whenNew; @SuppressWarnings("ALL") public class PcapServiceImplTest { @@ -193,7 +195,11 @@ public void getStatusShouldProperlyReturnStatus() throws Exception { .withState(JobStatus.State.SUCCEEDED) .withDescription("description") .withPercentComplete(100.0); + Pageable pageable = mock(Pageable.class); + when(pageable.getSize()).thenReturn(2); when(mockPcapJob.getStatus()).thenReturn(actualJobStatus); + when(mockPcapJob.isDone()).thenReturn(true); + when(mockPcapJob.get()).thenReturn(pageable); when(jobManager.getJob("user", "jobId")).thenReturn(mockPcapJob); PcapServiceImpl pcapService = new PcapServiceImpl(environment, configuration, mockPcapJobSupplier, jobManager); @@ -202,6 +208,7 @@ public void getStatusShouldProperlyReturnStatus() throws Exception { expectedPcapStatus.setJobStatus(JobStatus.State.SUCCEEDED.name()); expectedPcapStatus.setDescription("description"); expectedPcapStatus.setPercentComplete(100.0); + expectedPcapStatus.setPageTotal(2); Assert.assertEquals(expectedPcapStatus, pcapService.getJobStatus("user", "jobId")); } From aa8d92ea158231d14e88a5af01c40ed1fa3cb9d3 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Wed, 18 Jul 2018 12:22:47 -0600 Subject: [PATCH 34/36] Fix additional internal interim result path --- .../java/org/apache/metron/pcap/query/CliParser.java | 9 +++++---- .../java/org/apache/metron/pcap/query/PcapCliTest.java | 5 ++--- .../java/org/apache/metron/pcap/config/PcapConfig.java | 6 +++--- .../java/org/apache/metron/pcap/config/PcapOptions.java | 1 + .../org/apache/metron/pcap/finalizer/PcapFinalizer.java | 2 +- .../src/main/java/org/apache/metron/pcap/mr/PcapJob.java | 2 +- 6 files changed, 13 insertions(+), 12 deletions(-) diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliParser.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliParser.java index 07a1c3f79e..e6534c5a23 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliParser.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/CliParser.java @@ -26,7 +26,7 @@ */ public class CliParser { public static final String BASE_PATH_DEFAULT = "/apps/metron/pcap"; - public static final String BASE_OUTPUT_PATH_DEFAULT = "/tmp"; + public static final String BASE_INTERIM_OUTPUT_PATH_DEFAULT = "/tmp"; public static final int NUM_REDUCERS_DEFAULT = 10; public static final int NUM_RECORDS_PER_FILE_DEFAULT = 10000; private CommandLineParser parser; @@ -41,7 +41,8 @@ public Options buildOptions() { Options options = new Options(); options.addOption(newOption("h", "help", false, "Display help")); options.addOption(newOption("bp", "base_path", true, String.format("Base PCAP data path. Default is '%s'", BASE_PATH_DEFAULT))); - options.addOption(newOption("bop", "base_output_path", true, String.format("Query result output path. Default is '%s'", BASE_OUTPUT_PATH_DEFAULT))); + options.addOption(newOption("bop", "base_output_path", true, String.format("Query result output path. Default is '%s'", + BASE_INTERIM_OUTPUT_PATH_DEFAULT))); options.addOption(newOption("st", "start_time", true, "(required) Packet start time range.", true)); options.addOption(newOption("nr", "num_reducers", true, String.format("Number of reducers to use (defaults to %s)", NUM_REDUCERS_DEFAULT))); options.addOption(newOption("rpf", "records_per_file", true, String.format("Number of records to include in each output pcap file (defaults to %s)", NUM_RECORDS_PER_FILE_DEFAULT))); @@ -73,9 +74,9 @@ public void parse(CommandLine commandLine, PcapConfig config) throws java.text.P config.setBasePath(BASE_PATH_DEFAULT); } if (commandLine.hasOption("base_output_path")) { - config.setInterimResultPath(commandLine.getOptionValue("base_output_path")); + config.setBaseInterimResultPath(commandLine.getOptionValue("base_output_path")); } else { - config.setInterimResultPath(BASE_OUTPUT_PATH_DEFAULT); + config.setBaseInterimResultPath(BASE_INTERIM_OUTPUT_PATH_DEFAULT); } if (commandLine.hasOption("start_time")) { try { diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java index ced57f6398..763f0c6f76 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/query/PcapCliTest.java @@ -92,7 +92,7 @@ public void runs_fixed_pcap_filter_job_with_default_argument_list() throws Excep }}; FixedPcapConfig config = new FixedPcapConfig(prefixStrategy); PcapOptions.BASE_PATH.put(config, CliParser.BASE_PATH_DEFAULT); - PcapOptions.BASE_INTERIM_RESULT_PATH.put(config, CliParser.BASE_OUTPUT_PATH_DEFAULT); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(config, CliParser.BASE_INTERIM_OUTPUT_PATH_DEFAULT); PcapOptions.FIELDS.put(config, query); PcapOptions.NUM_REDUCERS.put(config, 10); PcapOptions.START_TIME_MS.put(config, 500L); @@ -130,7 +130,6 @@ public void describeTo(Description description) { }; } - @Test public void runs_fixed_pcap_filter_job_with_full_argument_list_and_default_dateformat() throws Exception { String[] args = { @@ -241,7 +240,7 @@ public void runs_query_pcap_filter_job_with_default_argument_list() throws Excep String query = "some query string"; FixedPcapConfig config = new FixedPcapConfig(prefixStrategy); PcapOptions.BASE_PATH.put(config, CliParser.BASE_PATH_DEFAULT); - PcapOptions.BASE_INTERIM_RESULT_PATH.put(config, CliParser.BASE_OUTPUT_PATH_DEFAULT); + PcapOptions.BASE_INTERIM_RESULT_PATH.put(config, CliParser.BASE_INTERIM_OUTPUT_PATH_DEFAULT); PcapOptions.FIELDS.put(config, query); PcapOptions.NUM_REDUCERS.put(config, 10); PcapOptions.START_TIME_MS.put(config, 500L); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java index 3201ef8fbf..26509be726 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapConfig.java @@ -41,7 +41,7 @@ public PcapConfig(PrefixStrategy prefixStrategy) { this(); setShowHelp(false); setBasePath(""); - setInterimResultPath(""); + setBaseInterimResultPath(""); setStartTimeMs(-1L); setEndTimeMs(-1L); setNumReducers(0); @@ -77,7 +77,7 @@ public String getBasePath() { return PcapOptions.BASE_PATH.get(this, String.class); } - public String getInterimResultPath() { + public String getBaseInterimResultPath() { return PcapOptions.BASE_INTERIM_RESULT_PATH.get(this, String.class); } @@ -93,7 +93,7 @@ public void setBasePath(String basePath) { PcapOptions.BASE_PATH.put(this, basePath); } - public void setInterimResultPath(String baseOutputPath) { + public void setBaseInterimResultPath(String baseOutputPath) { PcapOptions.BASE_INTERIM_RESULT_PATH.put(this, baseOutputPath); } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java index 4630850be3..09effd4a1c 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java @@ -26,6 +26,7 @@ public enum PcapOptions implements ConfigOption { JOB_NAME("jobName"), FINAL_FILENAME_PREFIX("finalFilenamePrefix"), BASE_PATH("basePath", (s, o) -> o == null ? null : new Path(o.toString())), + INTERIM_RESULT_PATH("interimResultPath", (s, o) -> o == null ? null : new Path(o.toString())), BASE_INTERIM_RESULT_PATH("baseInterimResultPath", (s, o) -> o == null ? null : new Path(o.toString())), FINAL_OUTPUT_PATH("finalOutputPath", (s, o) -> o == null ? null : new Path(o.toString())), NUM_REDUCERS("numReducers"), diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java index cb54cffa64..a2fada9b3b 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java @@ -57,7 +57,7 @@ protected PcapResultsWriter getResultsWriter() { public Pageable finalizeJob(Map config) throws JobException { Configuration hadoopConfig = PcapOptions.HADOOP_CONF.get(config, Configuration.class); int recPerFile = PcapOptions.NUM_RECORDS_PER_FILE.get(config, Integer.class); - Path interimResultPath = PcapOptions.BASE_INTERIM_RESULT_PATH + Path interimResultPath = PcapOptions.INTERIM_RESULT_PATH .get(config, PcapOptions.STRING_TO_PATH, Path.class); FileSystem fs = PcapOptions.FILESYSTEM.get(config, FileSystem.class); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java index 36b78c4da7..05c494b2fd 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/mr/PcapJob.java @@ -262,7 +262,7 @@ public Statusable query(Optional jobName, LOG.debug("Executing query {} on timerange from {} to {}", filterImpl.queryToString(fields), from, to); } Path interimResultPath = new Path(baseInterimResultPath, outputDirName); - PcapOptions.BASE_INTERIM_RESULT_PATH.put(configuration, interimResultPath); + PcapOptions.INTERIM_RESULT_PATH.put(configuration, interimResultPath); mrJob = createJob(jobName , basePath , interimResultPath From 374fe2e4943f1605e2bdcbf756f36ba682453c44 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Wed, 18 Jul 2018 13:32:26 -0600 Subject: [PATCH 35/36] Fix config option in pcap integration test. Add finalizer comments per cestella request. --- .../pcap/integration/PcapTopologyIntegrationTest.java | 5 +++-- .../apache/metron/pcap/finalizer/PcapCliFinalizer.java | 10 +++++++++- .../apache/metron/pcap/finalizer/PcapFinalizer.java | 6 ++++++ .../metron/pcap/finalizer/PcapFinalizerStrategies.java | 5 +++++ .../metron/pcap/finalizer/PcapRestFinalizer.java | 10 +++++++++- 5 files changed, 32 insertions(+), 4 deletions(-) diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java index 78fc9669ff..9ea7912d51 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java @@ -62,11 +62,13 @@ import org.apache.metron.pcap.PacketInfo; import org.apache.metron.pcap.PcapHelper; import org.apache.metron.pcap.PcapMerger; +import org.apache.metron.pcap.config.FixedPcapConfig; import org.apache.metron.pcap.config.PcapOptions; import org.apache.metron.pcap.filter.fixed.FixedPcapFilter; import org.apache.metron.pcap.filter.query.QueryPcapFilter; import org.apache.metron.pcap.finalizer.PcapFinalizerStrategies; import org.apache.metron.pcap.mr.PcapJob; +import org.apache.metron.pcap.query.PcapCli; import org.apache.metron.spout.pcap.Endianness; import org.apache.metron.spout.pcap.deserializer.Deserializers; import org.apache.metron.test.utils.UnitTestHelper; @@ -249,8 +251,7 @@ public ProcessorResult getResult() { } }); - - Map configuration = new HashMap<>(); + FixedPcapConfig configuration = new FixedPcapConfig(PcapCli.PREFIX_STRATEGY); Configuration hadoopConf = new Configuration(); PcapOptions.JOB_NAME.put(configuration, "jobName"); PcapOptions.HADOOP_CONF.put(configuration, hadoopConf); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java index ecdce4ecbf..e0321582e1 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapCliFinalizer.java @@ -27,11 +27,19 @@ */ public class PcapCliFinalizer extends PcapFinalizer { + /** + * Format will have the format <output-path>/pcap-data-<filename-prefix>+<partition-num>.pcap + * The filename prefix is pluggable, but in most cases it will be provided via the PcapConfig + * as a formatted timestamp + uuid. A final sample format will look as follows: + * /base/output/path/pcap-data-201807181911-09855b4ae3204dee8b63760d65198da3+0001.pcap + */ + private static final String PCAP_CLI_FILENAME_FORMAT = "%s/pcap-data-%s+%04d.pcap"; + @Override protected String getOutputFileName(Map config, int partition) { Path finalOutputPath = PcapOptions.FINAL_OUTPUT_PATH.get(config, PcapOptions.STRING_TO_PATH, Path.class); String prefix = PcapOptions.FINAL_FILENAME_PREFIX.get(config, String.class); - return String.format("%s/pcap-data-%s+%04d.pcap", finalOutputPath, prefix, partition); + return String.format(PCAP_CLI_FILENAME_FORMAT, finalOutputPath, prefix, partition); } } diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java index a2fada9b3b..d5ac67571f 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizer.java @@ -40,6 +40,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * Takes Pcap results from a specified path - for PCAP, it is assumed that these results are SequenceFileIterables. + * The results are then processed by partitioning the results based on a num records per file option + * into a final output file with a PCAP header for each partition, and written to a final output location. + * The MapReduce results are cleaned up after successfully writing out the final results. + */ public abstract class PcapFinalizer implements Finalizer { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizerStrategies.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizerStrategies.java index 8059036828..927d602838 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizerStrategies.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapFinalizerStrategies.java @@ -24,6 +24,11 @@ import org.apache.metron.job.JobException; import org.apache.metron.job.Pageable; +/** + * PcapJob runs a MapReduce job that outputs Sequence Files to HDFS. This Strategy/Factory class + * provides options for doing final processing on this raw MapReduce output for the CLI and REST + * API's. + */ public enum PcapFinalizerStrategies implements Finalizer { CLI(new PcapCliFinalizer()), REST(new PcapRestFinalizer()); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java index 665b103370..059bba2150 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java @@ -27,10 +27,18 @@ */ public class PcapRestFinalizer extends PcapFinalizer { + /** + * Format will have the format <output-path>/page-<page-num>.pcap + * The filename prefix is pluggable, but in most cases it will be provided via the PcapConfig + * as a formatted timestamp + uuid. A final sample format will look as follows: + * /base/output/path/pcap-data-201807181911-09855b4ae3204dee8b63760d65198da3+0001.pcap + */ + private static final String PCAP_CLI_FILENAME_FORMAT = "%s/page-%s.pcap"; + @Override protected String getOutputFileName(Map config, int partition) { Path finalOutputPath = PcapOptions.FINAL_OUTPUT_PATH.getTransformed(config, Path.class); - return String.format("%s/page-%s", finalOutputPath, partition); + return String.format(PCAP_CLI_FILENAME_FORMAT, finalOutputPath, partition); } } From 38edfd7484c10096627b0d3eea6042678eea9b08 Mon Sep 17 00:00:00 2001 From: merrimanr Date: Wed, 18 Jul 2018 15:22:02 -0500 Subject: [PATCH 36/36] added username to pcap options --- .../org/apache/metron/rest/config/PcapJobSupplier.java | 8 ++------ .../apache/metron/rest/service/impl/PcapServiceImpl.java | 8 +++----- .../java/org/apache/metron/pcap/config/PcapOptions.java | 1 + .../apache/metron/pcap/finalizer/PcapRestFinalizer.java | 8 ++------ 4 files changed, 8 insertions(+), 17 deletions(-) diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapJobSupplier.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapJobSupplier.java index 76f4e01213..1e79f6af0e 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapJobSupplier.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/PcapJobSupplier.java @@ -22,6 +22,7 @@ import org.apache.metron.job.JobException; import org.apache.metron.job.RuntimeJobException; import org.apache.metron.job.Statusable; +import org.apache.metron.pcap.finalizer.PcapFinalizerStrategies; import org.apache.metron.pcap.finalizer.PcapRestFinalizer; import org.apache.metron.pcap.mr.PcapJob; import org.apache.metron.rest.model.pcap.PcapRequest; @@ -31,13 +32,12 @@ public class PcapJobSupplier implements Supplier> { private PcapRequest pcapRequest; - private Finalizer finalizer; @Override public Statusable get() { try { PcapJob pcapJob = createPcapJob(); - return pcapJob.submit(finalizer, pcapRequest); + return pcapJob.submit(PcapFinalizerStrategies.REST, pcapRequest); } catch (JobException e) { throw new RuntimeJobException(e.getMessage()); } @@ -47,10 +47,6 @@ public void setPcapRequest(PcapRequest pcapRequest) { this.pcapRequest = pcapRequest; } - public void setFinalizer(PcapRestFinalizer pcapRestFinalizer) { - this.finalizer = pcapRestFinalizer; - } - protected PcapJob createPcapJob() { return new PcapJob(); } diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java index 2b228a43f2..218e9bed9d 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/PcapServiceImpl.java @@ -60,12 +60,9 @@ public PcapServiceImpl(Environment environment, Configuration configuration, Pca @Override public PcapStatus fixed(String username, FixedPcapRequest fixedPcapRequest) throws RestException { try { - setPcapOptions(fixedPcapRequest); + setPcapOptions(username, fixedPcapRequest); fixedPcapRequest.setFields(); pcapJobSupplier.setPcapRequest(fixedPcapRequest); - PcapRestFinalizer pcapRestFinalizer = new PcapRestFinalizer(); - pcapRestFinalizer.setUser(username); - pcapJobSupplier.setFinalizer(pcapRestFinalizer); JobStatus jobStatus = jobManager.submit(pcapJobSupplier, username); return jobStatusToPcapStatus(jobStatus); } catch (IOException | JobException e) { @@ -95,8 +92,9 @@ public PcapStatus getJobStatus(String username, String jobId) throws RestExcepti return pcapStatus; } - protected void setPcapOptions(PcapRequest pcapRequest) throws IOException { + protected void setPcapOptions(String username, PcapRequest pcapRequest) throws IOException { PcapOptions.JOB_NAME.put(pcapRequest, "jobName"); + PcapOptions.USERNAME.put(pcapRequest, username); PcapOptions.HADOOP_CONF.put(pcapRequest, configuration); PcapOptions.FILESYSTEM.put(pcapRequest, getFileSystem()); diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java index 1d18e730c3..3d7c4f61c8 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/config/PcapOptions.java @@ -25,6 +25,7 @@ public enum PcapOptions implements ConfigOption { JOB_NAME("jobName"), JOB_ID("jobId"), + USERNAME("username"), FINAL_FILENAME_PREFIX("finalFilenamePrefix"), BASE_PATH("basePath", (s, o) -> o == null ? null : new Path(o.toString())), INTERIM_RESULT_PATH("interimResultPath", (s, o) -> o == null ? null : new Path(o.toString())), diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java index 5879fb202b..95907df8a3 100644 --- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java +++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java @@ -31,17 +31,13 @@ public class PcapRestFinalizer extends PcapFinalizer { private static final String PCAP_REST_FILEPATH_FORMAT = "%s/%s/%s/%s/page-%s.pcap"; - private String user; private String jobType = Statusable.JobType.MAP_REDUCE.name(); - public void setUser(String user) { - this.user = user; - } - @Override protected Path getOutputPath(Map config, int partition) { - String jobId = PcapOptions.JOB_ID.get(config, String.class); String finalOutputPath = PcapOptions.FINAL_OUTPUT_PATH.get(config, String.class); + String user = PcapOptions.USERNAME.get(config, String.class); + String jobId = PcapOptions.JOB_ID.get(config, String.class); return new Path(String.format(PCAP_REST_FILEPATH_FORMAT, finalOutputPath, user, jobType, jobId, partition)); }