You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@samoa.apache.org by pwawrzyniak <gi...@git.apache.org> on 2017/06/07 11:06:48 UTC

[GitHub] incubator-samoa pull request #64: SAMOA-65 JSON serializer/deserializer dedi...

GitHub user pwawrzyniak opened a pull request:

    https://github.com/apache/incubator-samoa/pull/64

    SAMOA-65 JSON serializer/deserializer dedicated for Kafka components

    

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

    $ git pull https://github.com/pwawrzyniak/incubator-samoa SAMOA-65-JSON

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

    https://github.com/apache/incubator-samoa/pull/64.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 #64
    
----
commit 0e90fe8c107cd90a7baf16b5d4f334d1650a0afb
Author: pwawrzyniak <pi...@wawrzyniak.org.pl>
Date:   2017-03-14T16:43:25Z

    Initial structure of Kafka components.
    Initital code for Kafka Consumer

commit 2168400f86189605b06fb531511235733bf3b6ca
Author: pwawrzyniak <pi...@wawrzyniak.org.pl>
Date:   2017-03-17T10:05:14Z

    Code for KafkaEntranceProcessor (consuming messages from Kafka)

commit cf2ff369989fa3560f1570dfaf32bcb746d51c25
Author: pwawrzyniak <pi...@wawrzyniak.org.pl>
Date:   2017-03-17T10:54:47Z

    Updated comments

commit 26d86da6ebdd53e440e8b1e65f5ba5ed9f5f609e
Author: pwawrzyniak <pi...@wawrzyniak.org.pl>
Date:   2017-03-17T11:09:52Z

    Sample serializer/deserializer for JSON and InstanceContentEvent
    Updates in comments

