You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pirk.apache.org by ellisonanne <gi...@git.apache.org> on 2016/08/22 21:35:48 UTC

[GitHub] incubator-pirk pull request #76: [PIRK-21] [WIP] - Initial Spark Streaming R...

GitHub user ellisonanne opened a pull request:

    https://github.com/apache/incubator-pirk/pull/76

    [PIRK-21] [WIP] - Initial Spark Streaming Responder Implementation

    The initial Spark Streaming Responder implementation, including corresponding distributed tests. Initial input adaptors only include HDFS and Elasticsearch; additional adaptors (Kafka, etc) will be added as subsequent JIRAs. 
    
    Marking this PR as a WIP until scale testing is completed; functional testing via new streaming tests within the DistributedTestSuite is complete.
    
    Comments/suggestions welcome!

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

    $ git pull https://github.com/ellisonanne/incubator-pirk pirk-21

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

    https://github.com/apache/incubator-pirk/pull/76.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 #76
    
----
commit 74a6f11a1af491cc3b722af3fbb25b9342025b64
Author: ellisonanne <ea...@apache.org>
Date:   2016-07-29T19:45:21Z

    Merge pull request #5 from apache/master
    
    up

commit 42f3e8400f810377a4bae06638fc13c0e37707f3
Author: ellisonanne <ea...@apache.org>
Date:   2016-07-31T17:47:15Z

    Merge pull request #6 from apache/master
    
    updating fork

commit 887358883d26940845a3486b0dc87cd0cc6c3807
Author: ellisonanne <ea...@apache.org>
Date:   2016-08-01T13:06:35Z

    Merge pull request #7 from apache/master
    
    up

commit 2ad5c619084af2517cd135374f93be20410cc45c
Author: ellisonanne <ea...@apache.org>
Date:   2016-08-02T16:22:53Z

    Merge pull request #8 from apache/master
    
    updating fork

commit d3c6eeb37a0300ea882c0bb2bcd2939132e7e5cb
Author: Ellison Anne Williams <ea...@apache.org>
Date:   2016-08-06T14:23:24Z

    Merge pull request #9 from apache/master
    
    updating fork

commit 381f3529f97fa38b7b06be015c7b82471c8b95d5
Author: Ellison Anne Williams <ea...@apache.org>
Date:   2016-08-11T13:22:15Z

    Merge pull request #10 from apache/master
    
    updating fork

commit 6ab035dd32fc4e3464cfb377ccb3493ecf9a313d
Author: Ellison Anne Williams <ea...@apache.org>
Date:   2016-08-12T21:19:23Z

    Merge pull request #11 from apache/master
    
    up

commit 47a933f9c1d6c6c80f7b61ba5856a84af8162dca
Author: Ellison Anne Williams <ea...@apache.org>
Date:   2016-08-13T12:31:25Z

    Merge pull request #12 from apache/master
    
    up

commit 6ba5ed4b3ce4983f1e1aa958b6f55180613fa1a1
Author: Ellison Anne Williams <ea...@apache.org>
Date:   2016-08-16T13:30:13Z

    Merge pull request #13 from apache/master
    
    updating fork

commit 43818205074456b8e16aa09bb408706bfb9608ed
Author: Ellison Anne Williams <ea...@apache.org>
Date:   2016-08-18T17:38:55Z

    Merge pull request #14 from apache/master
    
    updating fork

commit 9da10044342663526b3456aa894f204f697a2866
Author: Ellison Anne Williams <ea...@apache.org>
Date:   2016-08-19T13:54:59Z

    Merge pull request #15 from apache/master
    
    updating fork

commit 195c518a8e4d0b131d47fbbe84f8183bdde164ec
Author: Ellison Anne Williams <ea...@apache.org>
Date:   2016-08-19T19:00:44Z

    Merge pull request #16 from apache/master
    
    various cleanup items - closes apache/incubator-pirk#73

commit b2cb424017a726ad1334818ee7b1f8d5cc6a6581
Author: eawilliams <ea...@apache.org>
Date:   2016-08-22T20:56:08Z

    initial spark streaming responder

