You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@rya.apache.org by kchilton2 <gi...@git.apache.org> on 2016/05/02 18:27:22 UTC

[GitHub] incubator-rya pull request: Rya 64

GitHub user kchilton2 opened a pull request:

    https://github.com/apache/incubator-rya/pull/38

    Rya 64

    

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

    $ git pull https://github.com/kchilton2/incubator-rya RYA-64

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

    https://github.com/apache/incubator-rya/pull/38.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 #38
    
----
commit 7eb678879f8f700f000d630c83170771e318167b
Author: isper3at <sm...@gmail.com>
Date:   2016-03-21T16:55:56Z

    RYA-53 Added visibility support
    
    Visibility support added for PCJ tables.
    If no visibilities are defined during the query,
    only binding sets visible to the accesssor will
    be returned.
    
    Added and updated tests for PCJ visibility.
    
    Added visibility support for Fluo backed Rya
    Added integration tests for inserting
    triples through fluo into PCJs with
    visibility.
    
    Added a Decorator pattern for BindingSets

commit 6d579329c0ad02b2c3750411d19508ad41a92ad8
Author: Kevin Chilton <ke...@localhost.localdomain>
Date:   2016-04-14T14:34:40Z

    RYA-64 - Integrated Rya PCJ Secondary Index support into core Rya.

----


---
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-rya pull request: Rya 64

Posted by pujav65 <gi...@git.apache.org>.
Github user pujav65 commented on the pull request:

    https://github.com/apache/incubator-rya/pull/38#issuecomment-219045554
  
    hey kevin, looks like this conflicted with some pull requests committed before yours.  Can you please resolve the conflicts and then I'll merge the pr in?