commit 45b3546f9c525af385f646447a09d3683b70937d
Author: pwawrzyniak <pi...@wawrzyniak.org.pl>
Date:   2017-03-17T14:40:25Z

    KafkaDestinationProcessor implementation (sending msg to Kafka

commit 2e6d502a2909731cb9859ba3403407e6a33a2fc2
Author: pwawrzyniak <pi...@wawrzyniak.org.pl>
Date:   2017-03-24T13:34:49Z

    Tests for kafkaUtils and KafkaEntranceProcessor, minor changes in classes

commit 208febc0524ddc328ae1a785f0642650173f7101
Author: pwawrzyniak <pi...@wawrzyniak.org.pl>
Date:   2017-04-11T14:44:30Z

    Tests for KafkaDestinationProcessor, minor changes in classes

commit ff6b84dc5403c102388646a31f120c11ee416063
Author: Jakub Jankowski <ja...@orange.com>
Date:   2017-04-28T15:08:29Z

    Added Kafka Avro serializer with unit tests. Added Kafka task with unit test

commit 7997fd538c258e8844cbd1a81d4c6c43d8788aa4
Author: Jakub Jankowski <ja...@orange.com>
Date:   2017-05-01T09:23:00Z

    Merge branch 'SAMOA-65' of https://github.com/pwawrzyniak/incubator-samoa into SAMOA-65

commit ae7889fce1f52afdaa6d7e6080cff54dd1b01b9b
Author: Jakub Jankowski <ja...@orange.com>
Date:   2017-05-10T12:37:25Z

    Fixing avro deserialization

commit 107c4c5f72fcacc38d02bfc7cd1552a1593cdd5a
Author: Jakub Jankowski <ja...@orange.com>
Date:   2017-05-10T12:45:16Z

    Fixing avro deserialization

commit cde0642ccc768f7492e8c5811b2f1232e58acec4
Author: pwawrzyniak <pi...@wawrzyniak.org.pl>
Date:   2017-05-12T15:17:27Z

    Rebuild of Consumer-related classes (added separate thread to read from Kafka, blocking until sth received)
    Major cleanup in tests

commit f20301a4c64c9487fdbcff17829b6a8018e9a00b
Author: pwawrzyniak <pi...@wawrzyniak.org.pl>
Date:   2017-05-12T15:22:54Z

    Minor change in tests

commit b4ee754c54ac5be51f9846704947e7c19c186e73
Author: Jakub Jankowski <ja...@orange.com>
Date:   2017-05-16T12:11:36Z

    Fixing avro deserializer (using ReflectDatumReader) to be able to read arrays in InstanceData objects

commit d636ba90851d8424a6d3e651d80c87d0a8d7394e
Author: pwawrzyniak <pi...@wawrzyniak.org.pl>
Date:   2017-05-16T13:00:33Z

    Changes in JSON mapper

commit 57f80a5bf7d83c31d517a7d98741dd7457a4c84c
Author: pwawrzyniak <pi...@wawrzyniak.org.pl>
Date:   2017-06-07T10:21:08Z

    repo cleanup, split code into 3 branches

commit 04a28bb5ecbc039f07c17a91d57859c4651f56d9
Author: pwawrzyniak <pi...@wawrzyniak.org.pl>
Date:   2017-06-07T10:50:15Z

    JSON mapper with test

----


---
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-samoa issue #64: SAMOA-65 JSON serializer/deserializer dedicated f...

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

    https://github.com/apache/incubator-samoa/pull/64
  
    Hi @pwawrzyniak, 
    trying to keep the discussion here to more specifics, do you think we will have conflicts if we merge this one and the other PRs (#59,#65)? (as I had mentioned, there seemed to be a repetition of some classes/methods).
    Does it make sense to merge first the Kafka PR and then simplify the Avro and Json ones?


---
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-samoa pull request #64: SAMOA-65 JSON serializer/deserializer dedi...

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

    https://github.com/apache/incubator-samoa/pull/64#discussion_r125582379
  
    --- Diff: samoa-api/src/main/java/org/apache/samoa/streams/kafka/KafkaJsonMapper.java ---
    @@ -0,0 +1,103 @@
    +/*
    --- End diff --
    
    I think the same comment also applies for #65 


---
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-samoa pull request #64: SAMOA-65 JSON serializer/deserializer dedi...

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

    https://github.com/apache/incubator-samoa/pull/64#discussion_r125466957
  
    --- Diff: samoa-api/src/main/java/org/apache/samoa/streams/kafka/KafkaJsonMapper.java ---
    @@ -0,0 +1,103 @@
    +/*
    --- End diff --
    
    Is this class the main difference from #59 ?


---
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-samoa pull request #64: SAMOA-65 JSON serializer/deserializer dedi...

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

    https://github.com/apache/incubator-samoa/pull/64#discussion_r122202692
  
    --- Diff: samoa-api/src/test/java/org/apache/samoa/streams/kafka/KafkaEntranceProcessorTest.java ---
    @@ -0,0 +1,187 @@
    +/*
    + * Copyright 2017 The Apache Software Foundation.
    + *
    + * Licensed 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.samoa.streams.kafka;
    +
    +/*
    + * #%L
    + * SAMOA
    + * %%
    + * Copyright (C) 2014 - 2017 Apache Software Foundation
    + * %%
    + * Licensed 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.
    + * #L%
    + */
    +import com.google.gson.Gson;
    +import java.io.IOException;
    +import java.nio.charset.Charset;
    +import java.nio.file.Files;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Random;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +import java.util.logging.Level;
    +import java.util.logging.Logger;
    +import org.apache.samoa.learners.InstanceContentEvent;
    +import org.junit.After;
    +import org.junit.AfterClass;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +import static org.junit.Assert.*;
    +import kafka.admin.AdminUtils;
    +import kafka.admin.RackAwareMode;
    +import kafka.server.KafkaConfig;
    +import kafka.server.KafkaServer;
    +import kafka.utils.MockTime;
    +import kafka.utils.TestUtils;
    +import org.apache.kafka.common.utils.Time;
    +import kafka.utils.ZKStringSerializer$;
    +import kafka.utils.ZkUtils;
    +import kafka.zk.EmbeddedZookeeper;
    +import org.I0Itec.zkclient.ZkClient;
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.samoa.instances.InstancesHeader;
    +
    +/**
    + *
    + * @author pwawrzyniak
    + * @author Jakub Jankowski
    + */
    +public class KafkaEntranceProcessorTest {
    +
    +    private static final String ZKHOST = "127.0.0.1";
    +    private static final String BROKERHOST = "127.0.0.1";
    +    private static final String BROKERPORT = "9092";
    +    private static final String TOPIC_AVRO = "samoa_test-avro";
    --- End diff --
    
    Maybe this line is not 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-samoa pull request #64: SAMOA-65 JSON serializer/deserializer dedi...

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

    https://github.com/apache/incubator-samoa/pull/64#discussion_r128761959
  
    --- Diff: samoa-api/src/main/java/org/apache/samoa/streams/kafka/KafkaJsonMapper.java ---
    @@ -0,0 +1,103 @@
    +/*
    --- End diff --
    
    Yes, this is the main difference, as well as the test case where KafkaJsonMapper is being used.


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

Re: [GitHub] incubator-samoa issue #64: SAMOA-65 JSON serializer/deserializer dedicated f...

Posted by Nicolas Kourtellis <nk...@gmail.com>.
Good point!
I think it came up last time we worked on a release and we need to address
it asap.
Thanks for the clarification Justin.
Nicolas

On Fri, Jun 16, 2017 at 12:25 AM, Justin Mclean <ju...@classsoftware.com>
wrote:

> Hi,
>
> ASF headers shouldn’t have an ASF copyright line, please see [1].
>
> Thanks,
> Justin
>
> 1. https://www.apache.org/legal/src-headers.html#headers




-- 
Nicolas Kourtellis

Re: [GitHub] incubator-samoa issue #64: SAMOA-65 JSON serializer/deserializer dedicated f...

Posted by Justin Mclean <ju...@classsoftware.com>.
Hi,

ASF headers shouldn’t have an ASF copyright line, please see [1].

Thanks,
Justin

1. https://www.apache.org/legal/src-headers.html#headers

[GitHub] incubator-samoa issue #64: SAMOA-65 JSON serializer/deserializer dedicated f...

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

    https://github.com/apache/incubator-samoa/pull/64
  
    Thank you @pwawrzyniak for the contributions!
    Regarding the current code:
    - Can you change the copyright to be current year? (I wonder if we should keep this year appearing. It needs constant updating every time we have a new release).
    - I added some minor comments on leftovers from the AVRO combined integration. If you can remove them it would be cleaner.
    - I noticed that there is redundancy / repetition between the three PRs (#59,#64#65). Is there a way to make them unique to each other? Otherwise I think there will be conflicts when trying to merge them. @gdfm what do you think?
    
    - After checking the code, I realized that this is dedicated for Kafka.
    A quick question: Can this JSON serializer/deserializer be extended/abstracted to be used by other interfaces besides Kafka (e.g., even storing/retrieving JSON files from disk)? Do you think it is feasible or needs a lot of work?
    - Can we do something similar for Avro?
    



---
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-samoa issue #64: SAMOA-65 JSON serializer/deserializer dedicated f...

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

    https://github.com/apache/incubator-samoa/pull/64
  
    Hi @nicolas-kourtellis,
    It is true that the 3 PRs are dependent, but I think if #59 would be merged first, then there would be no merge conflicts caused by JSON/AVRO components (i.e. #65 and #64).
    I think we can proceed in the way that Kafka components (without JSON/AVRO support) are merged and then simplify/combine AVRO/JSON components to improve reusability of the code- but that would require more time to investigate, of course.


---
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-samoa issue #64: SAMOA-65 JSON serializer/deserializer dedicated f...

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

    https://github.com/apache/incubator-samoa/pull/64
  
    Thank you nicolas-kourtellis for your feedback!
    
    Regarding your comments, I removed copyright notices from the code, fixed and clened-up the code according to your comments.
    
    Regarding AVRO code provided within SAMOA-65, the first difference between avro loader and our kafka avro mapper is that avro loader works with json files containing schema header and payload data (json or binary). In our case avro mapper works with byte stream received from kafka and the schema is defined in separate file. The other difference is that avro loader produces Instance object while Kafka Mapper was designed to work with InstanceContentEvent. Moreover kafka avro mapper serializes whole InstanceContentEvent object while avro loader reads data file, creates avro structure and, based on that, builds Instance object.It should be possible to use the concept from avro loader for processing kafka but I suppose it would require implementation of new generic loader, instead of using the old one. Other thing is that we need to have two way serialization and avro loader is used only for reading data, not writing.
    
    And regarding JSON parser, as of now it is prepared to parse messages coming from Apache Kafka in "one-by-one" style, serializing InstanceContentEvent class. It could be potenatially used to parse file (in line-by-line manner for example), but I believe currently, as the mapper accepts byte array as the input, it can be easily used to this task (i.e. as the parser when reading data from text/json file).


---
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-samoa pull request #64: SAMOA-65 JSON serializer/deserializer dedi...

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

    https://github.com/apache/incubator-samoa/pull/64#discussion_r122203064
  
    --- Diff: samoa-api/src/test/java/org/apache/samoa/streams/kafka/KafkaEntranceProcessorWithJsonTest.java ---
    @@ -0,0 +1,187 @@
    +/*
    + * Copyright 2017 The Apache Software Foundation.
    + *
    + * Licensed 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.samoa.streams.kafka;
    +
    +/*
    + * #%L
    + * SAMOA
    + * %%
    + * Copyright (C) 2014 - 2017 Apache Software Foundation
    + * %%
    + * Licensed 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.
    + * #L%
    + */
    +import com.google.gson.Gson;
    +import java.io.IOException;
    +import java.nio.charset.Charset;
    +import java.nio.file.Files;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Random;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +import java.util.logging.Level;
    +import java.util.logging.Logger;
    +import org.apache.samoa.learners.InstanceContentEvent;
    +import org.junit.After;
    +import org.junit.AfterClass;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +import static org.junit.Assert.*;
    +import kafka.admin.AdminUtils;
    +import kafka.admin.RackAwareMode;
    +import kafka.server.KafkaConfig;
    +import kafka.server.KafkaServer;
    +import kafka.utils.MockTime;
    +import kafka.utils.TestUtils;
    +import org.apache.kafka.common.utils.Time;
    +import kafka.utils.ZKStringSerializer$;
    +import kafka.utils.ZkUtils;
    +import kafka.zk.EmbeddedZookeeper;
    +import org.I0Itec.zkclient.ZkClient;
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.samoa.instances.InstancesHeader;
    +
    +/**
    + *
    + * @author pwawrzyniak
    + * @author Jakub Jankowski
    + */
    +public class KafkaEntranceProcessorWithJsonTest {
    +
    +    private static final String ZKHOST = "127.0.0.1";
    +    private static final String BROKERHOST = "127.0.0.1";
    +    private static final String BROKERPORT = "9092";
    +    private static final String TOPIC_AVRO = "samoa_test-avro";
    +    private static final String TOPIC_JSON = "samoa_test-json";
    +    private static final int NUM_INSTANCES = 11111;
    +
    +    private static KafkaServer kafkaServer;
    +    private static EmbeddedZookeeper zkServer;
    +    private static ZkClient zkClient;
    +    private static String zkConnect;
    +    private static int TIMEOUT = 1000;
    +
    +    public KafkaEntranceProcessorWithJsonTest() {
    +    }
    +
    +    @BeforeClass
    +    public static void setUpClass() throws IOException {
    +        // setup Zookeeper
    +        zkServer = new EmbeddedZookeeper();
    +        zkConnect = ZKHOST + ":" + zkServer.port();
    +        zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer$.MODULE$);
    +        ZkUtils zkUtils = ZkUtils.apply(zkClient, false);
    +
    +        // setup Broker
    +        Properties brokerProps = new Properties();
    +        brokerProps.setProperty("zookeeper.connect", zkConnect);
    +        brokerProps.setProperty("broker.id", "0");
    +        brokerProps.setProperty("log.dirs", Files.createTempDirectory("kafka-").toAbsolutePath().toString());
    +        brokerProps.setProperty("listeners", "PLAINTEXT://" + BROKERHOST + ":" + BROKERPORT);
    +        KafkaConfig config = new KafkaConfig(brokerProps);
    +        Time mock = new MockTime();
    +        kafkaServer = TestUtils.createServer(config, mock);
    +
    +        // create topics
    +        AdminUtils.createTopic(zkUtils, TOPIC_AVRO, 1, 1, new Properties(), RackAwareMode.Disabled$.MODULE$);
    --- End diff --
    
    Similar with previous 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-samoa pull request #64: SAMOA-65 JSON serializer/deserializer dedi...

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

    https://github.com/apache/incubator-samoa/pull/64#discussion_r122202974
  
    --- Diff: samoa-api/src/test/java/org/apache/samoa/streams/kafka/KafkaEntranceProcessorWithJsonTest.java ---
    @@ -0,0 +1,187 @@
    +/*
    + * Copyright 2017 The Apache Software Foundation.
    + *
    + * Licensed 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.samoa.streams.kafka;
    +
    +/*
    + * #%L
    + * SAMOA
    + * %%
    + * Copyright (C) 2014 - 2017 Apache Software Foundation
    + * %%
    + * Licensed 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.
    + * #L%
    + */
    +import com.google.gson.Gson;
    +import java.io.IOException;
    +import java.nio.charset.Charset;
    +import java.nio.file.Files;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Random;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +import java.util.logging.Level;
    +import java.util.logging.Logger;
    +import org.apache.samoa.learners.InstanceContentEvent;
    +import org.junit.After;
    +import org.junit.AfterClass;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +import static org.junit.Assert.*;
    +import kafka.admin.AdminUtils;
    +import kafka.admin.RackAwareMode;
    +import kafka.server.KafkaConfig;
    +import kafka.server.KafkaServer;
    +import kafka.utils.MockTime;
    +import kafka.utils.TestUtils;
    +import org.apache.kafka.common.utils.Time;
    +import kafka.utils.ZKStringSerializer$;
    +import kafka.utils.ZkUtils;
    +import kafka.zk.EmbeddedZookeeper;
    +import org.I0Itec.zkclient.ZkClient;
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.samoa.instances.InstancesHeader;
    +
    +/**
    + *
    + * @author pwawrzyniak
    + * @author Jakub Jankowski
    + */
    +public class KafkaEntranceProcessorWithJsonTest {
    +
    +    private static final String ZKHOST = "127.0.0.1";
    +    private static final String BROKERHOST = "127.0.0.1";
    +    private static final String BROKERPORT = "9092";
    +    private static final String TOPIC_AVRO = "samoa_test-avro";
    --- End diff --
    
    Maybe this line is not needed for the JSON test


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