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/07/25 23:18:13 UTC

[GitHub] incubator-pirk pull request #27: Pirk 27 - Improve Pirk's Use of Property Fi...

GitHub user ellisonanne opened a pull request:

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

    Pirk 27 - Improve Pirk's Use of Property Files

    Taking a second look, instead of adding property files for the QuerierDriver and ResponderDriver and then having to parse and validate the properties, I decided to instead provide scripts to aid in running the drivers. Thus, the arg/property validation is left in the CLI and the drivers are easier to run via the scripts.

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

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

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

    https://github.com/apache/incubator-pirk/pull/27.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 #27
    
----
commit 137a18e4bd23c52fd26dfe8def1a4300b8e204c7
Author: eawilliams <ea...@apache.org>
Date:   2016-07-25T13:41:43Z

    intermediate efforts

commit dbdc9694363f2b0a58628e136bb70bdb194c3b71
Author: eawilliams <ea...@apache.org>
Date:   2016-07-25T23:13:44Z

    adding scripts to aid in running the QuerierDriver and ResponderDriver

----


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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/27#discussion_r72703018
  
    --- Diff: src/main/resources/querier.properties ---
    @@ -0,0 +1,123 @@
    +###############################################################################
    +# 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.
    +###############################################################################
    +
    +##
    +## Required Properties
    +##
    +
    +#action -- required - 'encrypt' or 'decrypt' -- The action performed by the QuerierDriver
    +querier.action=
    +
    +#inputFile - required - Fully qualified file containing input
    +#The input is either:
    +#(1) For Encryption: A query file - Contains the query selectors, one per line;
    +#the first line must be the query number
    +#OR
    +#(2) For Decryption: A response file - Contains the serialized Response object
    +querier.inputFile=
    +
    +#outputFile -- required - Fully qualified file for the result output.
    +#The output file specifies either:
    +#(1) For encryption:
    +#(a) A file to contain the serialized Querier object named: <outputFile>-querier
    +#AND
    +#(b) A file to contain the serialized Query object named: <outputFile>-query
    +#OR
    +#(2) A file to contain the decryption results where each line is where each line
    +#corresponds to one hit and is a JSON object with the schema QuerySchema
    +querier.outputFile=
    +
    +#numThreads -- required -- Number of threads to use for encryption/decryption
    +querier.numThreads=
    --- End diff --
    
    Does this need to be required?  Seems like a sensible default could be provided as a convenience.


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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/27#discussion_r72701770
  
    --- Diff: src/main/java/org/apache/pirk/utils/SystemConfiguration.java ---
    @@ -160,4 +139,78 @@ public static void resetProperties()
         clearProperties();
         initialize();
       }
    +
    +  /**
    +   * Loads the properties from local properties file in the specified directory
    +   * <p>
    +   * Only files ending in '.properties' will be loaded
    +   */
    +  public static void loadPropsFromDir(String dirName)
    +  {
    +    File[] directoryListing = new File(dirName).listFiles(new FilenameFilter()
    +    {
    +      @Override
    +      public boolean accept(File dir, String name)
    +      {
    +        return name.endsWith(".properties");
    +      }
    +    });
    +
    +    if (directoryListing != null)
    +    {
    +      for (File file : directoryListing)
    +      {
    +        loadPropsFromFile(file);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Loads the properties from the specified file
    +   */
    +  public static void loadPropsFromFile(File file)
    +  {
    +    if (file.exists())
    +    {
    +      try (InputStream stream = new FileInputStream(file);)
    +      {
    +        logger.info("Loading properties file '" + file.getAbsolutePath() + "'");
    +        props.load(stream);
    +        stream.close();
    +      } catch (IOException e)
    +      {
    +        logger.error("Problem loading properties file '" + file.getAbsolutePath() + "'");
    +        e.printStackTrace();
    +      }
    +    }
    +    else
    +    {
    +      logger.warn("Properties file does not exist: '" + file.getAbsolutePath() + "'");
    +    }
    +  }
    +
    +  /**
    +   * Loads the properties from the specified file on the classpath
    +   */
    +  public static void loadPropsFromStream(String name)
    --- End diff --
    
    Consider renaming ```loadPropsFromStream``` -> ```loadPropsFromResource```


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Prope...

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/27#discussion_r72586812
  
    --- Diff: src/main/java/org/apache/pirk/utils/SystemConfiguration.java ---
    @@ -152,6 +105,31 @@ public static void setProperty(String propertyName, String value)
         props.setProperty(propertyName, value);
       }
     
    +  public static boolean hasProperty(String propertyName)
    +  {
    +    return props.containsKey(propertyName);
    +  }
    +
    +  /**
    +   * Append a property via a column separated list
    --- End diff --
    
    Typo?  Should say "comma" separated?


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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/27#discussion_r72732010
  
    --- Diff: src/main/java/org/apache/pirk/responder/wideskies/spark/ComputeResponse.java ---
    @@ -202,7 +203,7 @@ private void setup() throws Exception
         numColMultPartitions = Integer.parseInt(SystemConfiguration.getProperty("pir.numColMultPartitions", numDataPartsString));
     
         // Whether or not we are performing a reduceByKey or a groupByKey->reduce for column multiplication
    -    colMultReduceByKey = SystemConfiguration.getProperty("pir.colMultReduceByKey").equals("true");
    +    colMultReduceByKey = SystemConfiguration.getProperty("pir.colMultReduceByKey", "false").equals("true");
    --- End diff --
    
    In scalability testing, reduceByKey for column multiplication proved far less efficient than groupByKey followed by a map functionality (hence the 'false' default value). 


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Prope...

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/27#discussion_r72594517
  
    --- Diff: pom.xml ---
    @@ -117,6 +117,7 @@
     			<version>3.3</version>
     		</dependency>
     
    +<!--
    --- End diff --
    
    Same as above -- that's my intermediate scratch - I will remove once I remove the WIP from the pull request


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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/27#discussion_r72702338
  
    --- Diff: src/main/resources/pirk.properties-repo ---
    @@ -0,0 +1,243 @@
    +###############################################################################
    +# 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.
    +###############################################################################
    +
    +##
    +## Default System Configuration Properties
    +##
    +
    +# Name of log4j properties file (relative to current folder)
    +log4jPropertiesFile=log4j2.properties
    +
    +#Name of the directory holding the local properties files
    +#All property files must end in .properties
    +local.pirk.properties.dir=/root/properties
    +
    +##
    +##Spark path for SparkLauncher
    +##
    +spark.home = /usr
    +
    +##
    +## Data schema properties
    +##
    +## Each data schema should be specified in an xml file of the form; 
    +## 		all items are treated in a case insensitive manner:
    --- End diff --
    
    No longer true that they are all case insensitive.
    Consider referring to Pirk's website rather than recreate schema definition?


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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/27#discussion_r72698216
  
    --- Diff: src/main/java/org/apache/pirk/querier/wideskies/QuerierDriverCLI.java ---
    @@ -136,192 +111,26 @@ private boolean parseOptions()
       {
         boolean valid = true;
     
    -    // Parse general required options
    -    if (!hasOption(ACTION))
    -    {
    -      logger.info("Must have the option " + ACTION);
    -      return false;
    -    }
    -    String action = getOptionValue(ACTION).toLowerCase();
    -    if (!action.equals("encrypt") && !action.equals("decrypt"))
    -    {
    -      logger.info("Unsupported action: " + action);
    -    }
    -    SystemConfiguration.setProperty(ACTION, action);
    -
    -    if (!hasOption(INPUTFILE))
    -    {
    -      logger.info("Must have the option " + INPUTFILE);
    -      return false;
    -    }
    -    SystemConfiguration.setProperty(INPUTFILE, getOptionValue(INPUTFILE));
    -
    -    if (!hasOption(OUTPUTFILE))
    -    {
    -      logger.info("Must have the option " + OUTPUTFILE);
    -      return false;
    -    }
    -    SystemConfiguration.setProperty(OUTPUTFILE, getOptionValue(OUTPUTFILE));
    -
    -    if (!hasOption(NUMTHREADS))
    -    {
    -      logger.info("Must have the option " + NUMTHREADS);
    -      return false;
    -    }
    -    SystemConfiguration.setProperty(NUMTHREADS, getOptionValue(NUMTHREADS));
    -
    -    if (!hasOption(QUERYSCHEMAS))
    -    {
    -      logger.info("Must have the option " + QUERYSCHEMAS);
    -      return false;
    -    }
    -    SystemConfiguration.setProperty("query.schemas", getOptionValue(QUERYSCHEMAS));
    -
    -    if (!hasOption(DATASCHEMAS))
    -    {
    -      logger.info("Must have the option " + DATASCHEMAS);
    -      return false;
    -    }
    -    SystemConfiguration.setProperty("data.schemas", getOptionValue(DATASCHEMAS));
    -
    -    // Parse general optional args
    -    if (hasOption(EMBEDQUERYSCHEMA))
    +    // If we have a local.querier.properties file specified, load it
    +    if (hasOption(LOCALPROPFILE))
         {
    -      SystemConfiguration.setProperty("pir.embedQuerySchema", getOptionValue(EMBEDQUERYSCHEMA));
    +      System.out.println("getOptionValue(LOCALPROPFILE) = " + getOptionValue(LOCALPROPFILE));
    --- End diff --
    
    A rogue temporary debug line?


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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

    https://github.com/apache/incubator-pirk/pull/27#discussion_r72730877
  
    --- Diff: src/main/java/org/apache/pirk/responder/wideskies/spark/ComputeResponse.java ---
    @@ -202,7 +203,7 @@ private void setup() throws Exception
         numColMultPartitions = Integer.parseInt(SystemConfiguration.getProperty("pir.numColMultPartitions", numDataPartsString));
     
         // Whether or not we are performing a reduceByKey or a groupByKey->reduce for column multiplication
    -    colMultReduceByKey = SystemConfiguration.getProperty("pir.colMultReduceByKey").equals("true");
    +    colMultReduceByKey = SystemConfiguration.getProperty("pir.colMultReduceByKey", "false").equals("true");
    --- End diff --
    
    Why is the default for colMultReduceByKey not "true"?  Seems opposite what I'd expect to be optimal.



---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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/27#discussion_r72699772
  
    --- Diff: src/main/java/org/apache/pirk/querier/wideskies/QuerierDriverCLI.java ---
    @@ -136,192 +111,26 @@ private boolean parseOptions()
       {
         boolean valid = true;
     
    -    // Parse general required options
    -    if (!hasOption(ACTION))
    -    {
    -      logger.info("Must have the option " + ACTION);
    -      return false;
    -    }
    -    String action = getOptionValue(ACTION).toLowerCase();
    -    if (!action.equals("encrypt") && !action.equals("decrypt"))
    -    {
    -      logger.info("Unsupported action: " + action);
    -    }
    -    SystemConfiguration.setProperty(ACTION, action);
    -
    -    if (!hasOption(INPUTFILE))
    -    {
    -      logger.info("Must have the option " + INPUTFILE);
    -      return false;
    -    }
    -    SystemConfiguration.setProperty(INPUTFILE, getOptionValue(INPUTFILE));
    -
    -    if (!hasOption(OUTPUTFILE))
    -    {
    -      logger.info("Must have the option " + OUTPUTFILE);
    -      return false;
    -    }
    -    SystemConfiguration.setProperty(OUTPUTFILE, getOptionValue(OUTPUTFILE));
    -
    -    if (!hasOption(NUMTHREADS))
    -    {
    -      logger.info("Must have the option " + NUMTHREADS);
    -      return false;
    -    }
    -    SystemConfiguration.setProperty(NUMTHREADS, getOptionValue(NUMTHREADS));
    -
    -    if (!hasOption(QUERYSCHEMAS))
    -    {
    -      logger.info("Must have the option " + QUERYSCHEMAS);
    -      return false;
    -    }
    -    SystemConfiguration.setProperty("query.schemas", getOptionValue(QUERYSCHEMAS));
    -
    -    if (!hasOption(DATASCHEMAS))
    -    {
    -      logger.info("Must have the option " + DATASCHEMAS);
    -      return false;
    -    }
    -    SystemConfiguration.setProperty("data.schemas", getOptionValue(DATASCHEMAS));
    -
    -    // Parse general optional args
    -    if (hasOption(EMBEDQUERYSCHEMA))
    +    // If we have a local.querier.properties file specified, load it
    +    if (hasOption(LOCALPROPFILE))
         {
    -      SystemConfiguration.setProperty("pir.embedQuerySchema", getOptionValue(EMBEDQUERYSCHEMA));
    +      System.out.println("getOptionValue(LOCALPROPFILE) = " + getOptionValue(LOCALPROPFILE));
    --- End diff --
    
    Yep - removed


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Prope...

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/27#discussion_r72594478
  
    --- Diff: src/main/resources/pirk.properties ---
    @@ -31,7 +31,8 @@ local.pirk.properties=/root/local.pirk.properties
     ##
     ##Spark path for SparkLauncher
     ##
    -spark.home = /usr
    +#spark.home = /usr
    +spark.home=/opt/cloudera/parcels/CDH/lib/spark
    --- End diff --
    
    That's my intermediate scratch - I will remove once I remove the WIP from the pull request (I was running the distributed testing on a CDH cluster)


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Prope...

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/27#discussion_r72594556
  
    --- Diff: src/main/java/org/apache/pirk/querier/wideskies/QuerierProps.java ---
    @@ -0,0 +1,202 @@
    +package org.apache.pirk.querier.wideskies;
    +
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import org.apache.pirk.schema.data.LoadDataSchemas;
    +import org.apache.pirk.schema.query.LoadQuerySchemas;
    +import org.apache.pirk.utils.SystemConfiguration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Properties constants and validation for the Querier
    + */
    +public class QuerierProps
    +{
    +  private static final Logger logger = LoggerFactory.getLogger(QuerierProps.class);
    +
    +  // General properties
    +  public static final String ACTION = "querier.action";
    +  public static final String INPUTFILE = "querier.inputFile";
    +  public static final String OUTPUTFILE = "querier.outputFile";
    +  public static final String QUERYTYPE = "querier.queryType";
    +  public static final String NUMTHREADS = "querier.numThreads";
    +
    +  // Encryption properties
    +  public static final String HASHBITSIZE = "querier.hashBitSize";
    +  public static final String HASHKEY = "querier.hashKey";
    +  public static final String DATAPARTITIONSIZE = "querier.dataPartitionBitSize";
    +  public static final String PAILLIERBITSIZE = "querier.paillierBitSize";
    +  public static final String BITSET = "querier.bitSet";
    +  public static final String CERTAINTY = "querier.certainty";
    +  public static final String QUERYNAME = "querier.queryName";
    +  public static final String QUERYSCHEMAS = "querier.querySchemas";
    +  public static final String DATASCHEMAS = "querier.dataSchemas";
    +  public static final String EMBEDSELECTOR = "querier.embedSelector";
    +  public static final String USEMEMLOOKUPTABLE = "querier.memLookupTable";
    +  public static final String USEHDFSLOOKUPTABLE = "querier.useHDFSLookupTable";
    +  public static final String SR_ALGORITHM = "pallier.secureRandom.algorithm";
    +  public static final String SR_PROVIDER = "pallier.secureRandom.provider";
    +  public static final String EMBEDQUERYSCHEMA = "pir.embedQuerySchema";
    +
    +  // Decryption properties
    +  public static final String QUERIERFILE = "querier.querierFile";
    +
    +  public static final List<String> PROPSLIST = Arrays.asList(ACTION, INPUTFILE, OUTPUTFILE, QUERYTYPE, NUMTHREADS, EMBEDQUERYSCHEMA, HASHBITSIZE, HASHKEY,
    +      DATAPARTITIONSIZE, PAILLIERBITSIZE, BITSET, CERTAINTY, QUERYNAME, QUERYSCHEMAS, DATASCHEMAS, EMBEDSELECTOR, USEMEMLOOKUPTABLE, USEHDFSLOOKUPTABLE,
    +      SR_ALGORITHM, SR_PROVIDER);
    +
    +  /**
    +   * Validates the querier properties
    +   * 
    +   */
    +  public static boolean validateQuerierProperties()
    --- End diff --
    
    Good idea


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Prope...

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/27#discussion_r72588602
  
    --- Diff: src/main/resources/pirk.properties ---
    @@ -31,7 +31,8 @@ local.pirk.properties=/root/local.pirk.properties
     ##
     ##Spark path for SparkLauncher
     ##
    -spark.home = /usr
    +#spark.home = /usr
    +spark.home=/opt/cloudera/parcels/CDH/lib/spark
    --- End diff --
    
    Huh? :-)


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property Files

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

    https://github.com/apache/incubator-pirk/pull/27
  
    Did some cleaning as per Tim's comments...


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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/27#discussion_r72699712
  
    --- Diff: src/main/java/org/apache/pirk/responder/wideskies/ResponderProps.java ---
    @@ -0,0 +1,212 @@
    +package org.apache.pirk.responder.wideskies;
    --- End diff --
    
    Apache header?


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Prope...

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/27#discussion_r72586620
  
    --- Diff: src/main/java/org/apache/pirk/utils/SystemConfiguration.java ---
    @@ -24,8 +24,6 @@
     import java.io.InputStream;
     import java.util.Properties;
     
    -import org.apache.pirk.schema.data.LoadDataSchemas;
    -import org.apache.pirk.schema.query.LoadQuerySchemas;
    --- End diff --
    
    Hurray!


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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/27#discussion_r72701970
  
    --- Diff: src/main/java/org/apache/pirk/utils/SystemConfiguration.java ---
    @@ -160,4 +139,78 @@ public static void resetProperties()
         clearProperties();
         initialize();
       }
    +
    +  /**
    +   * Loads the properties from local properties file in the specified directory
    +   * <p>
    +   * Only files ending in '.properties' will be loaded
    +   */
    +  public static void loadPropsFromDir(String dirName)
    +  {
    +    File[] directoryListing = new File(dirName).listFiles(new FilenameFilter()
    +    {
    +      @Override
    +      public boolean accept(File dir, String name)
    +      {
    +        return name.endsWith(".properties");
    +      }
    +    });
    +
    +    if (directoryListing != null)
    +    {
    +      for (File file : directoryListing)
    +      {
    +        loadPropsFromFile(file);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Loads the properties from the specified file
    +   */
    +  public static void loadPropsFromFile(File file)
    +  {
    +    if (file.exists())
    +    {
    +      try (InputStream stream = new FileInputStream(file);)
    +      {
    +        logger.info("Loading properties file '" + file.getAbsolutePath() + "'");
    +        props.load(stream);
    +        stream.close();
    +      } catch (IOException e)
    +      {
    +        logger.error("Problem loading properties file '" + file.getAbsolutePath() + "'");
    +        e.printStackTrace();
    +      }
    +    }
    +    else
    +    {
    +      logger.warn("Properties file does not exist: '" + file.getAbsolutePath() + "'");
    +    }
    +  }
    +
    +  /**
    +   * Loads the properties from the specified file on the classpath
    +   */
    +  public static void loadPropsFromStream(String name)
    +  {
    +    try
    +    {
    +      InputStream stream = SystemConfiguration.class.getClassLoader().getResourceAsStream(name);
    +      if (stream != null)
    +      {
    +        logger.info("Loading file '" + name + "'");
    +        props.load(stream);
    +        stream.close();
    --- End diff --
    
    Can use try-with-resources to ensure the stream is closed even if the load causes an exception.


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Prope...

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/27#discussion_r72594248
  
    --- Diff: src/main/java/org/apache/pirk/utils/SystemConfiguration.java ---
    @@ -152,6 +105,31 @@ public static void setProperty(String propertyName, String value)
         props.setProperty(propertyName, value);
       }
     
    +  public static boolean hasProperty(String propertyName)
    +  {
    +    return props.containsKey(propertyName);
    +  }
    +
    +  /**
    +   * Append a property via a column separated list
    --- End diff --
    
    Yes, just a typo - thanks for pointing it out


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property Files

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

    https://github.com/apache/incubator-pirk/pull/27
  
    +1 to merge.


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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

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


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Prope...

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/27#discussion_r72594279
  
    --- Diff: src/main/java/org/apache/pirk/utils/SystemConfiguration.java ---
    @@ -160,4 +138,74 @@ public static void resetProperties()
         clearProperties();
         initialize();
       }
    +
    +  /**
    +   * Loads the properties from local properties file in the specified directory
    +   * <p>
    +   * Only files ending in '.properties' will be loaded
    +   */
    +  public static void loadPropsFromDir(String dirName)
    +  {
    +    File dir = new File(dirName);
    +    File[] directoryListing = dir.listFiles();
    --- End diff --
    
    Good idea


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Prope...

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/27#discussion_r72588302
  
    --- Diff: src/main/java/org/apache/pirk/utils/SystemConfiguration.java ---
    @@ -160,4 +138,74 @@ public static void resetProperties()
         clearProperties();
         initialize();
       }
    +
    +  /**
    +   * Loads the properties from local properties file in the specified directory
    +   * <p>
    +   * Only files ending in '.properties' will be loaded
    +   */
    +  public static void loadPropsFromDir(String dirName)
    +  {
    +    File dir = new File(dirName);
    +    File[] directoryListing = dir.listFiles();
    --- End diff --
    
    You could use File#list(FilenameFilter), but we are not expecting a huge number of files in the dir list, this works fine.
    
    ```
            File[] directoryListing = new File(dirName).listFiles(new FilenameFilter() {
                @Override
                public boolean accept(File dir, String name) {
                    return name.endsWith(".properties");
                }
            });
    ```


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Prope...

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/27#discussion_r72594185
  
    --- Diff: src/main/java/org/apache/pirk/utils/SystemConfiguration.java ---
    @@ -24,8 +24,6 @@
     import java.io.InputStream;
     import java.util.Properties;
     
    -import org.apache.pirk.schema.data.LoadDataSchemas;
    -import org.apache.pirk.schema.query.LoadQuerySchemas;
    --- End diff --
    
    Lol :)


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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/27#discussion_r72699845
  
    --- Diff: src/main/java/org/apache/pirk/querier/wideskies/QuerierProps.java ---
    @@ -0,0 +1,202 @@
    +package org.apache.pirk.querier.wideskies;
    --- End diff --
    
    No - it didn't fail the checks.... Source headers added - good catch


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Property Fil...

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

    https://github.com/apache/incubator-pirk/pull/27
  
    I also still need to make this branch up-to-date with master (and all of the query and data schema refactoring) before pulling the WIP off. I will remove the scripts in the bin folder too, unless folks object (I don't think they are needed)


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Prope...

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/27#discussion_r72592297
  
    --- Diff: src/main/java/org/apache/pirk/querier/wideskies/QuerierProps.java ---
    @@ -0,0 +1,202 @@
    +package org.apache.pirk.querier.wideskies;
    +
    +import java.util.Arrays;
    +import java.util.List;
    +
    +import org.apache.pirk.schema.data.LoadDataSchemas;
    +import org.apache.pirk.schema.query.LoadQuerySchemas;
    +import org.apache.pirk.utils.SystemConfiguration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * Properties constants and validation for the Querier
    + */
    +public class QuerierProps
    +{
    +  private static final Logger logger = LoggerFactory.getLogger(QuerierProps.class);
    +
    +  // General properties
    +  public static final String ACTION = "querier.action";
    +  public static final String INPUTFILE = "querier.inputFile";
    +  public static final String OUTPUTFILE = "querier.outputFile";
    +  public static final String QUERYTYPE = "querier.queryType";
    +  public static final String NUMTHREADS = "querier.numThreads";
    +
    +  // Encryption properties
    +  public static final String HASHBITSIZE = "querier.hashBitSize";
    +  public static final String HASHKEY = "querier.hashKey";
    +  public static final String DATAPARTITIONSIZE = "querier.dataPartitionBitSize";
    +  public static final String PAILLIERBITSIZE = "querier.paillierBitSize";
    +  public static final String BITSET = "querier.bitSet";
    +  public static final String CERTAINTY = "querier.certainty";
    +  public static final String QUERYNAME = "querier.queryName";
    +  public static final String QUERYSCHEMAS = "querier.querySchemas";
    +  public static final String DATASCHEMAS = "querier.dataSchemas";
    +  public static final String EMBEDSELECTOR = "querier.embedSelector";
    +  public static final String USEMEMLOOKUPTABLE = "querier.memLookupTable";
    +  public static final String USEHDFSLOOKUPTABLE = "querier.useHDFSLookupTable";
    +  public static final String SR_ALGORITHM = "pallier.secureRandom.algorithm";
    +  public static final String SR_PROVIDER = "pallier.secureRandom.provider";
    +  public static final String EMBEDQUERYSCHEMA = "pir.embedQuerySchema";
    +
    +  // Decryption properties
    +  public static final String QUERIERFILE = "querier.querierFile";
    +
    +  public static final List<String> PROPSLIST = Arrays.asList(ACTION, INPUTFILE, OUTPUTFILE, QUERYTYPE, NUMTHREADS, EMBEDQUERYSCHEMA, HASHBITSIZE, HASHKEY,
    +      DATAPARTITIONSIZE, PAILLIERBITSIZE, BITSET, CERTAINTY, QUERYNAME, QUERYSCHEMAS, DATASCHEMAS, EMBEDSELECTOR, USEMEMLOOKUPTABLE, USEHDFSLOOKUPTABLE,
    +      SR_ALGORITHM, SR_PROVIDER);
    +
    +  /**
    +   * Validates the querier properties
    +   * 
    +   */
    +  public static boolean validateQuerierProperties()
    --- End diff --
    
    How about checking the required properties in bulk? e.g. something like...
    
    ```
            static final String[] REQUIRED_PROPERTIES = new String[] {QUERYTYPE, HASHBITSIZE, ETC}; 
            ....
            List<String> missingProperties = Arrays.asList(REQUIRED_PROPERTIES).removeAll(SystemConfiguration.getPropertyNames());
            boolean valid = missingProperties.isEmpty(); 
            for (String property : missingProperties) {
              logger.info("Must have the option " + property);
            }
    ```
    (You will have to persuade ```SystemConfiguration``` to tell you all the property names.)


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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/27#discussion_r72703868
  
    --- Diff: src/main/java/org/apache/pirk/querier/wideskies/QuerierProps.java ---
    @@ -0,0 +1,202 @@
    +package org.apache.pirk.querier.wideskies;
    --- End diff --
    
    Hmm, need to investigate why RAT checks are not working.
    I can take a look tomorrow.


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Prope...

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/27#discussion_r72589271
  
    --- Diff: pom.xml ---
    @@ -117,6 +117,7 @@
     			<version>3.3</version>
     		</dependency>
     
    +<!--
    --- End diff --
    
    Not sure what's happening here...


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Property Fil...

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

    https://github.com/apache/incubator-pirk/pull/27
  
    Updating this PR to include querier.properties and responder.properties and reworking CLI and property validation


---
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 #27: [WIP] Pirk 27 - Improve Pirk's Use of Prope...

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/27#discussion_r72588448
  
    --- Diff: src/main/java/org/apache/pirk/utils/SystemConfiguration.java ---
    @@ -160,4 +138,74 @@ public static void resetProperties()
         clearProperties();
         initialize();
       }
    +
    +  /**
    +   * Loads the properties from local properties file in the specified directory
    +   * <p>
    +   * Only files ending in '.properties' will be loaded
    +   */
    +  public static void loadPropsFromDir(String dirName)
    +  {
    +    File dir = new File(dirName);
    +    File[] directoryListing = dir.listFiles();
    +    if (directoryListing != null)
    +    {
    +      for (File file : directoryListing)
    +      {
    +        if (file.getName().endsWith(".properties"))
    +        {
    +          loadPropsFromFile(file);
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Loads the properties from the specified file
    +   */
    +  public static void loadPropsFromFile(File file)
    +  {
    +    if (file.exists())
    +    {
    +      try (InputStream stream = new FileInputStream(file);)
    +      {
    +        logger.info("Loading properties file '" + file.getAbsolutePath() + "'");
    +        props.load(stream);
    +        stream.close();
    --- End diff --
    
    No need to explicitly close() the stream now, the try-with-resources does that for you.


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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/27#discussion_r72698895
  
    --- Diff: src/main/java/org/apache/pirk/querier/wideskies/QuerierProps.java ---
    @@ -0,0 +1,202 @@
    +package org.apache.pirk.querier.wideskies;
    --- End diff --
    
    Apache source header?
    (Didn't this fail the RAT checks?)


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property Files

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

    https://github.com/apache/incubator-pirk/pull/27
  
    This is ready for review


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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/27#discussion_r72716104
  
    --- Diff: src/main/resources/querier.properties ---
    @@ -0,0 +1,123 @@
    +###############################################################################
    +# 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.
    +###############################################################################
    +
    +##
    +## Required Properties
    +##
    +
    +#action -- required - 'encrypt' or 'decrypt' -- The action performed by the QuerierDriver
    +querier.action=
    +
    +#inputFile - required - Fully qualified file containing input
    +#The input is either:
    +#(1) For Encryption: A query file - Contains the query selectors, one per line;
    +#the first line must be the query number
    +#OR
    +#(2) For Decryption: A response file - Contains the serialized Response object
    +querier.inputFile=
    +
    +#outputFile -- required - Fully qualified file for the result output.
    +#The output file specifies either:
    +#(1) For encryption:
    +#(a) A file to contain the serialized Querier object named: <outputFile>-querier
    +#AND
    +#(b) A file to contain the serialized Query object named: <outputFile>-query
    +#OR
    +#(2) A file to contain the decryption results where each line is where each line
    +#corresponds to one hit and is a JSON object with the schema QuerySchema
    +querier.outputFile=
    +
    +#numThreads -- required -- Number of threads to use for encryption/decryption
    +querier.numThreads=
    --- End diff --
    
    Best to have the user specify, in my opinion. 


---
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 #27: Pirk 27 - Improve Pirk's Use of Property Files

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

    https://github.com/apache/incubator-pirk/pull/27
  
    +1


---
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 #27: [Pirk 27] - Improve Pirk's Use of Property ...

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/27#discussion_r72702440
  
    --- Diff: src/main/resources/pirk.properties-repo ---
    @@ -0,0 +1,243 @@
    +###############################################################################
    +# 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.
    +###############################################################################
    +
    +##
    +## Default System Configuration Properties
    +##
    +
    +# Name of log4j properties file (relative to current folder)
    +log4jPropertiesFile=log4j2.properties
    +
    +#Name of the directory holding the local properties files
    +#All property files must end in .properties
    +local.pirk.properties.dir=/root/properties
    +
    +##
    +##Spark path for SparkLauncher
    +##
    +spark.home = /usr
    +
    +##
    +## Data schema properties
    +##
    +## Each data schema should be specified in an xml file of the form; 
    +## 		all items are treated in a case insensitive manner:
    +##
    +##<schema>
    +##  <schemaName> name of the schema </schemaName>
    +##  <element>
    +##      <name> element name </name>
    +##      <type> class name or type name (if Java primitive type) of the element </type>
    +##      <isArray> true or false -- whether or not the schema element is an array within the data </isArray>
    +##      <partitioner> optional - Partitioner class for the element; defaults to primitive java type partitioner </partitioner> 
    +##  </element>
    +## </schema>
    +##
    +## Primitive Java types must be one of the following: "byte", "short", "int", "long", "float", 
    +##                  "double", "char", "string", "boolean"
    +##
    +
    +#Comma separated list of local data schema files to load, fully qualified file names
    +data.schemas = none
    +
    +##
    +## Query schema properties
    +##
    +## Each query schema should be specified in an xml file of the form; 
    +## 		all items are treated in a case insensitive manner:
    --- End diff --
    
    As above.


---
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.
---