Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

In order to support Partial updates, we will need to add an update API in ReadWriteTable and related interfaces. Update is a variant of write but sometimes works with a different record type when compared to Write record. AsyncReadWriteTable works with generic KV where K is the key type and V is the value type of data in the Table. Adding another generic type parameter, say U, to indicate an update type parameter is a backward incompatible change and would result in changes all across the Table API. 

Samza Table API Changes with Partial Update

The following changes have to be made:

...

Code Block
languagejava
titleMessageStream
collapsetrue
public interface MessageStream<M> {
/**
  * Allows sending update messages in this {@link MessageStream} to a {@link Table} and then propagates this
  * {@link MessageStream} to the next chained operator. The type of input message is expected to be {@link KV},
  * otherwise a {@link ClassCastException} will be thrown. The value is an UpdateMessage- update and default value.
  * Defaults are optional and can be used if the Remote Table integration supports inserting a default through PUT in
  * the event an update fails due to an existing record being absent.
  * <p>
  * Note: The update will be written but may not be flushed to the underlying table before its propagated to the
  * chained operators. Whether the message can be read back from the Table in the chained operator depends on whether
  * it was flushed and whether the Table offers read after write consistency. Messages retain the original partitioning
  * scheme when propagated to next operator.
  *
  * @param table the table to write messages to
  * @param args additional arguments passed to the table
  * @param <K> the type of key in the table
  * @param <V> the type of record value in the table
  * @param <U> the type of update value for the table
  * @return this {@link MessageStream}
  */
 <K, V, U> MessageStream<KV<K, UpdateMessage<U, V>>> sendUpdateTo(Table<KV<K, V>> table, Object ... args);
}

Handling First Time Updates

While partial updates are intended to update existing records, there will be certain cases which require support for first-time partial update updates i.e update to a record which doesn’t exist. To account for such cases, the design needs to have a provision to optionally provide a default record which can be PUT in the absence of an existing record. The update can then be applied on top of the default record.

The approach introduces an UpdateMessage class which captures the update and an optional default. sendUpdateTo operator which sends updates to a table is designed to take a key (which uniquely identifies a record) and the UpdateMessage as the value.

SendUpdateToTableOperatorImpl is the implementation of the send-update-to-table operator and whether this operator supports first-time partial update entirely depends on the Remote store's implementation of the TableWriteFunction. This approach introduces a RecordNotFoundException which is a custom exception to be thrown in the updateAsync method of TableWriteFunction if the update fails due to an existing record not present for the key. If the SendUpdateToTableOperatorImpl operator encounters this exception, it attempts to PUT a default (if one is provided) and then applies an update on top of it.

...

Code Block
languagejava
titleSendUpdateToTableOperatorImpl
collapsetrue
/*
 * 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.samza.operators.impl;

import java.util.Collection;
import java.util.Collections;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage;
import org.apache.samza.SamzaException;
import org.apache.samza.context.Context;
import org.apache.samza.operators.KV;
import org.apache.samza.operators.UpdateMessage;
import org.apache.samza.operators.spec.OperatorSpec;
import org.apache.samza.operators.spec.SendUpdateToTableOperatorSpec;
import org.apache.samza.table.ReadWriteTable;
import org.apache.samza.table.RecordNotFoundException;
import org.apache.samza.table.batching.CompactBatchProvider;
import org.apache.samza.table.remote.RemoteTable;
import org.apache.samza.task.MessageCollector;
import org.apache.samza.task.TaskCoordinator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;


/**
 * Implementation of a send-update-stream-to-table operator that applies updates to existing records
 * in the table. If there is no pre-existing record, based on Table's implementation it might attempt to write a
 * default record and then applies an update.
 *
 * @param <K> the type of the record key
 * @param <V> the type of the record value
 * @param <U> the type of the update applied to this table
 */