commit d306c6e85b264064d52b3c34929347e4401970c8
Author: eawilliams <ea...@apache.org>
Date:   2016-08-22T21:28:08Z

    updating DistTestSuite

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-pirk pull request #76: [PIRK-21] [WIP] - Initial Spark Streaming R...

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

    https://github.com/apache/incubator-pirk/pull/76#discussion_r75841051
  
    --- Diff: src/main/java/org/apache/pirk/responder/wideskies/spark/streaming/ComputeStreamingResponse.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.pirk.responder.wideskies.spark.streaming;
    +
    +import java.math.BigInteger;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Queue;
    +
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.io.MapWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapreduce.Job;
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
    +import org.apache.pirk.inputformat.hadoop.BaseInputFormat;
    +import org.apache.pirk.inputformat.hadoop.InputFormatConst;
    +import org.apache.pirk.query.wideskies.Query;
    +import org.apache.pirk.query.wideskies.QueryInfo;
    +import org.apache.pirk.responder.wideskies.spark.Accumulators;
    +import org.apache.pirk.responder.wideskies.spark.BroadcastVars;
    +import org.apache.pirk.responder.wideskies.spark.EncColMultGroupedMapper;
    +import org.apache.pirk.responder.wideskies.spark.EncColMultReducer;
    +import org.apache.pirk.responder.wideskies.spark.EncRowCalc;
    +import org.apache.pirk.responder.wideskies.spark.FilterData;
    +import org.apache.pirk.responder.wideskies.spark.HashSelectorsAndPartitionData;
    +import org.apache.pirk.schema.data.DataSchema;
    +import org.apache.pirk.schema.data.DataSchemaLoader;
    +import org.apache.pirk.schema.data.DataSchemaRegistry;
    +import org.apache.pirk.schema.query.QuerySchema;
    +import org.apache.pirk.schema.query.QuerySchemaLoader;
    +import org.apache.pirk.schema.query.QuerySchemaRegistry;
    +import org.apache.pirk.serialization.HadoopFileSystemStore;
    +import org.apache.pirk.utils.PIRException;
    +import org.apache.pirk.utils.SystemConfiguration;
    +import org.apache.spark.SparkConf;
    +import org.apache.spark.api.java.JavaPairRDD;
    +import org.apache.spark.api.java.JavaRDD;
    +import org.apache.spark.api.java.JavaSparkContext;
    +import org.apache.spark.api.java.function.Function;
    +import org.apache.spark.api.java.function.VoidFunction;
    +import org.apache.spark.streaming.Durations;
    +import org.apache.spark.streaming.api.java.JavaDStream;
    +import org.apache.spark.streaming.api.java.JavaPairDStream;
    +import org.apache.spark.streaming.api.java.JavaPairInputDStream;
    +import org.apache.spark.streaming.api.java.JavaStreamingContext;
    +import org.elasticsearch.hadoop.mr.EsInputFormat;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Master class for the PIR query spark streaming application
    + * <p>
    + * NOTE:
    + * <p>
    + * - NOT using Elasticsearch in practice - proved to be some speed issues with ES and Spark that appear to be ES-Spark specific - leave code in anticipation
    + * that the ES-Spark issues resolve...
    + * <p>
    + * - Even if rdd.count() calls are embedded in logger.debug statements, they are computed by Spark. Thus, they are commented out in the code below - uncomment
    + * for rdd.count() debug
    --- End diff --
    
    Surround with a ```logger.isDebugEnabled()``` check.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-pirk pull request #76: [PIRK-21] [WIP] - Initial Spark Streaming R...

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

    https://github.com/apache/incubator-pirk/pull/76#discussion_r75877912
  
    --- Diff: src/main/resources/pirk.properties ---
    @@ -28,7 +28,8 @@ local.pirk.properties.dir=/root/
     ##
     ##Spark path for SparkLauncher
     ##
    -spark.home = /usr
    +#spark.home = /usr
    +spark.home=/opt/cloudera/parcels/CDH/lib/spark
    --- End diff --
    
    Yes, this is for the CDH cluster that I am doing some of the testing on - I will remove it 
    
    (It is, however, an example of how to alternately set the spark.home property for different cluster configurations)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-pirk issue #76: [PIRK-21] - Initial Spark Streaming Responder Impl...

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

    https://github.com/apache/incubator-pirk/pull/76
  
    Merging now - will keep working on it via JIRA issues and PRs


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-pirk pull request #76: [PIRK-21] [WIP] - Initial Spark Streaming R...

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

    https://github.com/apache/incubator-pirk/pull/76#discussion_r75834938
  
    --- Diff: src/main/java/org/apache/pirk/responder/wideskies/spark/streaming/ComputeStreamingResponse.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.pirk.responder.wideskies.spark.streaming;
    +
    +import java.math.BigInteger;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Queue;
    +
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.io.MapWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapreduce.Job;
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
    +import org.apache.pirk.inputformat.hadoop.BaseInputFormat;
    +import org.apache.pirk.inputformat.hadoop.InputFormatConst;
    +import org.apache.pirk.query.wideskies.Query;
    +import org.apache.pirk.query.wideskies.QueryInfo;
    +import org.apache.pirk.responder.wideskies.spark.Accumulators;
    +import org.apache.pirk.responder.wideskies.spark.BroadcastVars;
    +import org.apache.pirk.responder.wideskies.spark.EncColMultGroupedMapper;
    +import org.apache.pirk.responder.wideskies.spark.EncColMultReducer;
    +import org.apache.pirk.responder.wideskies.spark.EncRowCalc;
    +import org.apache.pirk.responder.wideskies.spark.FilterData;
    +import org.apache.pirk.responder.wideskies.spark.HashSelectorsAndPartitionData;
    +import org.apache.pirk.schema.data.DataSchema;
    +import org.apache.pirk.schema.data.DataSchemaLoader;
    +import org.apache.pirk.schema.data.DataSchemaRegistry;
    +import org.apache.pirk.schema.query.QuerySchema;
    +import org.apache.pirk.schema.query.QuerySchemaLoader;
    +import org.apache.pirk.schema.query.QuerySchemaRegistry;
    +import org.apache.pirk.serialization.HadoopFileSystemStore;
    +import org.apache.pirk.utils.PIRException;
    +import org.apache.pirk.utils.SystemConfiguration;
    +import org.apache.spark.SparkConf;
    +import org.apache.spark.api.java.JavaPairRDD;
    +import org.apache.spark.api.java.JavaRDD;
    +import org.apache.spark.api.java.JavaSparkContext;
    +import org.apache.spark.api.java.function.Function;
    +import org.apache.spark.api.java.function.VoidFunction;
    +import org.apache.spark.streaming.Durations;
    +import org.apache.spark.streaming.api.java.JavaDStream;
    +import org.apache.spark.streaming.api.java.JavaPairDStream;
    +import org.apache.spark.streaming.api.java.JavaPairInputDStream;
    +import org.apache.spark.streaming.api.java.JavaStreamingContext;
    +import org.elasticsearch.hadoop.mr.EsInputFormat;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Master class for the PIR query spark streaming application
    + * <p>
    + * NOTE:
    + * <p>
    + * - NOT using Elasticsearch in practice - proved to be some speed issues with ES and Spark that appear to be ES-Spark specific - leave code in anticipation
    + * that the ES-Spark issues resolve...
    + * <p>
    + * - Even if rdd.count() calls are embedded in logger.debug statements, they are computed by Spark. Thus, they are commented out in the code below - uncomment
    + * for rdd.count() debug
    + * 
    + */
    +public class ComputeStreamingResponse
    +{
    +  private static final Logger logger = LoggerFactory.getLogger(ComputeStreamingResponse.class);
    +
    +  private String dataInputFormat = null;
    +  private String inputData = null;
    +  private String outputFile = null;
    +  private String outputDirExp = null;
    +
    +  private String queryInput = null;
    +  QuerySchema qSchema = null;
    +
    +  private String esQuery = "none";
    +  private String esResource = "none";
    +
    +  private FileSystem fs = null;
    +  private HadoopFileSystemStore storage = null;
    +  private JavaStreamingContext jssc = null;
    +
    +  boolean useQueueStream = false;
    +
    +  private long batchSeconds = 0;
    +  private long windowLength = 0;
    +
    +  private Accumulators accum = null;
    +  private BroadcastVars bVars = null;
    +
    +  private QueryInfo queryInfo = null;
    +  Query query = null;
    +
    +  private int numDataPartitions = 0;
    +  private int numColMultPartitions = 0;
    +
    +  private boolean colMultReduceByKey = false;
    +
    +  public ComputeStreamingResponse(FileSystem fileSys) throws Exception
    +  {
    +    fs = fileSys;
    +    storage = new HadoopFileSystemStore(fs);
    +
    +    dataInputFormat = SystemConfiguration.getProperty("pir.dataInputFormat");
    +    if (!InputFormatConst.ALLOWED_FORMATS.contains(dataInputFormat))
    +    {
    +      throw new IllegalArgumentException("inputFormat = " + dataInputFormat + " is of an unknown form");
    +    }
    +    logger.info("inputFormat = " + dataInputFormat);
    +    if (dataInputFormat.equals(InputFormatConst.BASE_FORMAT))
    +    {
    +      inputData = SystemConfiguration.getProperty("pir.inputData", "none");
    +      if (inputData.equals("none"))
    +      {
    +        throw new IllegalArgumentException("For inputFormat = " + dataInputFormat + " an inputFile must be specified");
    +      }
    +      logger.info("inputFile = " + inputData);
    +    }
    +    else if (dataInputFormat.equals(InputFormatConst.ES))
    +    {
    +      esQuery = SystemConfiguration.getProperty("pir.esQuery", "none");
    +      esResource = SystemConfiguration.getProperty("pir.esResource", "none");
    +      if (esQuery.equals("none"))
    +      {
    +        throw new IllegalArgumentException("esQuery must be specified");
    +      }
    +      if (esResource.equals("none"))
    +      {
    +        throw new IllegalArgumentException("esResource must be specified");
    +      }
    +      logger.info("esQuery = " + esQuery + " esResource = " + esResource);
    +    }
    +    outputFile = SystemConfiguration.getProperty("pir.outputFile");
    +    outputDirExp = outputFile + "_exp";
    +
    +    queryInput = SystemConfiguration.getProperty("pir.queryInput");
    +    String stopListFile = SystemConfiguration.getProperty("pir.stopListFile");
    +
    +    logger.info("outputFile = " + outputFile + " queryInputDir = " + queryInput + " stopListFile = " + stopListFile + " esQuery = " + esQuery
    +        + " esResource = " + esResource);
    +
    +    // Pull the batchSeconds and windowLength parameters
    +    batchSeconds = Long.parseLong(SystemConfiguration.getProperty("pir.sparkstreaming.batchSeconds", "30"));
    +    windowLength = Long.parseLong(SystemConfiguration.getProperty("pir.sparkstreaming.windowLength", "60"));
    +    if (windowLength % batchSeconds != 0)
    +    {
    +      throw new IllegalArgumentException("batchSeconds = " + batchSeconds + " must divide windowLength = " + windowLength);
    +    }
    +    useQueueStream = SystemConfiguration.getProperty("pir.sparkstreaming.useQueueStream", "false").equals("false");
    +
    +    // Set the necessary configurations
    +    SparkConf conf = new SparkConf().setAppName("SparkPIR").setMaster("yarn-cluster");
    +    conf.set("es.nodes", SystemConfiguration.getProperty("es.nodes", "none"));
    +    conf.set("es.port", SystemConfiguration.getProperty("es.port", "none"));
    +    conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
    +    conf.set("spark.streaming.stopGracefullyOnShutdown", SystemConfiguration.getProperty("spark.streaming.stopGracefullyOnShutdown", "false"));
    --- End diff --
    
    Quite a few places where the ```SystemConfiguration.get*Property``` could be used to help with typing variables.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-pirk pull request #76: [PIRK-21] [WIP] - Initial Spark Streaming R...

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

    https://github.com/apache/incubator-pirk/pull/76#discussion_r75834645
  
    --- Diff: src/main/java/org/apache/pirk/responder/wideskies/spark/streaming/ComputeStreamingResponse.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.pirk.responder.wideskies.spark.streaming;
    +
    +import java.math.BigInteger;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Queue;
    +
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.io.MapWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapreduce.Job;
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
    +import org.apache.pirk.inputformat.hadoop.BaseInputFormat;
    +import org.apache.pirk.inputformat.hadoop.InputFormatConst;
    +import org.apache.pirk.query.wideskies.Query;
    +import org.apache.pirk.query.wideskies.QueryInfo;
    +import org.apache.pirk.responder.wideskies.spark.Accumulators;
    +import org.apache.pirk.responder.wideskies.spark.BroadcastVars;
    +import org.apache.pirk.responder.wideskies.spark.EncColMultGroupedMapper;
    +import org.apache.pirk.responder.wideskies.spark.EncColMultReducer;
    +import org.apache.pirk.responder.wideskies.spark.EncRowCalc;
    +import org.apache.pirk.responder.wideskies.spark.FilterData;
    +import org.apache.pirk.responder.wideskies.spark.HashSelectorsAndPartitionData;
    +import org.apache.pirk.schema.data.DataSchema;
    +import org.apache.pirk.schema.data.DataSchemaLoader;
    +import org.apache.pirk.schema.data.DataSchemaRegistry;
    +import org.apache.pirk.schema.query.QuerySchema;
    +import org.apache.pirk.schema.query.QuerySchemaLoader;
    +import org.apache.pirk.schema.query.QuerySchemaRegistry;
    +import org.apache.pirk.serialization.HadoopFileSystemStore;
    +import org.apache.pirk.utils.PIRException;
    +import org.apache.pirk.utils.SystemConfiguration;
    +import org.apache.spark.SparkConf;
    +import org.apache.spark.api.java.JavaPairRDD;
    +import org.apache.spark.api.java.JavaRDD;
    +import org.apache.spark.api.java.JavaSparkContext;
    +import org.apache.spark.api.java.function.Function;
    +import org.apache.spark.api.java.function.VoidFunction;
    +import org.apache.spark.streaming.Durations;
    +import org.apache.spark.streaming.api.java.JavaDStream;
    +import org.apache.spark.streaming.api.java.JavaPairDStream;
    +import org.apache.spark.streaming.api.java.JavaPairInputDStream;
    +import org.apache.spark.streaming.api.java.JavaStreamingContext;
    +import org.elasticsearch.hadoop.mr.EsInputFormat;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Master class for the PIR query spark streaming application
    + * <p>
    + * NOTE:
    + * <p>
    + * - NOT using Elasticsearch in practice - proved to be some speed issues with ES and Spark that appear to be ES-Spark specific - leave code in anticipation
    + * that the ES-Spark issues resolve...
    + * <p>
    + * - Even if rdd.count() calls are embedded in logger.debug statements, they are computed by Spark. Thus, they are commented out in the code below - uncomment
    + * for rdd.count() debug
    + * 
    + */
    +public class ComputeStreamingResponse
    +{
    +  private static final Logger logger = LoggerFactory.getLogger(ComputeStreamingResponse.class);
    +
    +  private String dataInputFormat = null;
    +  private String inputData = null;
    +  private String outputFile = null;
    +  private String outputDirExp = null;
    +
    +  private String queryInput = null;
    +  QuerySchema qSchema = null;
    +
    +  private String esQuery = "none";
    +  private String esResource = "none";
    +
    +  private FileSystem fs = null;
    +  private HadoopFileSystemStore storage = null;
    +  private JavaStreamingContext jssc = null;
    +
    +  boolean useQueueStream = false;
    +
    +  private long batchSeconds = 0;
    +  private long windowLength = 0;
    +
    +  private Accumulators accum = null;
    +  private BroadcastVars bVars = null;
    +
    +  private QueryInfo queryInfo = null;
    +  Query query = null;
    +
    +  private int numDataPartitions = 0;
    +  private int numColMultPartitions = 0;
    +
    +  private boolean colMultReduceByKey = false;
    +
    +  public ComputeStreamingResponse(FileSystem fileSys) throws Exception
    +  {
    +    fs = fileSys;
    +    storage = new HadoopFileSystemStore(fs);
    +
    +    dataInputFormat = SystemConfiguration.getProperty("pir.dataInputFormat");
    +    if (!InputFormatConst.ALLOWED_FORMATS.contains(dataInputFormat))
    +    {
    +      throw new IllegalArgumentException("inputFormat = " + dataInputFormat + " is of an unknown form");
    +    }
    +    logger.info("inputFormat = " + dataInputFormat);
    +    if (dataInputFormat.equals(InputFormatConst.BASE_FORMAT))
    +    {
    +      inputData = SystemConfiguration.getProperty("pir.inputData", "none");
    +      if (inputData.equals("none"))
    +      {
    +        throw new IllegalArgumentException("For inputFormat = " + dataInputFormat + " an inputFile must be specified");
    +      }
    +      logger.info("inputFile = " + inputData);
    +    }
    +    else if (dataInputFormat.equals(InputFormatConst.ES))
    +    {
    +      esQuery = SystemConfiguration.getProperty("pir.esQuery", "none");
    +      esResource = SystemConfiguration.getProperty("pir.esResource", "none");
    +      if (esQuery.equals("none"))
    +      {
    +        throw new IllegalArgumentException("esQuery must be specified");
    +      }
    +      if (esResource.equals("none"))
    +      {
    +        throw new IllegalArgumentException("esResource must be specified");
    +      }
    +      logger.info("esQuery = " + esQuery + " esResource = " + esResource);
    +    }
    +    outputFile = SystemConfiguration.getProperty("pir.outputFile");
    +    outputDirExp = outputFile + "_exp";
    +
    +    queryInput = SystemConfiguration.getProperty("pir.queryInput");
    +    String stopListFile = SystemConfiguration.getProperty("pir.stopListFile");
    +
    +    logger.info("outputFile = " + outputFile + " queryInputDir = " + queryInput + " stopListFile = " + stopListFile + " esQuery = " + esQuery
    +        + " esResource = " + esResource);
    +
    +    // Pull the batchSeconds and windowLength parameters
    +    batchSeconds = Long.parseLong(SystemConfiguration.getProperty("pir.sparkstreaming.batchSeconds", "30"));
    --- End diff --
    
    ```Long``` is a bit generous for a duration in seconds.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-pirk pull request #76: [PIRK-21] [WIP] - Initial Spark Streaming R...

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

    https://github.com/apache/incubator-pirk/pull/76#discussion_r75834426
  
    --- Diff: src/main/resources/pirk.properties ---
    @@ -28,7 +28,8 @@ local.pirk.properties.dir=/root/
     ##
     ##Spark path for SparkLauncher
     ##
    -spark.home = /usr
    +#spark.home = /usr
    +spark.home=/opt/cloudera/parcels/CDH/lib/spark
    --- End diff --
    
    I'm guessing this ```pirk.properties``` was included in the commit by mistake.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-pirk pull request #76: [PIRK-21] - Initial Spark Streaming Respond...

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

    https://github.com/apache/incubator-pirk/pull/76


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-pirk pull request #76: [PIRK-21] [WIP] - Initial Spark Streaming R...

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

    https://github.com/apache/incubator-pirk/pull/76#discussion_r75878104
  
    --- Diff: src/main/java/org/apache/pirk/responder/wideskies/spark/streaming/ComputeStreamingResponse.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.pirk.responder.wideskies.spark.streaming;
    +
    +import java.math.BigInteger;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Queue;
    +
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.io.MapWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapreduce.Job;
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
    +import org.apache.pirk.inputformat.hadoop.BaseInputFormat;
    +import org.apache.pirk.inputformat.hadoop.InputFormatConst;
    +import org.apache.pirk.query.wideskies.Query;
    +import org.apache.pirk.query.wideskies.QueryInfo;
    +import org.apache.pirk.responder.wideskies.spark.Accumulators;
    +import org.apache.pirk.responder.wideskies.spark.BroadcastVars;
    +import org.apache.pirk.responder.wideskies.spark.EncColMultGroupedMapper;
    +import org.apache.pirk.responder.wideskies.spark.EncColMultReducer;
    +import org.apache.pirk.responder.wideskies.spark.EncRowCalc;
    +import org.apache.pirk.responder.wideskies.spark.FilterData;
    +import org.apache.pirk.responder.wideskies.spark.HashSelectorsAndPartitionData;
    +import org.apache.pirk.schema.data.DataSchema;
    +import org.apache.pirk.schema.data.DataSchemaLoader;
    +import org.apache.pirk.schema.data.DataSchemaRegistry;
    +import org.apache.pirk.schema.query.QuerySchema;
    +import org.apache.pirk.schema.query.QuerySchemaLoader;
    +import org.apache.pirk.schema.query.QuerySchemaRegistry;
    +import org.apache.pirk.serialization.HadoopFileSystemStore;
    +import org.apache.pirk.utils.PIRException;
    +import org.apache.pirk.utils.SystemConfiguration;
    +import org.apache.spark.SparkConf;
    +import org.apache.spark.api.java.JavaPairRDD;
    +import org.apache.spark.api.java.JavaRDD;
    +import org.apache.spark.api.java.JavaSparkContext;
    +import org.apache.spark.api.java.function.Function;
    +import org.apache.spark.api.java.function.VoidFunction;
    +import org.apache.spark.streaming.Durations;
    +import org.apache.spark.streaming.api.java.JavaDStream;
    +import org.apache.spark.streaming.api.java.JavaPairDStream;
    +import org.apache.spark.streaming.api.java.JavaPairInputDStream;
    +import org.apache.spark.streaming.api.java.JavaStreamingContext;
    +import org.elasticsearch.hadoop.mr.EsInputFormat;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Master class for the PIR query spark streaming application
    + * <p>
    + * NOTE:
    + * <p>
    + * - NOT using Elasticsearch in practice - proved to be some speed issues with ES and Spark that appear to be ES-Spark specific - leave code in anticipation
    + * that the ES-Spark issues resolve...
    + * <p>
    + * - Even if rdd.count() calls are embedded in logger.debug statements, they are computed by Spark. Thus, they are commented out in the code below - uncomment
    + * for rdd.count() debug
    + * 
    + */
    +public class ComputeStreamingResponse
    +{
    +  private static final Logger logger = LoggerFactory.getLogger(ComputeStreamingResponse.class);
    +
    +  private String dataInputFormat = null;
    +  private String inputData = null;
    +  private String outputFile = null;
    +  private String outputDirExp = null;
    +
    +  private String queryInput = null;
    +  QuerySchema qSchema = null;
    +
    +  private String esQuery = "none";
    +  private String esResource = "none";
    +
    +  private FileSystem fs = null;
    +  private HadoopFileSystemStore storage = null;
    +  private JavaStreamingContext jssc = null;
    +
    +  boolean useQueueStream = false;
    +
    +  private long batchSeconds = 0;
    +  private long windowLength = 0;
    +
    +  private Accumulators accum = null;
    +  private BroadcastVars bVars = null;
    +
    +  private QueryInfo queryInfo = null;
    +  Query query = null;
    +
    +  private int numDataPartitions = 0;
    +  private int numColMultPartitions = 0;
    +
    +  private boolean colMultReduceByKey = false;
    +
    +  public ComputeStreamingResponse(FileSystem fileSys) throws Exception
    +  {
    +    fs = fileSys;
    +    storage = new HadoopFileSystemStore(fs);
    +
    +    dataInputFormat = SystemConfiguration.getProperty("pir.dataInputFormat");
    +    if (!InputFormatConst.ALLOWED_FORMATS.contains(dataInputFormat))
    +    {
    +      throw new IllegalArgumentException("inputFormat = " + dataInputFormat + " is of an unknown form");
    +    }
    +    logger.info("inputFormat = " + dataInputFormat);
    +    if (dataInputFormat.equals(InputFormatConst.BASE_FORMAT))
    +    {
    +      inputData = SystemConfiguration.getProperty("pir.inputData", "none");
    +      if (inputData.equals("none"))
    +      {
    +        throw new IllegalArgumentException("For inputFormat = " + dataInputFormat + " an inputFile must be specified");
    +      }
    +      logger.info("inputFile = " + inputData);
    +    }
    +    else if (dataInputFormat.equals(InputFormatConst.ES))
    +    {
    +      esQuery = SystemConfiguration.getProperty("pir.esQuery", "none");
    +      esResource = SystemConfiguration.getProperty("pir.esResource", "none");
    +      if (esQuery.equals("none"))
    +      {
    +        throw new IllegalArgumentException("esQuery must be specified");
    +      }
    +      if (esResource.equals("none"))
    +      {
    +        throw new IllegalArgumentException("esResource must be specified");
    +      }
    +      logger.info("esQuery = " + esQuery + " esResource = " + esResource);
    +    }
    +    outputFile = SystemConfiguration.getProperty("pir.outputFile");
    +    outputDirExp = outputFile + "_exp";
    +
    +    queryInput = SystemConfiguration.getProperty("pir.queryInput");
    +    String stopListFile = SystemConfiguration.getProperty("pir.stopListFile");
    +
    +    logger.info("outputFile = " + outputFile + " queryInputDir = " + queryInput + " stopListFile = " + stopListFile + " esQuery = " + esQuery
    +        + " esResource = " + esResource);
    +
    +    // Pull the batchSeconds and windowLength parameters
    +    batchSeconds = Long.parseLong(SystemConfiguration.getProperty("pir.sparkstreaming.batchSeconds", "30"));
    +    windowLength = Long.parseLong(SystemConfiguration.getProperty("pir.sparkstreaming.windowLength", "60"));
    +    if (windowLength % batchSeconds != 0)
    +    {
    +      throw new IllegalArgumentException("batchSeconds = " + batchSeconds + " must divide windowLength = " + windowLength);
    +    }
    +    useQueueStream = SystemConfiguration.getProperty("pir.sparkstreaming.useQueueStream", "false").equals("false");
    +
    +    // Set the necessary configurations
    +    SparkConf conf = new SparkConf().setAppName("SparkPIR").setMaster("yarn-cluster");
    +    conf.set("es.nodes", SystemConfiguration.getProperty("es.nodes", "none"));
    +    conf.set("es.port", SystemConfiguration.getProperty("es.port", "none"));
    +    conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
    +    conf.set("spark.streaming.stopGracefullyOnShutdown", SystemConfiguration.getProperty("spark.streaming.stopGracefullyOnShutdown", "false"));
    --- End diff --
    
    Yes - I will update it with the new typing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-pirk issue #76: [PIRK-21] - Initial Spark Streaming Responder Impl...

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

    https://github.com/apache/incubator-pirk/pull/76
  
    Scale testing is in progress - I would like to go ahead and merge this into master before it gets too far behind... I will update as necessary with any changes that result from scale testing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-pirk pull request #76: [PIRK-21] [WIP] - Initial Spark Streaming R...

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

    https://github.com/apache/incubator-pirk/pull/76#discussion_r75835110
  
    --- Diff: src/main/resources/META-INF/bin-license-notice/LICENSE-bin ---
    @@ -0,0 +1,262 @@
    +
    --- End diff --
    
    I think these license changes are part of another commit.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-pirk pull request #76: [PIRK-21] [WIP] - Initial Spark Streaming R...

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

    https://github.com/apache/incubator-pirk/pull/76#discussion_r75879106
  
    --- Diff: src/main/java/org/apache/pirk/responder/wideskies/spark/streaming/ComputeStreamingResponse.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.pirk.responder.wideskies.spark.streaming;
    +
    +import java.math.BigInteger;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Queue;
    +
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.io.MapWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapreduce.Job;
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
    +import org.apache.pirk.inputformat.hadoop.BaseInputFormat;
    +import org.apache.pirk.inputformat.hadoop.InputFormatConst;
    +import org.apache.pirk.query.wideskies.Query;
    +import org.apache.pirk.query.wideskies.QueryInfo;
    +import org.apache.pirk.responder.wideskies.spark.Accumulators;
    +import org.apache.pirk.responder.wideskies.spark.BroadcastVars;
    +import org.apache.pirk.responder.wideskies.spark.EncColMultGroupedMapper;
    +import org.apache.pirk.responder.wideskies.spark.EncColMultReducer;
    +import org.apache.pirk.responder.wideskies.spark.EncRowCalc;
    +import org.apache.pirk.responder.wideskies.spark.FilterData;
    +import org.apache.pirk.responder.wideskies.spark.HashSelectorsAndPartitionData;
    +import org.apache.pirk.schema.data.DataSchema;
    +import org.apache.pirk.schema.data.DataSchemaLoader;
    +import org.apache.pirk.schema.data.DataSchemaRegistry;
    +import org.apache.pirk.schema.query.QuerySchema;
    +import org.apache.pirk.schema.query.QuerySchemaLoader;
    +import org.apache.pirk.schema.query.QuerySchemaRegistry;
    +import org.apache.pirk.serialization.HadoopFileSystemStore;
    +import org.apache.pirk.utils.PIRException;
    +import org.apache.pirk.utils.SystemConfiguration;
    +import org.apache.spark.SparkConf;
    +import org.apache.spark.api.java.JavaPairRDD;
    +import org.apache.spark.api.java.JavaRDD;
    +import org.apache.spark.api.java.JavaSparkContext;
    +import org.apache.spark.api.java.function.Function;
    +import org.apache.spark.api.java.function.VoidFunction;
    +import org.apache.spark.streaming.Durations;
    +import org.apache.spark.streaming.api.java.JavaDStream;
    +import org.apache.spark.streaming.api.java.JavaPairDStream;
    +import org.apache.spark.streaming.api.java.JavaPairInputDStream;
    +import org.apache.spark.streaming.api.java.JavaStreamingContext;
    +import org.elasticsearch.hadoop.mr.EsInputFormat;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Master class for the PIR query spark streaming application
    + * <p>
    + * NOTE:
    + * <p>
    + * - NOT using Elasticsearch in practice - proved to be some speed issues with ES and Spark that appear to be ES-Spark specific - leave code in anticipation
    + * that the ES-Spark issues resolve...
    + * <p>
    + * - Even if rdd.count() calls are embedded in logger.debug statements, they are computed by Spark. Thus, they are commented out in the code below - uncomment
    + * for rdd.count() debug
    + * 
    + */
    +public class ComputeStreamingResponse
    +{
    +  private static final Logger logger = LoggerFactory.getLogger(ComputeStreamingResponse.class);
    +
    +  private String dataInputFormat = null;
    +  private String inputData = null;
    +  private String outputFile = null;
    +  private String outputDirExp = null;
    +
    +  private String queryInput = null;
    +  QuerySchema qSchema = null;
    +
    +  private String esQuery = "none";
    +  private String esResource = "none";
    +
    +  private FileSystem fs = null;
    +  private HadoopFileSystemStore storage = null;
    +  private JavaStreamingContext jssc = null;
    +
    +  boolean useQueueStream = false;
    +
    +  private long batchSeconds = 0;
    +  private long windowLength = 0;
    +
    +  private Accumulators accum = null;
    +  private BroadcastVars bVars = null;
    +
    +  private QueryInfo queryInfo = null;
    +  Query query = null;
    +
    +  private int numDataPartitions = 0;
    +  private int numColMultPartitions = 0;
    +
    +  private boolean colMultReduceByKey = false;
    +
    +  public ComputeStreamingResponse(FileSystem fileSys) throws Exception
    +  {
    +    fs = fileSys;
    +    storage = new HadoopFileSystemStore(fs);
    +
    +    dataInputFormat = SystemConfiguration.getProperty("pir.dataInputFormat");
    +    if (!InputFormatConst.ALLOWED_FORMATS.contains(dataInputFormat))
    +    {
    +      throw new IllegalArgumentException("inputFormat = " + dataInputFormat + " is of an unknown form");
    +    }
    +    logger.info("inputFormat = " + dataInputFormat);
    +    if (dataInputFormat.equals(InputFormatConst.BASE_FORMAT))
    +    {
    +      inputData = SystemConfiguration.getProperty("pir.inputData", "none");
    +      if (inputData.equals("none"))
    +      {
    +        throw new IllegalArgumentException("For inputFormat = " + dataInputFormat + " an inputFile must be specified");
    +      }
    +      logger.info("inputFile = " + inputData);
    +    }
    +    else if (dataInputFormat.equals(InputFormatConst.ES))
    +    {
    +      esQuery = SystemConfiguration.getProperty("pir.esQuery", "none");
    +      esResource = SystemConfiguration.getProperty("pir.esResource", "none");
    +      if (esQuery.equals("none"))
    +      {
    +        throw new IllegalArgumentException("esQuery must be specified");
    +      }
    +      if (esResource.equals("none"))
    +      {
    +        throw new IllegalArgumentException("esResource must be specified");
    +      }
    +      logger.info("esQuery = " + esQuery + " esResource = " + esResource);
    +    }
    +    outputFile = SystemConfiguration.getProperty("pir.outputFile");
    +    outputDirExp = outputFile + "_exp";
    +
    +    queryInput = SystemConfiguration.getProperty("pir.queryInput");
    +    String stopListFile = SystemConfiguration.getProperty("pir.stopListFile");
    +
    +    logger.info("outputFile = " + outputFile + " queryInputDir = " + queryInput + " stopListFile = " + stopListFile + " esQuery = " + esQuery
    +        + " esResource = " + esResource);
    +
    +    // Pull the batchSeconds and windowLength parameters
    +    batchSeconds = Long.parseLong(SystemConfiguration.getProperty("pir.sparkstreaming.batchSeconds", "30"));
    --- End diff --
    
    'batchSeconds' is an input to the Durations class' static seconds method which requires a long (http://spark.apache.org/docs/latest/api/java/) 
    
    But, yes, I do think that a long is a bit of overkill ;)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-pirk issue #76: [PIRK-21] - Initial Spark Streaming Responder Impl...

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

    https://github.com/apache/incubator-pirk/pull/76
  
    +1, it is good enough to merge into master and refine from there.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---