[ 
https://issues.apache.org/jira/browse/RYA-246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16138574#comment-16138574
 ] 

ASF GitHub Bot commented on RYA-246:
------------------------------------

Github user meiercaleb commented on a diff in the pull request:

    https://github.com/apache/incubator-rya/pull/213#discussion_r134799456
  
    --- Diff: 
extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/kafka/KafkaSubGraphExporterParameters.java
 ---
    @@ -0,0 +1,79 @@
    +/*
    + * 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.rya.indexing.pcj.fluo.app.export.kafka;
    +
    +import java.util.Map;
    +import java.util.Properties;
    +
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.common.serialization.StringSerializer;
    +
    +
    +public class KafkaSubGraphExporterParameters extends 
KafkaExportParameterBase {
    +
    +    public static final String CONF_USE_KAFKA_SUBGRAPH_EXPORTER = 
"pcj.fluo.export.kafka.subgraph.enabled";
    +    public static final String CONF_KAFKA_SUBGRAPH_SERIALIZER = 
"pcj.fluo.export.kafka.subgraph.serializer";
    +    
    +    public KafkaSubGraphExporterParameters(final Map<String, String> 
params) {
    +        super(params);
    +    }
    +    
    +    /**
    +     * Instructs the Fluo application to use the Kafka BindingSet Exporter
    +     * and sets the appropriate Key/Value Serializer parameters for 
writing RyaSubGraphs to Kafka.
    +     * @param useExporter
    +     *            - {@code True} if the Fluo application should use the
    +     *            {@link KafkaRyaSubGraphExporter}; otherwise {@code 
false}.
    +     */
    +    public void setUseKafkaSubgraphExporter(final boolean useExporter) {
    +        setBoolean(params, CONF_USE_KAFKA_SUBGRAPH_EXPORTER, useExporter);
    +    }
    +
    +    /**
    +     * @return {@code True} if the Fluo application should use the {@link 
KafkaRyaSubGraphExporter}; otherwise
    +     *         {@code false}. Defaults to {@code false} if no value is 
present.
    +     */
    +    public boolean getUseKafkaSubgraphExporter() {
    +        return getBoolean(params, CONF_USE_KAFKA_SUBGRAPH_EXPORTER, false);
    +    }
    +    
    +    /**
    +     * 
    +     * @param serializer - Used for Serializing RyaSubGraphs pushed to 
Kafka
    +     */
    +    public void setKafkaSubGraphSerializer(String serializer) {
    +        params.put(CONF_KAFKA_SUBGRAPH_SERIALIZER, serializer);
    --- End diff --
    
    Null  check.


> Separate Fluo query result exporters, Rya PCJ intermingled kafka triggers
> -------------------------------------------------------------------------
>
>                 Key: RYA-246
>                 URL: https://issues.apache.org/jira/browse/RYA-246
>             Project: Rya
>          Issue Type: Improvement
>          Components: dao
>         Environment: Fluo, Kafka, PCJ, Accumulo
>            Reporter: David W. Lotts
>            Assignee: Caleb Meier
>             Fix For: 3.2.11
>
>
> Currently the Fluo query observer framework (FQF) has a great plugin-in-ish 
> way of doing exports.  (See field: {{exporters}} in class: 
> {{org.apache.rya.indexing.pcj.fluo.app.observers.QueryResultObserver}} )  
> Exports are query results and the action taken when they occur.  Currently 
> there are two.  Despite this awesome framework, there are three 
> interminglings that can become issues in a production system.  Feel free to 
> make sub tasks for each of these three tasks:
> 1. Create PCJ independently of alerts/triggers
> At query register time (with FQF), optionally create PCJ's and optionally 
> create alerts (with Alerting) preferably atomically, so if it fails in the 
> middle it does not leave dangling references and garbage -- currently there 
> is one method that does everything: CreatePCJ.withRyaInteraction()
> 2. Enable exporters per query -- currently all queries will be exported on 
> all globally enabled exporters.  When the query is registered (with FQF) 
> specify the exporters that should be used.
> 3. Separate export observer queues per exporter -- Currently retrying a 
> failed export observer will re-export all export types, even if some already 
> succeeded, causing duplicate triggers.
> Also there is a need to refine the terms for java identifiers (for example: 
> fields, classes) and documentation.  Please document here a decision about 
> what we name different things.  The term PCJ are the index tables used by Rya 
> to optimize it's queries.  But sometimes it is used to describe the whole 
> Fluo framework.  Please define a unique term for the following.  The starred 
> term will be used here, but feel free to rename them:
> - Rya query optimization PCJ  (ex: the Blabla exporter)  PCJ*
> - Kafka trigger alert notifier   (ex: the Foofoo exporter)  Alerting*
> - Fluo query observer framework  (ex: results from the Barbar)  FQF*
> Unique identifers:
> - FQF uses QueryID
> - Alerting uses TopicID 
> - PCJ uses PcjId
> Note that currently TopicID is assigned the same as QueryID, but that badly 
> presupposes queryID's cannot be grouped onto one Topic.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to