Re: Unit testing framework for Spark Jobs?

2016-05-21 Thread Lars Albertsson
 input to a Kafka topic, and wait for
>> >> the result to appear in Cassandra.
>> >>
>> >> With this setup, your application still runs in Scalatest, the tests
>> >> run without custom setup in maven/sbt/gradle, and you can easily run
>> >> and debug inside IntelliJ.
>> >>
>> >> Docker is suitable for spinning up external components. If you use
>> >> Kafka, the Docker image spotify/kafka is useful, since it bundles
>> >> Zookeeper.
>> >>
>> >> When waiting for output to appear, don't sleep for a long time and
>> >> then check, since it will slow down your tests. Instead enter a loop
>> >> where you poll for the results and sleep for a few milliseconds in
>> >> between, with a long timeout (~30s) before the test fails with a
>> >> timeout.
>> >
>> > org.scalatest.concurrent.Eventually is your friend there
>> >
>> > eventually(stdTimeout, stdInterval) {
>> > listRestAPIApplications(connector, webUI, true) should
>> contain(expectedAppId)
>> > }
>> >
>> > It has good exponential backoff, for fast initial success without using
>> too much CPU later, and is simple to use
>> >
>> > If it has weaknesses in my tests, they are
>> >
>> > 1. it will retry on all exceptions, rather than assertions. If there's
>> a bug in the test code then it manifests as a timeout. ( I think I could
>> play with Suite.anExceptionThatShouldCauseAnAbort()) here.
>> > 2. it's timeout action is simply to rethrow the fault; I like to exec a
>> closure to grab more diagnostics
>> > 3. It doesn't support some fail-fast exception which your code can
>> raise to indicate that the desired state is never going to be reached, and
>> so the test should fail fast. Here a new exception and another entry in
>> anExceptionThatShouldCauseAnAbort() may be the answer. I should sit down
>> and play with that some more.
>> >
>> >
>> >>
>> >> This poll and sleep strategy both makes tests quick in successful
>> >> cases, but still robust to occasional delays. The strategy does not
>> >> work if you want to test for absence, e.g. ensure that a particular
>> >> message if filtered. You can work around it by adding another message
>> >> afterwards and polling for its effect before testing for absence of
>> >> the first. Be aware that messages can be processed out of order in
>> >> Spark Streaming depending on partitioning, however.
>> >>
>> >>
>> >> I have tested Spark applications with both strategies described above,
>> >> and it is straightforward to set up. Let me know if you want
>> >> clarifications or assistance.
>> >>
>> >> Regards,
>> >>
>> >>
>> >>
>> >> Lars Albertsson
>> >> Data engineering consultant
>> >> www.mapflat.com
>> >> +46 70 7687109
>> >>
>> >>
>> >> On Wed, Mar 2, 2016 at 6:54 PM, SRK  wrote:
>> >>> Hi,
>> >>>
>> >>> What is a good unit testing framework for Spark batch/streaming jobs?
>> I have
>> >>> core spark, spark sql with dataframes and streaming api getting used.
>> Any
>> >>> good framework to cover unit tests for these APIs?
>> >>>
>> >>> Thanks!
>> >>>
>> >>>
>> >>>
>> >>> --
>> >>> View this message in context:
>> http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380.html
>> >>> Sent from the Apache Spark User List mailing list archive at
>> Nabble.com.
>> >>>
>> >>> -
>> >>> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>> >>> For additional commands, e-mail: user-h...@spark.apache.org
>> >>>
>> >>
>> >> -
>> >> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>> >> For additional commands, e-mail: user-h...@spark.apache.org
>> >>
>> >>
>> >
>>
>
>


Re: Unit testing framework for Spark Jobs?