---
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-rya pull request: Rya 64

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

    https://github.com/apache/incubator-rya/pull/38#discussion_r63053214
  
    --- Diff: extras/indexing/src/test/java/mvm/rya/indexing/external/PcjIntegrationTestingUtil.java ---
    @@ -25,22 +25,6 @@
     import java.util.HashSet;
    --- End diff --
    
    another try to avoid making non-meaningful changes comment -- makes it hard to 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-rya pull request: Rya 64

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

    https://github.com/apache/incubator-rya/pull/38#discussion_r63052862
  
    --- Diff: extras/indexing/src/main/java/mvm/rya/indexing/external/accumulo/AccumuloPcjStorageConfig.java ---
    @@ -0,0 +1,57 @@
    +/*
    + * 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 mvm.rya.indexing.external.accumulo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import mvm.rya.api.RdfCloudTripleStoreConfiguration;
    +
    +/**
    + * Configuration values required to initialize a {@link AccumuloPcjStorage}.
    + */
    +public class AccumuloPcjStorageConfig {
    +
    +    private final RdfCloudTripleStoreConfiguration config;
    +
    +    /**
    +     * Constructs an instance of {@link AccumuloPcjStorageConfig}.
    +     *
    +     * @param config - The configuration values that will be interpreted. (not null)
    +     */
    +    public AccumuloPcjStorageConfig(final Configuration config) {
    +        checkNotNull(config);
    +
    +        // Wrapping the config with this class so that we can use it's getTablePrefix() method.
    +        this.config = new RdfCloudTripleStoreConfiguration(config) {
    +            @Override
    +            public RdfCloudTripleStoreConfiguration clone() {
    +                return null;
    +            }
    +        };
    +    }
    +
    +    /**
    +     * @return The Rya Instance name the storage grants access to.
    +     */
    +    public String getRyaInstanceName() {
    --- End diff --
    
    any reason you didn't add this to rdfcloudtriplestorconfiguration?


---
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-rya pull request: Rya 64

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

    https://github.com/apache/incubator-rya/pull/38#discussion_r63052257
  
    --- Diff: extras/indexing/src/main/java/mvm/rya/indexing/accumulo/ConfigUtils.java ---
    @@ -118,22 +118,22 @@
         public static final String GEO_PREDICATES_LIST = "sc.geo.predicates";
     
         public static final String TEMPORAL_PREDICATES_LIST = "sc.temporal.predicates";
    -    
    +
         public static final String USE_MONGO = "sc.useMongo";
     
    -    public static boolean isDisplayQueryPlan(Configuration conf){
    +    public static boolean isDisplayQueryPlan(final Configuration conf){
             return conf.getBoolean(DISPLAY_QUERY_PLAN, false);
         }
    -    
    +
         /**
          * get a value from the configuration file and throw an exception if the value does not exist.
    -     * 
    +     *
          * @param conf
          * @param key
          * @return
          */
    -    private static String getStringCheckSet(Configuration conf, String key) {
    -        String value = conf.get(key);
    +    private static String getStringCheckSet(final Configuration conf, final String key) {
    --- End diff --
    
    don't add final for the sake of adding final.  makes it hard to see what really changed.


---
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-rya pull request: Rya 64

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

    https://github.com/apache/incubator-rya/pull/38#discussion_r63052396
  
    --- Diff: extras/indexing/src/main/java/mvm/rya/indexing/external/PrecomputedJoinIndexer.java ---
    @@ -0,0 +1,255 @@
    +/*
    + * 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 mvm.rya.indexing.external;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Collections;
    +
    +import javax.annotation.ParametersAreNonnullByDefault;
    +
    +import org.apache.accumulo.core.client.Connector;
    +import org.apache.accumulo.core.client.MultiTableBatchWriter;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.update.PrecomputedJoinUpdater;
    +import org.apache.rya.indexing.pcj.update.PrecomputedJoinUpdater.PcjUpdateException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Supplier;
    +
    +import mvm.rya.accumulo.experimental.AccumuloIndexer;
    +import mvm.rya.api.RdfCloudTripleStoreConfiguration;
    +import mvm.rya.api.domain.RyaStatement;
    +import mvm.rya.api.domain.RyaURI;
    +import mvm.rya.api.persist.RyaDAO;
    +import mvm.rya.indexing.external.accumulo.AccumuloPcjStorage;
    +import mvm.rya.indexing.external.accumulo.AccumuloPcjStorageSupplier;
    +import mvm.rya.indexing.external.fluo.FluoPcjUpdaterSupplier;
    +
    +/**
    + * Updates the state of the Precomputed Join indices that are used by Rya.
    + */
    +@ParametersAreNonnullByDefault
    +public class PrecomputedJoinIndexer implements AccumuloIndexer {
    +    private static final Logger log = Logger.getLogger(PrecomputedJoinIndexer.class);
    +
    +    /**
    +     * This configuration object must be set before {@link #init()} is invoked.
    +     * It is set by {@link #setConf(Configuration)}.
    +     */
    +    private Optional<Configuration> conf = Optional.absent();
    +
    +    /**
    +     * The Accumulo Connector that must be used when accessing an Accumulo storage.
    +     * This value is provided by {@link #setConnector(Connector)}.
    +     */
    +    private Optional<Connector> accumuloConn = Optional.absent();
    +
    +    /**
    +     * Provides access to the {@link Configuration} that was provided to this class
    +     * using {@link #setConf(Configuration)}.
    +     */
    +    private final Supplier<Configuration> configSupplier = new Supplier<Configuration>() {
    +        @Override
    +        public Configuration get() {
    +            return getConf();
    +        }
    +    };
    +
    +    /**
    +     * Provides access to the Accumulo {@link Connector} that was provided to
    +     * this class using {@link #setConnector(Connector)}.
    +     */
    +    private final Supplier<Connector> accumuloSupplier = new Supplier<Connector>() {
    +        @Override
    +        public Connector get() {
    +            return accumuloConn.get();
    +        }
    +    };
    +
    +    /**
    +     * Creates and grants access to the {@link PrecomputedJoinStorage} that will be used
    +     * to interact with the PCJ results that are stored and used by Rya.
    +     */
    +    private final PrecomputedJoinStorageSupplier pcjStorageSupplier =
    +            new PrecomputedJoinStorageSupplier(
    +                    configSupplier,
    +                    new AccumuloPcjStorageSupplier(configSupplier, accumuloSupplier));
    +
    +    /**
    +     * Creates and grants access to the {@link PrecomputedJoinUpdater} that will
    +     * be used to update the state stored within the PCJ tables that are stored
    +     * in Accumulo.
    +     */
    +    private final PrecomputedJoinUpdaterSupplier pcjUpdaterSupplier =
    +            new PrecomputedJoinUpdaterSupplier(
    +                    configSupplier,
    +                    new FluoPcjUpdaterSupplier(configSupplier));
    +
    +    @Override
    +    public void setConf(final Configuration conf) {
    +        this.conf = Optional.fromNullable(conf);
    +    }
    +
    +    @Override
    +    public Configuration getConf() {
    +        return this.conf.get();
    +    }
    +
    +    /**
    +     * Set the connector that will be used by {@link AccumuloPcjStorage} if the
    +     * application is configured to store the PCJs within Accumulo.
    +     */
    +    @Override
    +    public void setConnector(final Connector connector) {
    +        checkNotNull(connector);
    +        accumuloConn = Optional.of( connector );
    +    }
    +
    +    /**
    +     * This is invoked when the host {@link RyaDAO#init()} method is invoked.
    +     */
    +    @Override
    +    public void init() {
    +        pcjStorageSupplier.get();
    +        pcjUpdaterSupplier.get();
    +    }
    +
    +    @Override
    +    public void storeStatement(final RyaStatement statement) throws IOException {
    +        checkNotNull(statement);
    +        storeStatements( Collections.singleton(statement) );
    +    }
    +
    +    @Override
    +    public void storeStatements(final Collection<RyaStatement> statements) throws IOException {
    +        checkNotNull(statements);
    +        try {
    +            pcjUpdaterSupplier.get().addStatements(statements);
    +        } catch (final PcjUpdateException e) {
    +            throw new IOException("Could not update the PCJs by adding the provided statements.", e);
    +        }
    +    }
    +
    +    @Override
    +    public void deleteStatement(final RyaStatement statement) throws IOException {
    +        checkNotNull(statement);
    +        try {
    +            pcjUpdaterSupplier.get().deleteStatements( Collections.singleton(statement) );
    --- End diff --
    
    does this do anything now?


---
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-rya pull request: Rya 64

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

    https://github.com/apache/incubator-rya/pull/38


---
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-rya pull request: Rya 64

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

    https://github.com/apache/incubator-rya/pull/38#discussion_r63217600
  
    --- Diff: extras/indexing/src/main/java/mvm/rya/indexing/external/PrecomputedJoinIndexer.java ---
    @@ -0,0 +1,255 @@
    +/*
    + * 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 mvm.rya.indexing.external;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Collections;
    +
    +import javax.annotation.ParametersAreNonnullByDefault;
    +
    +import org.apache.accumulo.core.client.Connector;
    +import org.apache.accumulo.core.client.MultiTableBatchWriter;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
    +import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
    +import org.apache.rya.indexing.pcj.update.PrecomputedJoinUpdater;
    +import org.apache.rya.indexing.pcj.update.PrecomputedJoinUpdater.PcjUpdateException;
    +
    +import com.google.common.base.Optional;
    +import com.google.common.base.Supplier;
    +
    +import mvm.rya.accumulo.experimental.AccumuloIndexer;
    +import mvm.rya.api.RdfCloudTripleStoreConfiguration;
    +import mvm.rya.api.domain.RyaStatement;
    +import mvm.rya.api.domain.RyaURI;
    +import mvm.rya.api.persist.RyaDAO;
    +import mvm.rya.indexing.external.accumulo.AccumuloPcjStorage;
    +import mvm.rya.indexing.external.accumulo.AccumuloPcjStorageSupplier;
    +import mvm.rya.indexing.external.fluo.FluoPcjUpdaterSupplier;
    +
    +/**
    + * Updates the state of the Precomputed Join indices that are used by Rya.
    + */
    +@ParametersAreNonnullByDefault
    +public class PrecomputedJoinIndexer implements AccumuloIndexer {
    +    private static final Logger log = Logger.getLogger(PrecomputedJoinIndexer.class);
    +
    +    /**
    +     * This configuration object must be set before {@link #init()} is invoked.
    +     * It is set by {@link #setConf(Configuration)}.
    +     */
    +    private Optional<Configuration> conf = Optional.absent();
    +
    +    /**
    +     * The Accumulo Connector that must be used when accessing an Accumulo storage.
    +     * This value is provided by {@link #setConnector(Connector)}.
    +     */
    +    private Optional<Connector> accumuloConn = Optional.absent();
    +
    +    /**
    +     * Provides access to the {@link Configuration} that was provided to this class
    +     * using {@link #setConf(Configuration)}.
    +     */
    +    private final Supplier<Configuration> configSupplier = new Supplier<Configuration>() {
    +        @Override
    +        public Configuration get() {
    +            return getConf();
    +        }
    +    };
    +
    +    /**
    +     * Provides access to the Accumulo {@link Connector} that was provided to
    +     * this class using {@link #setConnector(Connector)}.
    +     */
    +    private final Supplier<Connector> accumuloSupplier = new Supplier<Connector>() {
    +        @Override
    +        public Connector get() {
    +            return accumuloConn.get();
    +        }
    +    };
    +
    +    /**
    +     * Creates and grants access to the {@link PrecomputedJoinStorage} that will be used
    +     * to interact with the PCJ results that are stored and used by Rya.
    +     */
    +    private final PrecomputedJoinStorageSupplier pcjStorageSupplier =
    +            new PrecomputedJoinStorageSupplier(
    +                    configSupplier,
    +                    new AccumuloPcjStorageSupplier(configSupplier, accumuloSupplier));
    +
    +    /**
    +     * Creates and grants access to the {@link PrecomputedJoinUpdater} that will
    +     * be used to update the state stored within the PCJ tables that are stored
    +     * in Accumulo.
    +     */
    +    private final PrecomputedJoinUpdaterSupplier pcjUpdaterSupplier =
    +            new PrecomputedJoinUpdaterSupplier(
    +                    configSupplier,
    +                    new FluoPcjUpdaterSupplier(configSupplier));
    +
    +    @Override
    +    public void setConf(final Configuration conf) {
    +        this.conf = Optional.fromNullable(conf);
    +    }
    +
    +    @Override
    +    public Configuration getConf() {
    +        return this.conf.get();
    +    }
    +
    +    /**
    +     * Set the connector that will be used by {@link AccumuloPcjStorage} if the
    +     * application is configured to store the PCJs within Accumulo.
    +     */
    +    @Override
    +    public void setConnector(final Connector connector) {
    +        checkNotNull(connector);
    +        accumuloConn = Optional.of( connector );
    +    }
    +
    +    /**
    +     * This is invoked when the host {@link RyaDAO#init()} method is invoked.
    +     */
    +    @Override
    +    public void init() {
    +        pcjStorageSupplier.get();
    +        pcjUpdaterSupplier.get();
    +    }
    +
    +    @Override
    +    public void storeStatement(final RyaStatement statement) throws IOException {
    +        checkNotNull(statement);
    +        storeStatements( Collections.singleton(statement) );
    +    }
    +
    +    @Override
    +    public void storeStatements(final Collection<RyaStatement> statements) throws IOException {
    +        checkNotNull(statements);
    +        try {
    +            pcjUpdaterSupplier.get().addStatements(statements);
    +        } catch (final PcjUpdateException e) {
    +            throw new IOException("Could not update the PCJs by adding the provided statements.", e);
    +        }
    +    }
    +
    +    @Override
    +    public void deleteStatement(final RyaStatement statement) throws IOException {
    +        checkNotNull(statement);
    +        try {
    +            pcjUpdaterSupplier.get().deleteStatements( Collections.singleton(statement) );
    --- End diff --
    
    If you provide a Fluo PcjUpdater, then it just logs that delete isn't supported. In theory another implementation could support deletes (like, maybe this triggers a batch job), so it's appropriate for the indexer to pass the delete command along to the updater.


---
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-rya pull request: Rya 64

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

    https://github.com/apache/incubator-rya/pull/38#discussion_r63217692
  
    --- Diff: extras/indexing/src/main/java/mvm/rya/indexing/external/accumulo/AccumuloPcjStorageConfig.java ---
    @@ -0,0 +1,57 @@
    +/*
    + * 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 mvm.rya.indexing.external.accumulo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import mvm.rya.api.RdfCloudTripleStoreConfiguration;
    +
    +/**
    + * Configuration values required to initialize a {@link AccumuloPcjStorage}.
    + */
    +public class AccumuloPcjStorageConfig {
    +
    +    private final RdfCloudTripleStoreConfiguration config;
    +
    +    /**
    +     * Constructs an instance of {@link AccumuloPcjStorageConfig}.
    +     *
    +     * @param config - The configuration values that will be interpreted. (not null)
    +     */
    +    public AccumuloPcjStorageConfig(final Configuration config) {
    +        checkNotNull(config);
    +
    +        // Wrapping the config with this class so that we can use it's getTablePrefix() method.
    +        this.config = new RdfCloudTripleStoreConfiguration(config) {
    +            @Override
    +            public RdfCloudTripleStoreConfiguration clone() {
    +                return null;
    +            }
    +        };
    +    }
    +
    +    /**
    +     * @return The Rya Instance name the storage grants access to.
    +     */
    +    public String getRyaInstanceName() {
    --- End diff --
    
    Didn't add what?


---
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-rya pull request: Rya 64

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

    https://github.com/apache/incubator-rya/pull/38#discussion_r63053017
  
    --- Diff: extras/indexing/src/main/java/mvm/rya/indexing/external/fluo/FluoPcjUpdaterConfig.java ---
    @@ -0,0 +1,120 @@
    +/*
    + * 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 mvm.rya.indexing.external.fluo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.google.common.base.Optional;
    +
    +import mvm.rya.indexing.accumulo.ConfigUtils;
    +
    +/**
    + * Configuration values required to initialize a {@link FluoPcjUpdater}.
    + */
    +public final class FluoPcjUpdaterConfig {
    +
    +    // Defines which Fluo application is running for this instance of Rya.
    +    public static final String FLUO_APP_NAME = "rya.indexing.pcj.fluo.fluoAppName";
    +
    +    // Values that define which Accumulo instance hosts the Fluo application's table.
    +    public static final String ACCUMULO_ZOOKEEPERS = ConfigUtils.CLOUDBASE_ZOOKEEPERS;
    --- End diff --
    
    no related to this, but we should consider changing those config constants


---
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-rya pull request: Rya 64

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

    https://github.com/apache/incubator-rya/pull/38#discussion_r63217949
  
    --- Diff: extras/indexing/src/main/java/mvm/rya/indexing/external/fluo/FluoPcjUpdaterConfig.java ---
    @@ -0,0 +1,120 @@
    +/*
    + * 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 mvm.rya.indexing.external.fluo;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import org.apache.hadoop.conf.Configuration;
    +
    +import com.google.common.base.Optional;
    +
    +import mvm.rya.indexing.accumulo.ConfigUtils;
    +
    +/**
    + * Configuration values required to initialize a {@link FluoPcjUpdater}.
    + */
    +public final class FluoPcjUpdaterConfig {
    +
    +    // Defines which Fluo application is running for this instance of Rya.
    +    public static final String FLUO_APP_NAME = "rya.indexing.pcj.fluo.fluoAppName";
    +
    +    // Values that define which Accumulo instance hosts the Fluo application's table.
    +    public static final String ACCUMULO_ZOOKEEPERS = ConfigUtils.CLOUDBASE_ZOOKEEPERS;
    --- End diff --
    
    We could add a Jira Improvement ticket to do that if you would like to update them in the future.


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