public class SendUpdateToTableOperatorImpl<K, V, U>
    extends OperatorImpl<KV<K, UpdateMessage<U, V>>, KV<K, UpdateMessage<U, V>>> {
  private static final Logger LOG = LoggerFactory.getLogger(SendUpdateToTableOperatorImpl.class);

  private final SendUpdateToTableOperatorSpec<K, V, U> sendUpdateToTableOpSpec;
  private final ReadWriteTable<K, V, U> table;

  public SendUpdateToTableOperatorImpl(SendUpdateToTableOperatorSpec<K, V, U>  sendUpdateToTableOpSpec, Context context) {
    this.sendUpdateToTableOpSpec = sendUpdateToTableOpSpec;
    this.table = context.getTaskContext().getTable(sendUpdateToTableOpSpec.getTableId());
    if (context.getTaskContext().getTable(sendUpdateToTableOpSpec.getTableId()) instanceof RemoteTable) {
      RemoteTable<K, V, U> remoteTable = (RemoteTable<K, V, U>) table;
      if (remoteTable.getBatchProvider() instanceof CompactBatchProvider) {
        throw new SamzaException("Batching is not supported with Compact Batches for partial updates");
      }
    }
  }

  @Override
  protected void handleInit(Context context) {
  }

  @Override
  protected CompletionStage<Collection<KV<K, UpdateMessage<U, V>>>> handleMessageAsync(KV<K, UpdateMessage<U, V>> message,
      MessageCollector collector, TaskCoordinator coordinator) {
    final CompletableFuture<Void> updateFuture = table.updateAsync(message.getKey(), message.getValue().getUpdate(),
        sendUpdateToTableOpSpec.getArgs());

    return updateFuture
        .handle((result, ex) -> {
          if (ex == null) {
            // success, no need to Put a default value
            return false;
          } else if (ex.getCause() instanceof RecordNotFoundException && message.getValue().getDefault() != null) {
            // If update fails for a given key due to a RecordDoesNotExistException exception thrown and a default is
            // provided, then attempt to PUT a default record for the key and then apply the update
            return true;
          } else {
            throw new SamzaException("Update failed with exception: ", ex);
          }
        })
        .thenCompose(shouldPutDefault -> {
          if (shouldPutDefault) {
            final CompletableFuture<Void> putFuture = table.putAsync(message.getKey(), message.getValue().getDefault(),
                sendUpdateToTableOpSpec.getArgs());
            return putFuture
                .exceptionally(ex -> {
                  LOG.warn("PUT default failed due to an exception. Ignoring the exception and proceeding with update. "
                          + "The exception encountered is: ", ex);
                  return null;
                })
                .thenCompose(res -> table.updateAsync(message.getKey(), message.getValue().getUpdate(),
                sendUpdateToTableOpSpec.getArgs()));
          } else {
            return CompletableFuture.completedFuture(null);
          }
        }).thenApply(result -> Collections.singleton(message));
  }

  @Override
  protected void handleClose() {
    table.close();
  }

  @Override
  protected OperatorSpec<KV<K, UpdateMessage<U, V>>, KV<K, UpdateMessage<U, V>>> getOperatorSpec() {
    return sendUpdateToTableOpSpec;
  }
}

Update support in different Table types

Batching Table

Batching tables supports batching of table operations. The key classes/interfaces are:

  • Batch: Maintains a sequences of operations
  • BatchProvider: Creates a batch
  • Operation: Represents a table operation that can be batched
  • BatchProcessor: Places operations into batches
  • BatchHandler: Defines how batches will be handled

Current components are designed to work with K, V generic types of a Table and would need to be modified to include update generic type U as well. The signature of the Operation interface will be changed to add get Update as well. In addition, an UpdateOperation class representing an Update table operation will be created. BatchHandler and BatchProcessor will be modified to handle UpdateOperations.


Code Block
languagejava
titleOperation
collapsetrue
/**
 * Interface for table operations that can be batched.
 *
 * @param <K> The key type associated with the operation.
 * @param <V> The value type associated with the operation.
 * @param <U> The update type associated with the operation.
 */
public interface Operation<K, V, U> {
  /**
   * @return The key associated with the operation.
   */
  K getKey();

  /**
   * @return The value associated with the operation.
   */
  V getValue();

  /**
   * @return The update associated with the operation.
   */
  U getUpdate();

  /**
   * @return The extra arguments associated with the operation.
   */
  Object[] getArgs();
}

Retriable Table

AsyncRetriableTable currently uses Failsafe library for handling retries. Reads (get) and Writes (puts, deletes) each currently have a RetryPolicy and metrics associated. The metrics reported are retry count, success count, perm failure count and retry timer. We will reuse the Write RetryPolicy for updates as well and metrics reported would be the same as for writes.