ajothomas commented on a change in pull request #1560: URL: https://github.com/apache/samza/pull/1560#discussion_r787156643
########## File path: samza-core/src/main/java/org/apache/samza/operators/impl/SendToTableWithUpdateOperatorImpl.java ########## @@ -0,0 +1,123 @@ +/* + * 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.UpdateOptions; +import org.apache.samza.operators.UpdateMessage; +import org.apache.samza.operators.spec.OperatorSpec; +import org.apache.samza.operators.spec.SendToTableWithUpdateOperatorSpec; +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-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 SendToTableWithUpdateOperatorImpl<K, V, U> + extends OperatorImpl<KV<K, UpdateMessage<U, V>>, KV<K, UpdateMessage<U, V>>> { + private static final Logger LOG = LoggerFactory.getLogger(SendToTableWithUpdateOperatorImpl.class); + + private final SendToTableWithUpdateOperatorSpec<K, V, U> spec; + private final ReadWriteTable<K, V, U> table; + + public SendToTableWithUpdateOperatorImpl(SendToTableWithUpdateOperatorSpec<K, V, U> spec, Context context) { + this.spec = spec; + this.table = context.getTaskContext().getTable(spec.getTableId()); + if (context.getTaskContext().getTable(spec.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) { + } Review comment: Table is initialized by the `TableProvider` when we get it from `TaskContext`. Don't need to do a table.init() again here. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
