Getting Started
Documentation

Programming Model


Introduction

Samza provides multiple programming APIs to fit your use case:

  1. Java APIs: Samza’s provides two Java programming APIs that are ideal for building advanced Stream Processing applications.
    1. High Level Streams API: Samza’s flexible High Level Streams API lets you describe your complex stream processing pipeline in the form of a Directional Acyclic Graph (DAG) of operations on message streams. It provides a rich set of built-in operators that simplify common stream processing operations such as filtering, projection, repartitioning, joins, and windows.
    2. Low Level Task API: Samza’s powerful Low Level Task API lets you write your application in terms of processing logic for each incoming message.
  2. Samza SQL: Samza SQL provides a declarative query language for describing your stream processing logic. It lets you manipulate streams using SQL predicates and UDFs instead of working with the physical implementation details.
  3. Apache Beam API: Samza also provides a Apache Beam runner to run applications written using the Apache Beam API. This is considered as an extension to the operators supported by the High Level Streams API in Samza.

Key Concepts

The following sections will talk about key concepts in writing your Samza applications in Java.

Samza Applications

A SamzaApplication describes the inputs, outputs, state, configuration and the logic for processing data from one or more streaming sources.

You can implement a StreamApplication and use the provided StreamApplicationDescriptor to describe the processing logic using Samza’s High Level Streams API in terms of MessageStream operators.

    public class MyStreamApplication implements StreamApplication {
        @Override
        public void describe(StreamApplicationDescriptor appDesc) {
            // Describe your application here 
        }
    }

Alternatively, you can implement a TaskApplication and use the provided TaskApplicationDescriptor to describe it using Samza’s Low Level API in terms of per-message processing logic.

  • For TaskApplication:
    
    public class MyTaskApplication implements TaskApplication {
        @Override
        public void describe(TaskApplicationDescriptor appDesc) {
            // Describe your application here
        }
    }

Streams and Table Descriptors

Descriptors let you specify the properties of various aspects of your application from within it.

InputDescriptors and OutputDescriptors can be used for specifying Samza and implementation-specific properties of the streaming inputs and outputs for your application. You can obtain InputDescriptors and OutputDescriptors using a SystemDescriptor for your system. This SystemDescriptor can be used for specify Samza and implementation-specific properties of the producer and consumers for your I/O system. Most Samza system implementations come with their own SystemDescriptors, but if one isn’t available, you can use the GenericSystemDescriptor.

A TableDescriptor can be used for specifying Samza and implementation-specific properties of a Table. You can use a Local TableDescriptor (e.g. RocksDbTableDescriptor or a RemoteTableDescriptor.

The following example illustrates how you can use input and output descriptors for a Kafka system, and a table descriptor for a local RocksDB table within your application:

    
    public class MyStreamApplication implements StreamApplication {
      @Override
      public void describe(StreamApplicationDescriptor appDescriptor) {
        KafkaSystemDescriptor ksd = new KafkaSystemDescriptor("kafka")
            .withConsumerZkConnect(ImmutableList.of("..."))
            .withProducerBootstrapServers(ImmutableList.of("...", "..."));
        KafkaInputDescriptor<PageView> kid = 
            ksd.getInputDescriptor(page-views, new JsonSerdeV2<>(PageView.class));
        KafkaOutputDescriptor<DecoratedPageView> kod = 
            ksd.getOutputDescriptor(decorated-page-views, new JsonSerdeV2<>(DecoratedPageView.class));

        RocksDbTableDescriptor<String, Integer> td = 
            new RocksDbTableDescriptor(viewCounts, KVSerde.of(new StringSerde(), new IntegerSerde()));
            
        // Implement your processing logic here
      }
    }
    

The same code in the above describe method applies to TaskApplication as well.

Stream Processing Logic

Samza provides two sets of APIs to define the main stream processing logic, High Level Streams API and Low Level Task API, via StreamApplication and TaskApplication, respectively.

High Level Streams API allows you to describe the processing logic in a connected DAG of transformation operators, like the example below:

    public class BadPageViewFilter implements StreamApplication {
      @Override
      public void describe(StreamApplicationDescriptor appDesc) {
        KafkaSystemDescriptor ksd = new KafkaSystemDescriptor();
        InputDescriptor<PageView> pageViewInput = kafka.getInputDescriptor(page-views, new JsonSerdeV2<>(PageView.class));
        OutputDescriptor<DecoratedPageView> pageViewOutput = kafka.getOutputDescriptor(decorated-page-views, new JsonSerdeV2<>(DecoratedPageView.class));
        RocksDbTableDescriptor<String, Integer> viewCountTable = new RocksDbTableDescriptor(
            pageViewCountTable, KVSerde.of(new StringSerde(), new IntegerSerde()));

        // Now, implement your main processing logic
        MessageStream<PageView> pageViews = appDesc.getInputStream(pageViewInput);
        pageViews.filter(this::isValidPageView)
             .map(this::addProfileInformation)
             .sendTo(pageViewOutput);
      }
    }
    

Low Level Task API allows you to describe the processing logic in a customized StreamTaskFactory or AsyncStreamTaskFactory, like the example below:

    public class BadPageViewFilter implements TaskApplication {
      @Override
      public void describe(TaskApplicationDescriptor appDesc) {
        KafkaSystemDescriptor kafka = new KafkaSystemDescriptor();
        InputDescriptor<PageView> pageViewInput = kafka.getInputDescriptor(page-views, new JsonSerdeV2<>(PageView.class));
        OutputDescriptor<DecoratedPageView> pageViewOutput = kafka.getOutputDescriptor(decorated-page-views, new JsonSerdeV2<>(DecoratedPageView.class));
        RocksDbTableDescriptor<String, Integer> viewCountTable = new RocksDbTableDescriptor(
            pageViewCountTable, KVSerde.of(new StringSerde(), new IntegerSerde()));

        // Now, implement your main processing logic
        appDesc.withInputStream(pageViewInput)
           .withOutputStream(pageViewOutput)
           .withTaskFactory(new PageViewFilterTaskFactory());
      }
    }
    

Configuration for a Samza Application

To deploy a Samza application, you need to specify the implementation class for your application and the ApplicationRunner to launch your application. The following is an incomplete example of minimum required configuration to set up the Samza application and the runner. For additional configuration, see the Configuration Reference.

    
    # This is the class implementing StreamApplication
    app.class=com.example.samza.PageViewFilter

    # This is defining the ApplicationRunner class to launch the application
    app.runner.class=org.apache.samza.runtime.RemoteApplicationRunner