2016-05-18 Thread Todd Nist
he tests
>> >> run without custom setup in maven/sbt/gradle, and you can easily run
>> >> and debug inside IntelliJ.
>> >>
>> >> Docker is suitable for spinning up external components. If you use
>> >> Kafka, the Docker image spotify/kafka is useful, since it bundles
>> >> Zookeeper.
>> >>
>> >> When waiting for output to appear, don't sleep for a long time and
>> >> then check, since it will slow down your tests. Instead enter a loop
>> >> where you poll for the results and sleep for a few milliseconds in
>> >> between, with a long timeout (~30s) before the test fails with a
>> >> timeout.
>> >
>> > org.scalatest.concurrent.Eventually is your friend there
>> >
>> > eventually(stdTimeout, stdInterval) {
>> > listRestAPIApplications(connector, webUI, true) should
>> contain(expectedAppId)
>> > }
>> >
>> > It has good exponential backoff, for fast initial success without using
>> too much CPU later, and is simple to use
>> >
>> > If it has weaknesses in my tests, they are
>> >
>> > 1. it will retry on all exceptions, rather than assertions. If there's
>> a bug in the test code then it manifests as a timeout. ( I think I could
>> play with Suite.anExceptionThatShouldCauseAnAbort()) here.
>> > 2. it's timeout action is simply to rethrow the fault; I like to exec a
>> closure to grab more diagnostics
>> > 3. It doesn't support some fail-fast exception which your code can
>> raise to indicate that the desired state is never going to be reached, and
>> so the test should fail fast. Here a new exception and another entry in
>> anExceptionThatShouldCauseAnAbort() may be the answer. I should sit down
>> and play with that some more.
>> >
>> >
>> >>
>> >> This poll and sleep strategy both makes tests quick in successful
>> >> cases, but still robust to occasional delays. The strategy does not
>> >> work if you want to test for absence, e.g. ensure that a particular
>> >> message if filtered. You can work around it by adding another message
>> >> afterwards and polling for its effect before testing for absence of
>> >> the first. Be aware that messages can be processed out of order in
>> >> Spark Streaming depending on partitioning, however.
>> >>
>> >>
>> >> I have tested Spark applications with both strategies described above,
>> >> and it is straightforward to set up. Let me know if you want
>> >> clarifications or assistance.
>> >>
>> >> Regards,
>> >>
>> >>
>> >>
>> >> Lars Albertsson
>> >> Data engineering consultant
>> >> www.mapflat.com
>> >> +46 70 7687109
>> >>
>> >>
>> >> On Wed, Mar 2, 2016 at 6:54 PM, SRK  wrote:
>> >>> Hi,
>> >>>
>> >>> What is a good unit testing framework for Spark batch/streaming jobs?
>> I have
>> >>> core spark, spark sql with dataframes and streaming api getting used.
>> Any
>> >>> good framework to cover unit tests for these APIs?
>> >>>
>> >>> Thanks!
>> >>>
>> >>>
>> >>>
>> >>> --
>> >>> View this message in context:
>> http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380.html
>> >>> Sent from the Apache Spark User List mailing list archive at
>> Nabble.com.
>> >>>
>> >>> -
>> >>> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>> >>> For additional commands, e-mail: user-h...@spark.apache.org
>> >>>
>> >>
>> >> -
>> >> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>> >> For additional commands, e-mail: user-h...@spark.apache.org
>> >>
>> >>
>> >
>>
>
>


Re: Unit testing framework for Spark Jobs?

2016-05-18 Thread swetha kasireddy
t; >> between, with a long timeout (~30s) before the test fails with a
> >> timeout.
> >
> > org.scalatest.concurrent.Eventually is your friend there
> >
> > eventually(stdTimeout, stdInterval) {
> > listRestAPIApplications(connector, webUI, true) should
> contain(expectedAppId)
> > }
> >
> > It has good exponential backoff, for fast initial success without using
> too much CPU later, and is simple to use
> >
> > If it has weaknesses in my tests, they are
> >
> > 1. it will retry on all exceptions, rather than assertions. If there's a
> bug in the test code then it manifests as a timeout. ( I think I could play
> with Suite.anExceptionThatShouldCauseAnAbort()) here.
> > 2. it's timeout action is simply to rethrow the fault; I like to exec a
> closure to grab more diagnostics
> > 3. It doesn't support some fail-fast exception which your code can raise
> to indicate that the desired state is never going to be reached, and so the
> test should fail fast. Here a new exception and another entry in
> anExceptionThatShouldCauseAnAbort() may be the answer. I should sit down
> and play with that some more.
> >
> >
> >>
> >> This poll and sleep strategy both makes tests quick in successful
> >> cases, but still robust to occasional delays. The strategy does not
> >> work if you want to test for absence, e.g. ensure that a particular
> >> message if filtered. You can work around it by adding another message
> >> afterwards and polling for its effect before testing for absence of
> >> the first. Be aware that messages can be processed out of order in
> >> Spark Streaming depending on partitioning, however.
> >>
> >>
> >> I have tested Spark applications with both strategies described above,
> >> and it is straightforward to set up. Let me know if you want
> >> clarifications or assistance.
> >>
> >> Regards,
> >>
> >>
> >>
> >> Lars Albertsson
> >> Data engineering consultant
> >> www.mapflat.com
> >> +46 70 7687109
> >>
> >>
> >> On Wed, Mar 2, 2016 at 6:54 PM, SRK  wrote:
> >>> Hi,
> >>>
> >>> What is a good unit testing framework for Spark batch/streaming jobs?
> I have
> >>> core spark, spark sql with dataframes and streaming api getting used.
> Any
> >>> good framework to cover unit tests for these APIs?
> >>>
> >>> Thanks!
> >>>
> >>>
> >>>
> >>> --
> >>> View this message in context:
> http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380.html
> >>> Sent from the Apache Spark User List mailing list archive at
> Nabble.com.
> >>>
> >>> -
> >>> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
> >>> For additional commands, e-mail: user-h...@spark.apache.org
> >>>
> >>
> >> -
> >> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
> >> For additional commands, e-mail: user-h...@spark.apache.org
> >>
> >>
> >
>


Re: Unit testing framework for Spark Jobs?

2016-03-30 Thread Lars Albertsson
a timeout. ( I think I could play
with Suite.anExceptionThatShouldCauseAnAbort()) here.
> 2. it's timeout action is simply to rethrow the fault; I like to exec a
closure to grab more diagnostics
> 3. It doesn't support some fail-fast exception which your code can raise
to indicate that the desired state is never going to be reached, and so the
test should fail fast. Here a new exception and another entry in
anExceptionThatShouldCauseAnAbort() may be the answer. I should sit down
and play with that some more.
>
>
>>
>> This poll and sleep strategy both makes tests quick in successful
>> cases, but still robust to occasional delays. The strategy does not
>> work if you want to test for absence, e.g. ensure that a particular
>> message if filtered. You can work around it by adding another message
>> afterwards and polling for its effect before testing for absence of
>> the first. Be aware that messages can be processed out of order in
>> Spark Streaming depending on partitioning, however.
>>
>>
>> I have tested Spark applications with both strategies described above,
>> and it is straightforward to set up. Let me know if you want
>> clarifications or assistance.
>>
>> Regards,
>>
>>
>>
>> Lars Albertsson
>> Data engineering consultant
>> www.mapflat.com
>> +46 70 7687109
>>
>>
>> On Wed, Mar 2, 2016 at 6:54 PM, SRK  wrote:
>>> Hi,
>>>
>>> What is a good unit testing framework for Spark batch/streaming jobs? I
have
>>> core spark, spark sql with dataframes and streaming api getting used.
Any
>>> good framework to cover unit tests for these APIs?
>>>
>>> Thanks!
>>>
>>>
>>>
>>> --
>>> View this message in context:
http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380.html
>>> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>>>
>>> -
>>> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>>> For additional commands, e-mail: user-h...@spark.apache.org
>>>
>>
>> -
>> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>> For additional commands, e-mail: user-h...@spark.apache.org
>>
>>
>


Re: Unit testing framework for Spark Jobs?

2016-03-28 Thread Steve Loughran
 test for absence, e.g. ensure that a particular
> message if filtered. You can work around it by adding another message
> afterwards and polling for its effect before testing for absence of
> the first. Be aware that messages can be processed out of order in
> Spark Streaming depending on partitioning, however.
> 
> 
> I have tested Spark applications with both strategies described above,
> and it is straightforward to set up. Let me know if you want
> clarifications or assistance.
> 
> Regards,
> 
> 
> 
> Lars Albertsson
> Data engineering consultant
> www.mapflat.com
> +46 70 7687109
> 
> 
> On Wed, Mar 2, 2016 at 6:54 PM, SRK  wrote:
>> Hi,
>> 
>> What is a good unit testing framework for Spark batch/streaming jobs? I have
>> core spark, spark sql with dataframes and streaming api getting used. Any
>> good framework to cover unit tests for these APIs?
>> 
>> Thanks!
>> 
>> 
>> 
>> --
>> View this message in context: 
>> http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380.html
>> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>> 
>> -
>> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>> For additional commands, e-mail: user-h...@spark.apache.org
>> 
> 
> -
> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
> For additional commands, e-mail: user-h...@spark.apache.org
> 
> 


-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org



Re: Unit testing framework for Spark Jobs?

2016-03-24 Thread Shiva Ramagopal
Hi Lars,

Very pragmatic ideas around testing of Spark applications end-to-end!

-Shiva

On Fri, Mar 18, 2016 at 12:35 PM, Lars Albertsson  wrote:

> I would recommend against writing unit tests for Spark programs, and
> instead focus on integration tests of jobs or pipelines of several
> jobs. You can still use a unit test framework to execute them. Perhaps
> this is what you meant.
>
> You can use any of the popular unit test frameworks to drive your
> tests, e.g. JUnit, Scalatest, Specs2. I prefer Scalatest, since it
> gives you choice of TDD vs BDD, and it is also well integrated with
> IntelliJ.
>
> I would also recommend against using testing frameworks tied to a
> processing technology, such as Spark Testing Base. Although it does
> seem well crafted, and makes it easy to get started with testing,
> there are drawbacks:
>
> 1. I/O routines are not tested. Bundled test frameworks typically do
> not materialise datasets on storage, but pass them directly in memory.
> (I have not verified this for Spark Testing Base, but it looks so.)
> I/O routines are therefore not exercised, and they often hide bugs,
> e.g. related to serialisation.
>
> 2. You create a strong coupling between processing technology and your
> tests. If you decide to change processing technology (which can happen
> soon in this fast paced world...), you need to rewrite your tests.
> Therefore, during a migration process, the tests cannot detect bugs
> introduced in migration, and help you migrate fast.
>
> I recommend that you instead materialise input datasets on local disk,
> run your Spark job, which writes output datasets to local disk, read
> output from disk, and verify the results. You can still use Spark
> routines to read and write input and output datasets. A Spark context
> is expensive to create, so for speed, I would recommend reusing the
> Spark context between input generation, running the job, and reading
> output.
>
> This is easy to set up, so you don't need a dedicated framework for
> it. Just put your common boilerplate in a shared test trait or base
> class.
>
> In the future, when you want to replace your Spark job with something
> shinier, you can still use the old tests, and only replace the part
> that runs your job, giving you some protection from regression bugs.
>
>
> Testing Spark Streaming applications is a different beast, and you can
> probably not reuse much from your batch testing.
>
> For testing streaming applications, I recommend that you run your
> application inside a unit test framework, e.g, Scalatest, and have the
> test setup create a fixture that includes your input and output
> components. For example, if your streaming application consumes from
> Kafka and updates tables in Cassandra, spin up single node instances
> of Kafka and Cassandra on your local machine, and connect your
> application to them. Then feed input to a Kafka topic, and wait for
> the result to appear in Cassandra.
>
> With this setup, your application still runs in Scalatest, the tests
> run without custom setup in maven/sbt/gradle, and you can easily run
> and debug inside IntelliJ.
>
> Docker is suitable for spinning up external components. If you use
> Kafka, the Docker image spotify/kafka is useful, since it bundles
> Zookeeper.
>
> When waiting for output to appear, don't sleep for a long time and
> then check, since it will slow down your tests. Instead enter a loop
> where you poll for the results and sleep for a few milliseconds in
> between, with a long timeout (~30s) before the test fails with a
> timeout.
>
> This poll and sleep strategy both makes tests quick in successful
> cases, but still robust to occasional delays. The strategy does not
> work if you want to test for absence, e.g. ensure that a particular
> message if filtered. You can work around it by adding another message
> afterwards and polling for its effect before testing for absence of
> the first. Be aware that messages can be processed out of order in
> Spark Streaming depending on partitioning, however.
>
>
> I have tested Spark applications with both strategies described above,
> and it is straightforward to set up. Let me know if you want
> clarifications or assistance.
>
> Regards,
>
>
>
> Lars Albertsson
> Data engineering consultant
> www.mapflat.com
> +46 70 7687109
>
>
> On Wed, Mar 2, 2016 at 6:54 PM, SRK  wrote:
> > Hi,
> >
> > What is a good unit testing framework for Spark batch/streaming jobs? I
> have
> > core spark, spark sql with dataframes and streaming api getting used. Any
> > good framework to cover unit tests for these APIs?
> >
> > Thanks!
> >
> >
> >
> > --
&g

Re: Unit testing framework for Spark Jobs?

2016-03-19 Thread Vikas Kawadia
I just wrote a blog post on Unit testing Apache Spark with py.test
https://engblog.nextdoor.com/unit-testing-apache-spark-with-py-test-3b8970dc013b

If you prefer using the py.test framework, then it might be useful.

-vikas

On Wed, Mar 2, 2016 at 10:59 AM, radoburansky 
wrote:

> I am sure you have googled this:
> https://github.com/holdenk/spark-testing-base
>
> On Wed, Mar 2, 2016 at 6:54 PM, SRK [via Apache Spark User List] <[hidden
> email] <http:///user/SendEmail.jtp?type=node&node=26384&i=0>> wrote:
>
>> Hi,
>>
>> What is a good unit testing framework for Spark batch/streaming jobs? I
>> have core spark, spark sql with dataframes and streaming api getting used.
>> Any good framework to cover unit tests for these APIs?
>>
>> Thanks!
>>
>> --
>> If you reply to this email, your message will be added to the discussion
>> below:
>>
>> http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380.html
>> To start a new topic under Apache Spark User List, email [hidden email]
>> <http:///user/SendEmail.jtp?type=node&node=26384&i=1>
>> To unsubscribe from Apache Spark User List, click here.
>> NAML
>> <http://apache-spark-user-list.1001560.n3.nabble.com/template/NamlServlet.jtp?macro=macro_viewer&id=instant_html%21nabble%3Aemail.naml&base=nabble.naml.namespaces.BasicNamespace-nabble.view.web.template.NabbleNamespace-nabble.view.web.template.NodeNamespace&breadcrumbs=notify_subscribers%21nabble%3Aemail.naml-instant_emails%21nabble%3Aemail.naml-send_instant_email%21nabble%3Aemail.naml>
>>
>
>
> --
> View this message in context: Re: Unit testing framework for Spark Jobs?
> <http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380p26384.html>
> Sent from the Apache Spark User List mailing list archive
> <http://apache-spark-user-list.1001560.n3.nabble.com/> at Nabble.com.
>


Re: Unit testing framework for Spark Jobs?

2016-03-19 Thread Lars Albertsson
I would recommend against writing unit tests for Spark programs, and
instead focus on integration tests of jobs or pipelines of several
jobs. You can still use a unit test framework to execute them. Perhaps
this is what you meant.

You can use any of the popular unit test frameworks to drive your
tests, e.g. JUnit, Scalatest, Specs2. I prefer Scalatest, since it
gives you choice of TDD vs BDD, and it is also well integrated with
IntelliJ.

I would also recommend against using testing frameworks tied to a
processing technology, such as Spark Testing Base. Although it does
seem well crafted, and makes it easy to get started with testing,
there are drawbacks:

1. I/O routines are not tested. Bundled test frameworks typically do
not materialise datasets on storage, but pass them directly in memory.
(I have not verified this for Spark Testing Base, but it looks so.)
I/O routines are therefore not exercised, and they often hide bugs,
e.g. related to serialisation.

2. You create a strong coupling between processing technology and your
tests. If you decide to change processing technology (which can happen
soon in this fast paced world...), you need to rewrite your tests.
Therefore, during a migration process, the tests cannot detect bugs
introduced in migration, and help you migrate fast.

I recommend that you instead materialise input datasets on local disk,
run your Spark job, which writes output datasets to local disk, read
output from disk, and verify the results. You can still use Spark
routines to read and write input and output datasets. A Spark context
is expensive to create, so for speed, I would recommend reusing the
Spark context between input generation, running the job, and reading
output.

This is easy to set up, so you don't need a dedicated framework for
it. Just put your common boilerplate in a shared test trait or base
class.

In the future, when you want to replace your Spark job with something
shinier, you can still use the old tests, and only replace the part
that runs your job, giving you some protection from regression bugs.


Testing Spark Streaming applications is a different beast, and you can
probably not reuse much from your batch testing.

For testing streaming applications, I recommend that you run your
application inside a unit test framework, e.g, Scalatest, and have the
test setup create a fixture that includes your input and output
components. For example, if your streaming application consumes from
Kafka and updates tables in Cassandra, spin up single node instances
of Kafka and Cassandra on your local machine, and connect your
application to them. Then feed input to a Kafka topic, and wait for
the result to appear in Cassandra.

With this setup, your application still runs in Scalatest, the tests
run without custom setup in maven/sbt/gradle, and you can easily run
and debug inside IntelliJ.

Docker is suitable for spinning up external components. If you use
Kafka, the Docker image spotify/kafka is useful, since it bundles
Zookeeper.

When waiting for output to appear, don't sleep for a long time and
then check, since it will slow down your tests. Instead enter a loop
where you poll for the results and sleep for a few milliseconds in
between, with a long timeout (~30s) before the test fails with a
timeout.

This poll and sleep strategy both makes tests quick in successful
cases, but still robust to occasional delays. The strategy does not
work if you want to test for absence, e.g. ensure that a particular
message if filtered. You can work around it by adding another message
afterwards and polling for its effect before testing for absence of
the first. Be aware that messages can be processed out of order in
Spark Streaming depending on partitioning, however.


I have tested Spark applications with both strategies described above,
and it is straightforward to set up. Let me know if you want
clarifications or assistance.

Regards,



Lars Albertsson
Data engineering consultant
www.mapflat.com
+46 70 7687109


On Wed, Mar 2, 2016 at 6:54 PM, SRK  wrote:
> Hi,
>
> What is a good unit testing framework for Spark batch/streaming jobs? I have
> core spark, spark sql with dataframes and streaming api getting used. Any
> good framework to cover unit tests for these APIs?
>
> Thanks!
>
>
>
> --
> View this message in context: 
> http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380.html
> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>
> -
> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
> For additional commands, e-mail: user-h...@spark.apache.org
>

-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org



Re: Unit testing framework for Spark Jobs?

2016-03-02 Thread radoburansky
I am sure you have googled this:
https://github.com/holdenk/spark-testing-base

On Wed, Mar 2, 2016 at 6:54 PM, SRK [via Apache Spark User List] <
ml-node+s1001560n2638...@n3.nabble.com> wrote:

> Hi,
>
> What is a good unit testing framework for Spark batch/streaming jobs? I
> have core spark, spark sql with dataframes and streaming api getting used.
> Any good framework to cover unit tests for these APIs?
>
> Thanks!
>
> --
> If you reply to this email, your message will be added to the discussion
> below:
>
> http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380.html
> To start a new topic under Apache Spark User List, email
> ml-node+s1001560n1...@n3.nabble.com
> To unsubscribe from Apache Spark User List, click here
> <http://apache-spark-user-list.1001560.n3.nabble.com/template/NamlServlet.jtp?macro=unsubscribe_by_code&node=1&code=cmFkb2J1cmFuc2t5QGdtYWlsLmNvbXwxfC03MDA2NjE5MjQ=>
> .
> NAML
> <http://apache-spark-user-list.1001560.n3.nabble.com/template/NamlServlet.jtp?macro=macro_viewer&id=instant_html%21nabble%3Aemail.naml&base=nabble.naml.namespaces.BasicNamespace-nabble.view.web.template.NabbleNamespace-nabble.view.web.template.NodeNamespace&breadcrumbs=notify_subscribers%21nabble%3Aemail.naml-instant_emails%21nabble%3Aemail.naml-send_instant_email%21nabble%3Aemail.naml>
>




--
View this message in context: 
http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380p26384.html
Sent from the Apache Spark User List mailing list archive at Nabble.com.

Re: Unit testing framework for Spark Jobs?

2016-03-02 Thread Ricardo Paiva
I use the plain and old Junit

Spark batch example:

import org.apache.spark.SparkConf
import org.apache.spark.SparkContext
import org.apache.spark.sql.SQLContext
import org.junit.AfterClass
import org.junit.Assert.assertEquals
import org.junit.BeforeClass
import org.junit.Test

object TestMyCode {

  var sc: SparkContext = null

  @BeforeClass
  def setup(): Unit = {
val sparkConf = new SparkConf()
  .setAppName("Test Spark")
  .setMaster("local[*]")
sc = new SparkContext(sparkConf)
  }

  @AfterClass
  def cleanup(): Unit = {
sc.stop()
  }
}

class TestMyCode {

  @Test
  def testSaveNumbersToExtractor(): Unit = {
val sql = new SQLContext(TestDataframeToTableau.sc)
import sql.implicits._

val numList = List(1, 2, 3, 4, 5)
val df = TestDataframeToTableau.sc.parallelize(numList).toDF
val numDf = df.select(df("_1").alias("num"))
assertEquals(5, numDf.count)
  }

}

On Wed, Mar 2, 2016 at 2:54 PM, SRK [via Apache Spark User List] <
ml-node+s1001560n26380...@n3.nabble.com> wrote:

> Hi,
>
> What is a good unit testing framework for Spark batch/streaming jobs? I
> have core spark, spark sql with dataframes and streaming api getting used.
> Any good framework to cover unit tests for these APIs?
>
> Thanks!
>
> --
> If you reply to this email, your message will be added to the discussion
> below:
>
> http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380.html
> To start a new topic under Apache Spark User List, email
> ml-node+s1001560n1...@n3.nabble.com
> To unsubscribe from Apache Spark User List, click here
> <http://apache-spark-user-list.1001560.n3.nabble.com/template/NamlServlet.jtp?macro=unsubscribe_by_code&node=1&code=cmljYXJkby5wYWl2YUBjb3JwLmdsb2JvLmNvbXwxfDQ1MDcxMTc2Mw==>
> .
> NAML
> <http://apache-spark-user-list.1001560.n3.nabble.com/template/NamlServlet.jtp?macro=macro_viewer&id=instant_html%21nabble%3Aemail.naml&base=nabble.naml.namespaces.BasicNamespace-nabble.view.web.template.NabbleNamespace-nabble.view.web.template.NodeNamespace&breadcrumbs=notify_subscribers%21nabble%3Aemail.naml-instant_emails%21nabble%3Aemail.naml-send_instant_email%21nabble%3Aemail.naml>
>



-- 
Ricardo Paiva
Big Data / Semântica
2483-6432
*globo.com* <http://www.globo.com>




--
View this message in context: 
http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380p26383.html
Sent from the Apache Spark User List mailing list archive at Nabble.com.

Re: Unit testing framework for Spark Jobs?

2016-03-02 Thread Silvio Fiorito
Please check out the following for some good resources:

https://github.com/holdenk/spark-testing-base


https://spark-summit.org/east-2016/events/beyond-collect-and-parallelize-for-tests/





On 3/2/16, 12:54 PM, "SRK"  wrote:

>Hi,
>
>What is a good unit testing framework for Spark batch/streaming jobs? I have
>core spark, spark sql with dataframes and streaming api getting used. Any
>good framework to cover unit tests for these APIs?
>
>Thanks!
>
>
>
>--
>View this message in context: 
>http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380.html
>Sent from the Apache Spark User List mailing list archive at Nabble.com.
>
>-
>To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>For additional commands, e-mail: user-h...@spark.apache.org
>

-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org



Re: Unit testing framework for Spark Jobs?

2016-03-02 Thread Yin Yang
Cycling prior bits:

http://search-hadoop.com/m/q3RTto4sby1Cd2rt&subj=Re+Unit+test+with+sqlContext

On Wed, Mar 2, 2016 at 9:54 AM, SRK  wrote:

> Hi,
>
> What is a good unit testing framework for Spark batch/streaming jobs? I
> have
> core spark, spark sql with dataframes and streaming api getting used. Any
> good framework to cover unit tests for these APIs?
>
> Thanks!
>
>
>
> --
> View this message in context:
> http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380.html
> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>
> -
> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
> For additional commands, e-mail: user-h...@spark.apache.org
>
>


Unit testing framework for Spark Jobs?

2016-03-02 Thread SRK
Hi,

What is a good unit testing framework for Spark batch/streaming jobs? I have
core spark, spark sql with dataframes and streaming api getting used. Any
good framework to cover unit tests for these APIs?

Thanks!



--
View this message in context: 
http://apache-spark-user-list.1001560.n3.nabble.com/Unit-testing-framework-for-Spark-Jobs-tp26380.html
Sent from the Apache Spark User List mailing list archive at Nabble.com.

-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org