You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by GitBox <gi...@apache.org> on 2019/05/09 16:53:41 UTC

[GitHub] [samza] shanthoosh commented on a change in pull request #1008: SAMZA-2174: Throw a record too large exception for oversized records in changelog

shanthoosh commented on a change in pull request #1008: SAMZA-2174: Throw a record too large exception for oversized records in changelog
URL: https://github.com/apache/samza/pull/1008#discussion_r282570530
 
 

 ##########
 File path: samza-kv/src/main/scala/org/apache/samza/storage/kv/LoggedStore.scala
 ##########
 @@ -19,34 +19,44 @@
 
 package org.apache.samza.storage.kv
 
+import org.apache.samza.SamzaException
+import org.apache.samza.config.Config
 import org.apache.samza.util.Logging
 import org.apache.samza.system.{OutgoingMessageEnvelope, SystemStreamPartition}
 import org.apache.samza.task.MessageCollector
 
+import scala.collection.JavaConverters.asScalaBufferConverter
+
 /**
  * A key/value store decorator that adds a changelog for any changes made to the underlying store
  */
-class LoggedStore[K, V](
-  val store: KeyValueStore[K, V],
+class LoggedStore(
+  val store: KeyValueStore[Array[Byte], Array[Byte]],
+  val storeName: String,
+  val storeConfig: Config,
   val systemStreamPartition: SystemStreamPartition,
   val collector: MessageCollector,
-  val metrics: LoggedStoreMetrics = new LoggedStoreMetrics) extends KeyValueStore[K, V] with Logging {
+  val metrics: LoggedStoreMetrics = new LoggedStoreMetrics) extends KeyValueStore[Array[Byte], Array[Byte]] with Logging {
 
   val systemStream = systemStreamPartition.getSystemStream
   val partitionId = systemStreamPartition.getPartition.getPartitionId
 
+  private val DEFAULT_CHANGELOG_MAX_MSG_SIZE_BYTES = 1000000
 
 Review comment:
   Just curious. 
   
   Maximum message size accepted by a kafka broker is defined by the configuration: `message.max.bytes`.  There're new asynchronous API's added in kafka-AdminClient which allows the users to query the broker configuration through the `AdminClient` API. Broker configurations can be accessed using  `AdminClient.describeConfigs();` API
   
   Rather than hardcoding the maximum value from samza's-end, would it be beneficial to query the configured value directly once from the broker during container-startup. By that, for different kind of brokers(with different supported max-message-size), users might not have to tune this configuration correctly at samza-end. 
   
   What do you think?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services