Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-7510

KStreams RecordCollectorImpl leaks data to logs on error

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 2.2.0
    • streams

    Description

      org.apache.kafka.streams.processor.internals.RecordCollectorImpl leaks data on error as it dumps the value / message payload to the logs.

      This is problematic as it may contain personally identifiable information (pii) or other secret information to plain text log files which can then be propagated to other log systems i.e Splunk.

      I suggest the key, and value fields be moved to debug level as it is useful for some people while error level contains the errorMessage, timestamp, topic and stackTrace.

      private <K, V> void recordSendError(
          final K key,
          final V value,
          final Long timestamp,
          final String topic,
          final Exception exception
      ) {
          String errorLogMessage = LOG_MESSAGE;
          String errorMessage = EXCEPTION_MESSAGE;
          if (exception instanceof RetriableException) {
              errorLogMessage += PARAMETER_HINT;
              errorMessage += PARAMETER_HINT;
          }
          log.error(errorLogMessage, key, value, timestamp, topic, exception.toString());
          sendException = new StreamsException(
              String.format(
                  errorMessage,
                  logPrefix,
                  "an error caught",
                  key,
                  value,
                  timestamp,
                  topic,
                  exception.toString()
              ),
              exception);
      }

      Attachments

        Issue Links

          Activity

            People

              MrKafka Mr Kafka
              MrKafka Mr Kafka
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: