You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@metron.apache.org by merrimanr <gi...@git.apache.org> on 2018/07/16 21:45:40 UTC

[GitHub] metron pull request #1109: METRON-1674: Create REST endpoint for job status ...

GitHub user merrimanr opened a pull request:

    https://github.com/apache/metron/pull/1109

    METRON-1674: Create REST endpoint for job status abstraction

    ## Contributor Comments
    This PR is built on top of https://github.com/apache/metron/pull/1108 and should be merged and final review done after that is accepted into the feature branch.  This exposes the job manager and job status abstraction through a REST endpoint.
    
    A summary of the changes included:
    
    - Adjustments to the existing PcapService to accommodate the new JobManager abstraction
      - JobManager is now a Spring bean and job submission/status is done through that
      - New properties were added to application.yml and PcapServiceImpl
      - PcapJobSupplier was added that allows switching to a mock pcap job during integration testing
      - Unit and integration tests were adjusted to match
      - Time parameters are now `startTimeMs` and `endTimeMs`
    - FixedPcapRequest now matches the pattern used in PcapRequest
    - Addition of a PcapStatus object to return status in a simple, consumable structure for the UI
    - Endpoint to get job status was added
    - ConfigOption was adjusted to automatically handle type conversions in cases where Jackson deserialization is used (Integer to Long for example)
    - InMemoryJobManager now throws a JobNotFoundException when jobs don't exist for a username/job id combination
    - PcapJob will automatically convert PcapOptions.START_TIME_MS to PcapOptions.START_TIME_NS when START_TIME_NS is not set (same goes for END_TIME_NS)
    - New unit and integration tests added for get status endpoint and service
    
    This has been lightly tested in full dev.  The HDFS paths must be created manually for the paths specified in application.yml.  For now and only job submission and subsequent status retrieval had been tested.  
    
    This PR should strictly be used for testing at this point.  To test in full dev, create the HDFS directories mentioned above and put pcap data in `/apps/metron/pcap/input`.  Submit a fixed pcap query:
    ```
    curl -X POST --header 'Content-Type: application/json' --header 'Accept: application/json' -d '{
      "endTime": 1458240269424,
      "startTime": 1458240269419
    }' 'http://node1:8082/api/v1/pcap/fixed'
    ```
    A job id should be returned in the response:
    ```
    {
      "jobId": "job_1531258337010_0021",
      "jobStatus": "RUNNING",
      "description": "map: 0.0%, reduce: 0.0%",
      "percentComplete": 0,
      "size": 0
    }
    ```
    Job status can now be retrieved using the get job status endpoint:
    ```
    curl -X GET --header 'Accept: application/json' 'http://node1:8082/api/v1/pcap/job_1531258337010_0021'
    ```
    
    A full, comprehensive review should be done after https://github.com/apache/metron/pull/1108 has been merged.
    
    ## Pull Request Checklist
    
    Thank you for submitting a contribution to Apache Metron.  
    Please refer to our [Development Guidelines](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=61332235) for the complete guide to follow for contributions.  
    Please refer also to our [Build Verification Guidelines](https://cwiki.apache.org/confluence/display/METRON/Verifying+Builds?show-miniview) for complete smoke testing guides.  
    
    
    In order to streamline the review of the contribution we ask you follow these guidelines and ask you to double check the following:
    
    ### For all changes:
    - [x] Is there a JIRA ticket associated with this PR? If not one needs to be created at [Metron Jira](https://issues.apache.org/jira/browse/METRON/?selectedTab=com.atlassian.jira.jira-projects-plugin:summary-panel).
    - [x] Does your PR title start with METRON-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
    - [x] Has your PR been rebased against the latest commit within the target branch (typically master)?
    
    
    ### For code changes:
    - [ ] Have you included steps to reproduce the behavior or problem that is being changed or addressed?
    - [ ] Have you included steps or a guide to how the change may be verified and tested manually?
    - [ ] Have you ensured that the full suite of tests and checks have been executed in the root metron folder via:
      ```
      mvn -q clean integration-test install && dev-utilities/build-utils/verify_licenses.sh 
      ```
    
    - [x] Have you written or updated unit tests and or integration tests to verify your changes?
    - [x] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
    - [x] Have you verified the basic functionality of the build by building and running locally with Vagrant full-dev environment or the equivalent?
    
    ### For documentation related changes:
    - [ ] Have you ensured that format looks appropriate for the output in which it is rendered by building and verifying the site-book? If not then run the following commands and the verify changes via `site-book/target/site/index.html`:
    
      ```
      cd site-book
      mvn site
      ```
    
    #### Note:
    Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.
    It is also recommended that [travis-ci](https://travis-ci.org) is set up for your personal repository such that your branches are built there before submitting a pull request.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/merrimanr/incubator-metron pcap-job-manager

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/metron/pull/1109.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1109
    
----
commit 41ecf36cd7c2da0399d03a37fee5916a9bfa87e7
Author: Michael Miklavcic <mi...@...>
Date:   2018-06-13T01:48:41Z

    Add metron-job project. Update pcap to be Statusable.

commit a83b472778e8e40cd81132f86402873ab4b8b2b1
Author: Michael Miklavcic <mi...@...>
Date:   2018-06-16T23:10:49Z

    Save progress on Pageable implementation

commit 976e1bcdf9619cc0672de4bcd7dd0d865a01cc1b
Author: Michael Miklavcic <mi...@...>
Date:   2018-06-25T19:00:03Z

    Rev metron-job to 0.5.1 after merge with master

commit f9fc106f4a33f323dc06260d861fc543726e2518
Author: Michael Miklavcic <mi...@...>
Date:   2018-06-26T22:03:21Z

    Move result writing code to pcapjob. Get pcapclitest working again.

commit 8b4ef9cdc37235c333f3cbd294f20ae15a7438aa
Author: Michael Miklavcic <mi...@...>
Date:   2018-06-27T05:31:21Z

    Add Pageable results.

commit cd302a989320a6a8970b18a94df8fbf4f11576b1
Author: Michael Miklavcic <mi...@...>
Date:   2018-06-27T15:12:42Z

    change pom change with spaces to tabs

commit 7c1d4a0bedb23e86e323b0daec21c73113065360
Author: Michael Miklavcic <mi...@...>
Date:   2018-06-27T23:20:16Z

    Address review comments. Fix local FS write path problem.

commit a00e300d8bd812589fd09970ce4db1f25dc29165
Author: Michael Miklavcic <mi...@...>
Date:   2018-06-28T03:31:02Z

    Let's try this again.

commit ba6cf9030f45b6b1a3cf270a376e4b1f5e67d6fb
Author: Michael Miklavcic <mi...@...>
Date:   2018-06-29T12:14:44Z

    Job manager and job service implementations

commit 157aa1e3715d77373089a5c3eacfcc1dcd8d0a08
Author: Michael Miklavcic <mi...@...>
Date:   2018-07-05T17:58:55Z

    Refactor jobs to have a finalize step

commit 9fb3a81c144cab23f81de94007e85e33552fe10c
Author: Michael Miklavcic <mi...@...>
Date:   2018-07-10T16:57:05Z

    Prototyping rest api bits

commit 2bca32ed87dac15538263bdfe15ceba3d3ec3b79
Author: Michael Miklavcic <mi...@...>
Date:   2018-07-11T02:22:01Z

    merge with feature branch

commit 36456d637d9cd13a13ef34735f74d06d350772cf
Author: Michael Miklavcic <mi...@...>
Date:   2018-07-12T11:52:42Z

    Redo the pcap asychronicity  to use a Timer for checking status. Pass in config as Map

commit 6e0b64ce1c9b1c3ca1b7a6e2b58ba74c188d702a
Author: cstella <ce...@...>
Date:   2018-07-12T17:18:09Z

    Refactoring the config options to translate between object and map.

commit 0aa8e6eb967c6163c88b02c42074ced96243a7c3
Author: cstella <ce...@...>
Date:   2018-07-12T18:10:09Z

    Redoing some stuff

commit 9c6eaea48b7dc5b2cf1d6e7b7373258afbcda965
Author: cstella <ce...@...>
Date:   2018-07-12T18:15:38Z

    Removed some duplication

commit e52c5f00b9a720d7895180d36bca7285c0d0d335
Author: cstella <ce...@...>
Date:   2018-07-12T18:17:20Z

    forgot licenses

commit d4faf56078cca6b259d836b852979f6b38447399
Author: Michael Miklavcic <mi...@...>
Date:   2018-07-12T18:20:35Z

    Merge pull request #24 from cestella/pcap-job-manager-generics
    
    Pcap job manager generics from cestella

commit 7971ecba71b811b23187e70d8f41e9ec89c8da18
Author: merrimanr <me...@...>
Date:   2018-07-12T21:51:24Z

    initial commit

commit 720dab608ffcf3cb40cabc79ec69e98541b709ee
Author: Michael Miklavcic <mi...@...>
Date:   2018-07-13T19:10:49Z

    Move pcap config to metron-pcap module. Remove metron-api module completely

commit 69cb9640fab4cb645e62923565f71c37f083da9d
Author: Michael Miklavcic <mi...@...>
Date:   2018-07-13T19:21:21Z

    Move pcapoptions to config package

commit 0615ae60f7542a182f8dbd31cc07ede6b038be43
Author: Michael Miklavcic <mi...@...>
Date:   2018-07-13T23:12:23Z

    Revert PcapServiceImpl

commit 433d0a37501ae0bdc8b3e5a7b0fc3ceb9c125ebf
Author: Michael Miklavcic <mi...@...>
Date:   2018-07-14T04:02:03Z

    Get job manager working. Fix pcap cli tests. Make pcap pageable better for synchronicity. Kill remnants of the job service.

commit 935dfc84f9f8556c5fbde029561dec258c82c577
Author: merrimanr <me...@...>
Date:   2018-07-16T14:42:05Z

    Merge branch 'pcap-job-manager' of https://github.com/mmiklavc/metron into pcap-job-manager

commit 71f387160dad1d9ad53b94462ea437d29ee11323
Author: merrimanr <me...@...>
Date:   2018-07-16T14:49:25Z

    updated pcap integration test

commit bcb2af02a1718b12da75b78c192a840c12bb4789
Author: merrimanr <me...@...>
Date:   2018-07-16T21:23:15Z

    rest endpoint for job status

----


---

[GitHub] metron pull request #1109: METRON-1674: Create REST endpoint for job status ...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1109#discussion_r203047956
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java ---
    @@ -29,6 +29,16 @@
     
     public class HDFSUtils {
     
    +  public static byte[] readBytes(String path) throws IOException {
    +    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);
    --- End diff --
    
    This should be in a try with resources block otherwise you're leaving streams open.


---

[GitHub] metron pull request #1109: METRON-1674: Create REST endpoint for job status ...

Posted by merrimanr <gi...@git.apache.org>.
Github user merrimanr commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1109#discussion_r203370606
  
    --- Diff: 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<Statusable<Path>> {
    +
    +  private PcapRequest pcapRequest;
    +
    +  @Override
    +  public Statusable<Path> get() {
    +    try {
    +      PcapJob<Path> 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());
    --- End diff --
    
    Yes this was a mistake.  A runtime exception is thrown now.


---

[GitHub] metron pull request #1109: METRON-1674: Create REST endpoint for job status ...

Posted by merrimanr <gi...@git.apache.org>.
Github user merrimanr commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1109#discussion_r203518190
  
    --- Diff: metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java ---
    @@ -0,0 +1,46 @@
    +/**
    + * 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 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;
    +
    +/**
    + * Write to HDFS.
    + */
    +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 Path getOutputPath(Map<String, Object> config, int partition) {
    +    String jobId = PcapOptions.JOB_ID.get(config, String.class);
    +    String finalOutputPath = PcapOptions.FINAL_OUTPUT_PATH.get(config, String.class);
    --- End diff --
    
    Done


---

[GitHub] metron pull request #1109: METRON-1674: Create REST endpoint for job status ...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1109#discussion_r203046824
  
    --- Diff: metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java ---
    @@ -17,75 +17,65 @@
      */
     package org.apache.metron.rest.model.pcap;
     
    -public class PcapRequest {
    +import org.apache.commons.collections4.map.AbstractMapDecorator;
    +import org.apache.metron.pcap.config.PcapOptions;
     
    -  private String baseOutputPath;
    -  private String basePath;
    -  private Long startTime = 0L;
    -  private Long endTime = System.currentTimeMillis();
    -  private Integer numReducers = 1;
    +import java.util.HashMap;
     
    -  public String getBaseOutputPath() {
    -    return baseOutputPath;
    -  }
    +public class PcapRequest extends AbstractMapDecorator<String, Object> {
     
    -  public void setBaseOutputPath(String baseOutputPath) {
    -    this.baseOutputPath = baseOutputPath;
    +  public PcapRequest() {
    +    super(new HashMap<>());
    +    setStartTimeMs(0L);
    +    setEndTimeMs(System.currentTimeMillis());
    +    setNumReducers(1);
    --- End diff --
    
    It might be worthwhile to pick the number of reducers > 1 just because this is going to be slow by default.  Maybe 10 would be a sensible middle ground?


---

[GitHub] metron pull request #1109: METRON-1674: Create REST endpoint for job status ...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1109#discussion_r203047410
  
    --- Diff: 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<Statusable<Path>> {
    +
    +  private PcapRequest pcapRequest;
    +
    +  @Override
    +  public Statusable<Path> get() {
    +    try {
    +      PcapJob<Path> 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());
    --- End diff --
    
    Is this intended?  Seems like we should be passing back info about the exception or at the very least throwing a runtime exception, right?


---

[GitHub] metron pull request #1109: METRON-1674: Create REST endpoint for job status ...

Posted by mmiklavc <gi...@git.apache.org>.
Github user mmiklavc commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1109#discussion_r203515166
  
    --- Diff: metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java ---
    @@ -0,0 +1,46 @@
    +/**
    + * 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 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;
    +
    +/**
    + * Write to HDFS.
    + */
    +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 Path getOutputPath(Map<String, Object> config, int partition) {
    +    String jobId = PcapOptions.JOB_ID.get(config, String.class);
    +    String finalOutputPath = PcapOptions.FINAL_OUTPUT_PATH.get(config, String.class);
    --- End diff --
    
    @merrimanr Yeah, let's do what we did with the interim output path. We've gone down the Map config path so I think we should keep that consistent. There may be some brainstorming we can do around creating pointed option validation (e.g. specific to a finalizer), but for now the PcapOptions route seems sufficient to me.


---

[GitHub] metron pull request #1109: METRON-1674: Create REST endpoint for job status ...

Posted by merrimanr <gi...@git.apache.org>.
Github user merrimanr commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1109#discussion_r203511113
  
    --- Diff: metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java ---
    @@ -0,0 +1,46 @@
    +/**
    + * 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 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;
    +
    +/**
    + * Write to HDFS.
    + */
    +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 Path getOutputPath(Map<String, Object> config, int partition) {
    +    String jobId = PcapOptions.JOB_ID.get(config, String.class);
    +    String finalOutputPath = PcapOptions.FINAL_OUTPUT_PATH.get(config, String.class);
    --- End diff --
    
    The reason I did it that way is because that user property is only used here.  Happy to change it to a config.


---

[GitHub] metron issue #1109: METRON-1674: Create REST endpoint for job status abstrac...

Posted by tiborm <gi...@git.apache.org>.
Github user tiborm commented on the issue:

    https://github.com/apache/metron/pull/1109
  
    Is it necessary to have /{user}/{jobType}/ in the output url?
    Right now our UI calling this:
    /apps/metron/pcap/output/{jobId}/{pageId}
    
    On our side, pageId is a simple int at the moment. Is "page-*" at the end of the URL pattern means the service expecting something like "page-5"?


---

[GitHub] metron issue #1109: METRON-1674: Create REST endpoint for job status abstrac...

Posted by james-sirota <gi...@git.apache.org>.
Github user james-sirota commented on the issue:

    https://github.com/apache/metron/pull/1109
  
    +1 works great. I was able to put the output of the pcap topology into a directory in HDFS, run a query over it, got a page in the output directory (with status correctly updating while the job was running), was able to open the page in wireshark 


---

[GitHub] metron issue #1109: METRON-1674: Create REST endpoint for job status abstrac...

Posted by merrimanr <gi...@git.apache.org>.
Github user merrimanr commented on the issue:

    https://github.com/apache/metron/pull/1109
  
    The latest commit addresses feedback received so far.  I also changed the structure of the HDFS paths to avoid issues around reading input and also to output data to unique locations.  Previously all data was written to `/apps/metron/pcap/output/page-*` which obviously won't work.  I will update the PR description with new testing instructions.
    
    @james-sirota this should resolve the error you were getting.  


---

[GitHub] metron issue #1109: METRON-1674: Create REST endpoint for job status abstrac...

Posted by merrimanr <gi...@git.apache.org>.
Github user merrimanr commented on the issue:

    https://github.com/apache/metron/pull/1109
  
    @tiborm That is an HDFS path, not a URL.  It's of no concern to the UI.


---

[GitHub] metron pull request #1109: METRON-1674: Create REST endpoint for job status ...

Posted by merrimanr <gi...@git.apache.org>.
Github user merrimanr commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1109#discussion_r203370393
  
    --- Diff: metron-interface/metron-rest-client/src/main/java/org/apache/metron/rest/model/pcap/PcapRequest.java ---
    @@ -17,75 +17,65 @@
      */
     package org.apache.metron.rest.model.pcap;
     
    -public class PcapRequest {
    +import org.apache.commons.collections4.map.AbstractMapDecorator;
    +import org.apache.metron.pcap.config.PcapOptions;
     
    -  private String baseOutputPath;
    -  private String basePath;
    -  private Long startTime = 0L;
    -  private Long endTime = System.currentTimeMillis();
    -  private Integer numReducers = 1;
    +import java.util.HashMap;
     
    -  public String getBaseOutputPath() {
    -    return baseOutputPath;
    -  }
    +public class PcapRequest extends AbstractMapDecorator<String, Object> {
     
    -  public void setBaseOutputPath(String baseOutputPath) {
    -    this.baseOutputPath = baseOutputPath;
    +  public PcapRequest() {
    +    super(new HashMap<>());
    +    setStartTimeMs(0L);
    +    setEndTimeMs(System.currentTimeMillis());
    +    setNumReducers(1);
    --- End diff --
    
    Default is now 10.


---

[GitHub] metron pull request #1109: METRON-1674: Create REST endpoint for job status ...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1109#discussion_r203502396
  
    --- Diff: metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/finalizer/PcapRestFinalizer.java ---
    @@ -0,0 +1,46 @@
    +/**
    + * 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 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;
    +
    +/**
    + * Write to HDFS.
    + */
    +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 Path getOutputPath(Map<String, Object> config, int partition) {
    +    String jobId = PcapOptions.JOB_ID.get(config, String.class);
    +    String finalOutputPath = PcapOptions.FINAL_OUTPUT_PATH.get(config, String.class);
    --- End diff --
    
    Why isn't the user passed into the config rather than being in a member variable?  This seems inconsistent.  I'd expect a PcapOptions.USERNAME.get(config, String.class) rather than a member variable.


---

[GitHub] metron issue #1109: METRON-1674: Create REST endpoint for job status abstrac...

Posted by tiborm <gi...@git.apache.org>.
Github user tiborm commented on the issue:

    https://github.com/apache/metron/pull/1109
  
    I see, thanks. Is there an endpoint for the PDML output?
    Or it is out of the scope of this PR?


---

[GitHub] metron pull request #1109: METRON-1674: Create REST endpoint for job status ...

Posted by merrimanr <gi...@git.apache.org>.
Github user merrimanr closed the pull request at:

    https://github.com/apache/metron/pull/1109


---

[GitHub] metron issue #1109: METRON-1674: Create REST endpoint for job status abstrac...

Posted by merrimanr <gi...@git.apache.org>.
Github user merrimanr commented on the issue:

    https://github.com/apache/metron/pull/1109
  
    That PR should be up for review soon.


---

[GitHub] metron issue #1109: METRON-1674: Create REST endpoint for job status abstrac...

Posted by james-sirota <gi...@git.apache.org>.
Github user james-sirota commented on the issue:

    https://github.com/apache/metron/pull/1109
  
    Here is what I've done so far:
    
    Capture a random PCAP using tcpdump, place that PCAP into /apps/metron/pcap/input on HDFS, run the service in Swager under POST /api/v1/pcap/fixed
    
    I send 
    
    {
      "endTime":1531892977000,
      "startTime":1458240269424
    }
    
    which are the right timestamps for my PCAP, and get back an error:
    
    {
      "timestamp": "2018-07-18 06:00:45",
      "status": 500,
      "error": "Internal Server Error",
      "message": "-1",
      "path": "/api/v1/pcap/fixed"
    }
    
    Which in metron-rest.log corresponds to:
    
    Jul 18, 2018 6:00:45 AM org.apache.catalina.core.StandardWrapperValve invoke
    SEVERE: Servlet.service() for servlet [dispatcherServlet] in context with path [] threw exception [Request processing failed; nested exception is java.lang.ArrayIndexOutOfBoundsException: -1] with root cause
    java.lang.ArrayIndexOutOfBoundsException: -1
            at org.apache.metron.pcap.PcapFilenameHelper.getKafkaPartition(PcapFilenameHelper.java:66)
            at org.apache.metron.pcap.utils.FileFilterUtil.getFilesByPartition(FileFilterUtil.java:63)
            at org.apache.metron.pcap.utils.FileFilterUtil.getPathsInTimeRange(FileFilterUtil.java:50)
            at org.apache.metron.pcap.mr.PcapJob.createJob(PcapJob.java:382)
            at org.apache.metron.pcap.mr.PcapJob.query(PcapJob.java:276)
            at org.apache.metron.pcap.mr.PcapJob.submit(PcapJob.java:235)
            at org.apache.metron.rest.config.PcapJobSupplier.get(PcapJobSupplier.java:38)
            at org.apache.metron.rest.config.PcapJobSupplier.get(PcapJobSupplier.java:30)
            at org.apache.metron.job.manager.InMemoryJobManager.submit(InMemoryJobManager.java:48)
            at org.apache.metron.rest.service.impl.PcapServiceImpl.fixed(PcapServiceImpl.java:64)
            at org.apache.metron.rest.controller.PcapController.fixed(PcapController.java:58)
            at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
            at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
            at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
            at java.lang.reflect.Method.invoke(Method.java:498)
            at org.springframework.web.method.support.InvocableHandlerMethod.doInvoke(InvocableHandlerMethod.java:209)
            at org.springframework.web.method.support.InvocableHandlerMethod.invokeForRequest(InvocableHandlerMethod.java:136)


---

[GitHub] metron pull request #1109: METRON-1674: Create REST endpoint for job status ...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1109#discussion_r203049598
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HDFSUtils.java ---
    @@ -29,6 +29,16 @@
     
     public class HDFSUtils {
     
    +  public static byte[] readBytes(String path) throws IOException {
    +    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);
    --- End diff --
    
    I retract this, it originated in @mmiklavc 's PR.  I moved the comment there.


---